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 2022/08/04 15:12:18 UTC

[GitHub] [spark] EnricoMi opened a new pull request, #37407: [SPARK-39876][SQL][WIP] Add UNPIVOT to SQL syntax

EnricoMi opened a new pull request, #37407:
URL: https://github.com/apache/spark/pull/37407

   ### What changes were proposed in this pull request?
   This adds `UNPIVOT` SQL syntax. It follows the same syntax as  [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unpivot_operator), [T-SQL](https://docs.microsoft.com/en-us/sql/t-sql/queries/from-using-pivot-and-unpivot?view=sql-server-ver15#unpivot-example), [Oracle](https://www.oracletutorial.com/oracle-basics/oracle-unpivot/).
   
   ### Why are the changes needed?
   To support `UNPIVOR` in SQL syntax.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, adds `UNPIVOT` to SQL syntax.
   
   ### How was this patch tested?
   Added end-to-end tests to `SQLQueryTestSuite`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959705709


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1375,28 +1375,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],

Review Comment:
   Can I assume that `ids` and `values` must be the same after the plan is analyzed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r970574953


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3064,7 +3064,7 @@ def cube(self, *cols: "ColumnOrName") -> "GroupedData":  # type: ignore[misc]
 
     def unpivot(
         self,
-        ids: Optional[Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]]],
+        ids: Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]],

Review Comment:
   Just to differentiate the semantics of `ids` from `values`. When `values` is `None`, it means "magically" all non-id columns. When `ids` is `None`, it does not magically mean all non-value columns. Disallowing `None` for `ids` requires the user to explicitly state `ids=[]`, and that is what you get.
   
   Similar to the Scala / Java API where you have to provide `ids` as well, even if an empty `Array`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969630564


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3064,7 +3064,7 @@ def cube(self, *cols: "ColumnOrName") -> "GroupedData":  # type: ignore[misc]
 
     def unpivot(
         self,
-        ids: Optional[Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]]],
+        ids: Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]],

Review Comment:
   why is `ids` not optional any more?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972844049


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Yes, this implements `.unpivot(Array, String, String)`, when no value are given, similar to when no ids are given in SQL.
   
   This is the implementation of
   ```
      * Note: A column that is referenced by an id column expression is considered an id column itself.
      * For instance `$"id" * 2` references column `id`, so `id` is considered an id column and not a
      * value column
   ```
   
   I thought, the following would be annoying:
   ```
   scala> val df = spark.range(5).select($"id", ($"id"*10).as("val"))
   scala> df.show()
   +---+---+
   | id|val|
   +---+---+
   |  0|  0|
   |  1| 10|
   |  2| 20|
   |  3| 30|
   |  4| 40|
   +---+---+
   
   df.unpivot(Array($"id" * 2), "col", "val").show()
   +--------+---+---+
   |(id * 2)|col|val|
   +--------+---+---+
   |       0|id|  0|
   |       0|val|  0|
   |       2|id| 1|
   |       2|val| 10|
   |       4|id 2|
   |       4|val| 20|
   |       6|id 3|
   |       6|val| 30|
   |       8|id 4|
   |       8|val| 40|
   +--------+---+---+
   ```
   
   Of course, that id manipulation can be done before `unpivot` to "materialize" it as a reference:
   ```
   df.withColumn("id", $"id" * 2).unpivot(Array($"id"), "col", "val").show()
   +---+---+---+
   | id|col|val|
   +---+---+---+
   |  0|val|  0|
   |  2|val| 10|
   |  4|val| 20|
   |  6|val| 30|
   |  8|val| 40|
   +---+---+---+
   ```
   
   Happy to remove that complexity.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980005240


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Your proposal makes sense 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981170092


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),
+        None,
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    } else {
+      val unpivotClause = ctx.unpivotOperator().unpivotMultiValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val (unpivotColumns, unpivotAliases) =
+        unpivotClause.unpivotColumnSets.asScala.map(visitUnpivotColumnSet).toSeq.unzip
+
+      Unpivot(
+        None,
+        Some(unpivotColumns),
+        Some(unpivotAliases),
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    }
+
+    // exclude null values
+    val filtered = if (ctx.nullOperator != null && ctx.nullOperator.EXCLUDE() != null) {

Review Comment:
   Fixed in cbd5be96.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981173804


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -535,6 +548,98 @@ class DatasetUnpivotSuite extends QueryTest
         "val"),
       longStructDataRows)
   }
+
+  test("unpivot with struct expressions") {
+    checkAnswer(
+      wideDataDs.unpivot(
+        Array($"id"),
+        Array(
+          struct($"str1".as("str"), $"int1".cast(LongType).as("long")).as("str-int"),

Review Comment:
   Moved to #38019.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r983226707


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   I am still missing why we do not want to allow all `AttributeReference`s here, why shall we exclude `GetStructField` and any of the other `ExtractValue`s here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982215819


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   Method `visitUnpivotColumn` returns an `UnresolvedAttribute`, which turns into an `AttributeReference`. Otherwise the analyser fails.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on PR #37407:
URL: https://github.com/apache/spark/pull/37407#issuecomment-1270015416

   Thanks for the excellent code review and guidance!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r974946131


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1374,33 +1374,110 @@ case class Pivot(
   override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild)
 }
 
+/**
+ * Expression for [[Unpivot]] for one unpivot value column (one or more expressions)
+ * and an optional alias. This node itself is not evaluable and resolvable.
+ * Only its children are to be resolved.
+ *
+ * @param exprs expressions to unpivot
+ * @param alias optional alias
+ */
+case class UnpivotExpr(exprs: Seq[NamedExpression], alias: Option[String]) extends Unevaluable {
+  override val children: Seq[NamedExpression] = exprs
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+
+  // this expression is never resolved, it gets replaced together with its parent Unpivot plan
+  override lazy val resolved = false
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[Expression]): Expression = {
+    // turn expressions into named expressions
+    copy(exprs = newChildren.map {
+      case ne: NamedExpression => ne
+      case e: Expression => UnresolvedAlias(e)
+    })
+  }
+}
+
 /**
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1"), None),
+ *       (Seq("val2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), None),
+ *       (Seq("val2.1", "val2.2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), Some("val1")),
+ *       (Seq("val2.1", "val2.2"), Some("val2"))
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
- * @param values             Value columns to unpivot
+ * @param values             Value column sets to unpivot with optional aliases
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[UnpivotExpr]],

Review Comment:
   or we can do `values: Option[Seq[(Seq[NamedExpression], Option[String])]]` and update `QueryPlan.mapExpressions` to support `Tuple2`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988507447


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -552,6 +552,12 @@
       "Unable to acquire <requestedBytes> bytes of memory, got <receivedBytes>"
     ]
   },
+  "UNPIVOT_REQUIRES_ATTRIBUTES" : {
+    "message" : [
+      "UNPIVOT requires given {given} to be Attributes when no {empty} are given: [<types>]"

Review Comment:
   ```suggestion
         "UNPIVOT requires given {given} to be columns when no {empty} are given, but got: [<expressions>]"
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980958517


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),
+        None,
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    } else {
+      val unpivotClause = ctx.unpivotOperator().unpivotMultiValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val (unpivotColumns, unpivotAliases) =
+        unpivotClause.unpivotColumnSets.asScala.map(visitUnpivotColumnSet).toSeq.unzip
+
+      Unpivot(
+        None,
+        Some(unpivotColumns),
+        Some(unpivotAliases),
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    }
+
+    // exclude null values
+    val filtered = if (ctx.nullOperator != null && ctx.nullOperator.EXCLUDE() != null) {
+      Filter(IsNotNull(Coalesce(valueColumnNames.map(UnresolvedAttribute(_)))), unpivot)

Review Comment:
   so we only skip one row if all its values are null?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r970801714


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],
+    aliases: Option[Seq[Option[String]]],

Review Comment:
   Done in 3ae224e. The tuple does not make the code more readable, though.
   
   Now that `Unpivot.values: Option[Seq[(Seq[NamedExpression]), Option[String])]]` contains tuples, the attributes in those tuples are not resolved anymore. Where is that magic happening that resolved the attributes in `Unpivot.values: Option[Seq[Seq[NamedExpression])]]`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r971407420


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except

Review Comment:
   Ah I see, let's add an assert here to guarantee this assumption.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r970561575


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   Without an alias result contains string representation of the column name tuples:
   https://github.com/apache/spark/pull/37407/files#diff-f306d1dfcd20c4cdcdb5d631416ee9d92bf6893422b0a9683ca4493d7215e5e8R199-R212
   ```
   -- !query
   SELECT * FROM courseEarningsAndSales
   UNPIVOT (
     (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))
   )
   -- !query schema
   struct<course:string,year:string,earnings:int,sales:int>
   -- !query output
   Java	(`earnings2012`,`sales2012`)	20000	1
   Java	(`earnings2013`,`sales2013`)	30000	2
   Java	(`earnings2014`,`sales2014`)	NULL	NULL
   dotNET	(`earnings2012`,`sales2012`)	15000	2
   dotNET	(`earnings2013`,`sales2013`)	48000	1
   dotNET	(`earnings2014`,`sales2014`)	22500	1
   ```
   
   If users are happy with that, why forcing an alias?
   
   This is optional in [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unpivot_operator) as well, the other SQL flavours mentioned in the PR description are not specific in that 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r970547787


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3082,6 +3082,13 @@ def unpivot(
         When no "id" columns are given, the unpivoted DataFrame consists of only the
         "variable" and "value" columns.
 
+        The `values` columns must not be empty so at least one value must be given to be unpivoted.
+        When `values` is `None`, all non-id columns will be unpivoted.
+
+        Note: A column that is referenced by an id column expression is considered an id column
+        itself. For instance `col("id") * 2` references column `id`, so `id` is considered an
+        id column and not a value column.

Review Comment:
   No, it is not, `pyspark.pandas.DataFrame.melt` compares value column candidates literally with id column names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r965653249


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   ping @EnricoMi 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972803465


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3082,6 +3082,13 @@ def unpivot(
         When no "id" columns are given, the unpivoted DataFrame consists of only the
         "variable" and "value" columns.
 
+        The `values` columns must not be empty so at least one value must be given to be unpivoted.
+        When `values` is `None`, all non-id columns will be unpivoted.
+
+        Note: A column that is referenced by an id column expression is considered an id column
+        itself. For instance `col("id") * 2` references column `id`, so `id` is considered an
+        id column and not a value column.

Review Comment:
   Is this to match other SQL databases?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969627592


##########
docs/sql-ref-syntax-qry-select-unpivot.md:
##########
@@ -0,0 +1,142 @@
+---
+layout: global
+title: UNPIVOT Clause
+displayTitle: UNPIVOT Clause
+license: |
+  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.
+---
+
+### Description
+
+The `UNPIVOT` clause transforms multiple columns into multiple rows used in `SELECT` clause. The `UNPIVOT` clause can be specified after the table name or subquery.
+
+### Syntax
+
+```sql
+UNPIVOT [ { INCLUDE | EXCLUDE } NULLS ] (
+    { single_value_column_unpivot | multi_value_column_unpivot }
+) [[AS] alias]
+
+single_value_column_unpivot:
+    values_column
+    FOR name_column
+    IN (unpivot_column [, ...])
+
+multi_value_column_unpivot:
+    (values_column [, ...])
+    FOR name_column
+    IN ((unpivot_column [, ...]) [[AS] alias] [, ...])

Review Comment:
   ```suggestion
       IN ((unpivot_column [, ...]) [AS] alias [, ...])
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972862941


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   For DataFrame API, we'd better follow pandas even if it's annoying... SQL API is a different story. Do other databases have the same behavior as this PR does when id columns are not specified?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980941315


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq

Review Comment:
   shall we respect the column order in the relation? how about
   ```
   val idAttrs = AttributeSet(up.ids.get)
   val values = up.child.output.filterNot(idAttrs.contains)
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
+      case up: Unpivot if up.childrenResolved &&
+        up.values.exists(_.forall(_.forall(_.resolved))) && up.ids.isEmpty =>
+        val ids = (AttributeSet(up.child.output) -- AttributeSet(up.values.get.flatten)).toSeq

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980939958


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq

Review Comment:
   I think `AttributeSet` is fine because we only allow attributes now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982381621


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   `SELECT a.b.c` also produce `UnresolvedAttribute`, and it may end up with column `c` in table `b` under database `a` (thus it's `AttributeReference`), or nested column `c` of column `b` in table `a` (thus it's `GetStructField`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982631649


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   Yea, because there is no ambiguity in doing so.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981218176


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   regarding https://github.com/apache/spark/pull/37407/files#r981213360 , I think a simple rule is, we require id/value columns to be `Attribute` only, if only one is specified and Spark needs to infer the other. Thus, users can still use arbitrary expressions as id/value columns if they specify both.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r963266219


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   If this is a spark-specific syntax, shall we make it simple and don't allow per-expression alias here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r967091275


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   `expression` does not allow for an alias, `namedExpression` does:
   
   ```g4
   namedExpression
       : expression (AS? (name=errorCapturingIdentifier | identifierList))?
       ;
   ```
   
   I have changed that in 004bb692.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988508252


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -564,6 +570,12 @@
     ],
     "sqlState" : "42000"
   },
+  "UNPIVOT_VALUE_SIZE_MISMATCH" : {
+    "message" : [
+      "All unpivot value columns must have the same size as there are value column names (<names>): [<sizes>]"

Review Comment:
   We probably don't need to put `<sizes>`, as it's very clear from the SQL statement.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959695554


##########
docs/sql-ref-syntax-qry-select-unpivot.md:
##########
@@ -0,0 +1,142 @@
+---
+layout: global
+title: UNPIVOT Clause
+displayTitle: UNPIVOT Clause
+license: |
+  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.
+---
+
+### Description
+
+The `UNPIVOT` clause transforms multiple columns into multiple rows used in `SELECT` clause. The `UNPIVOT` clause can be specified after the table name or subquery.
+
+### Syntax
+
+```sql
+UNPIVOT [ { INCLUDE | EXCLUDE } NULLS ] (
+    { single_value_column_unpivot | multi_value_column_unpivot }
+) [[AS] alias]
+
+single_value_column_unpivot:
+    values_column
+    FOR name_column
+    IN (unpivot_column [[AS] alias] [, ...])
+
+multi_value_column_unpivot:
+    (values_column [, ...])
+    FOR name_column
+    IN ((unpivot_column [[AS] alias] [, ...]) [, ...])

Review Comment:
   BTW, I think alias is required here? otherwise I have no idea what should be the name of things like `(q1, q2)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959961418


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   Yes, `unpivotColumn` itself is defined as `expression (AS? identifier)?`, so each individual column can have an alias, and the entire set can. However, with an alias for the set, the alias of the column is hidden. But the syntax is valid.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959962403


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1375,28 +1375,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],

Review Comment:
   Same in which sense? They are usually disjoint, but can overlap if the user sets them explicitly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969645973


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.

Review Comment:
   since value columns cannot be Nil, can we use `values: Seq[Seq[NamedExpression]]` and then set it as `Nil` to indicate that the value columns are to be inferred?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969636837


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   I think the alias at the end is not optional but must-have?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972860057


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1374,32 +1374,104 @@ case class Pivot(
   override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild)
 }
 
+/**
+ * Expression for [[Unpivot]] for one unpivot value column (one or more expressions)
+ * and an optional alias. This node itself is not evaluable and resolvable.
+ * Only its children are to be resolved.
+ *
+ * @param exprs expressions to unpivot
+ * @param alias optional alias
+ */
+case class UnpivotExpr(exprs: Seq[NamedExpression], alias: Option[String]) extends Unevaluable {

Review Comment:
   This reminds me of `ArraysZip`, where we need to enforce named expressions.
   
   We can follow it to avoid analyzer hacks here and there
   ```
   case class UnpivotExpr(exprs: Seq[Expression], names: Seq[Expression], alias: Option[String]) extends Unevaluable {
     def this(exprs: Seq[Expression], alias: Option[String]) = {
       this(exprs, exprs.map {
         case u: UnresolvedAttribute => Literal(u.nameParts.last)
         case e: NamedExpression if e.resolved => Literal(e.name)
         ...
       }, alias)
     }
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988768918


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up

Review Comment:
   parser tests should check the unresolved plan (the raw parsed plan). feel free to add the parser test suite in a followup PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988687011


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -552,6 +552,12 @@
       "Unable to acquire <requestedBytes> bytes of memory, got <receivedBytes>"
     ]
   },
+  "UNPIVOT_REQUIRES_ATTRIBUTES" : {
+    "message" : [
+      "UNPIVOT requires given {given} to be Attributes when no {empty} are given: [<types>]"

Review Comment:
   Fixed in 2c8f53df, though `but got` implies `expressions` is an exhaustive list, but it is only the non-attributes.
   
   I have rephrased that to
   
       ... expressions are given. These are not columns: [<expressions>].



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988687754


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -101,16 +101,45 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
         "pivotType" -> pivotCol.dataType.catalogString))
   }
 
+  def unpivotRequiresAttributes(given: String,
+                                empty: String,
+                                expressions: Seq[NamedExpression]): Throwable = {

Review Comment:
   fixed in 2c8f53df



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988508483


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,52 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given, and only AttributeReference are given
+      case up @Unpivot(Some(ids), None, _, _, _, _) if up.childrenResolved &&

Review Comment:
   ```suggestion
         case up @ Unpivot(Some(ids), None, _, _, _, _) if up.childrenResolved &&
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r973076852


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   I mean, how other databases infer the id columns? It's `allOutput -- valueColumns` or `allOutput -- valueColumns.flatMap(_.references)`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   I mean, how do other databases infer the id columns? It's `allOutput -- valueColumns` or `allOutput -- valueColumns.flatMap(_.references)`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980944809


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
+      case up: Unpivot if up.childrenResolved &&
+        up.values.exists(_.forall(_.forall(_.resolved))) && up.ids.isEmpty =>
+        val ids = (AttributeSet(up.child.output) -- AttributeSet(up.values.get.flatten)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(ids = Some(up.child.output.intersect(ids)))
+
+      case up: Unpivot if !up.childrenResolved || !up.ids.exists(_.forall(_.resolved)) ||
+        !up.values.exists(_.nonEmpty) || !up.values.exists(_.forall(_.forall(_.resolved))) ||
+        !up.values.get.forall(_.length == up.valueColumnNames.length) ||
+        !up.valuesTypeCoercioned => up
 
       // TypeCoercionBase.UnpivotCoercion determines valueType
       // and casts values once values are set and resolved
-      case Unpivot(ids, values, variableColumnName, valueColumnName, child) =>
+      case Unpivot(Some(ids), Some(values), aliases, variableColumnName, valueColumnNames, child) =>
+
+        def toString(values: Seq[NamedExpression]): String =
+          "(" + values.map(v => quoteIdentifier(v.name)).mkString(",") + ")"

Review Comment:
   does this match other databases?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] HyukjinKwon commented on pull request #37407: [SPARK-39876][SQL][WIP] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on PR #37407:
URL: https://github.com/apache/spark/pull/37407#issuecomment-1205907899

   cc @maryannxue FYI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] MaxGekk commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r956604761


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -310,9 +327,9 @@ class DatasetUnpivotSuite extends QueryTest
       errorClass = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH",
       parameters = Map(
         "types" ->
-          (""""BIGINT" \(`long1#\d+L`, `long2#\d+L`\), """ +
-           """"INT" \(`int1#\d+`, `int2#\d+`, `int3#\d+`, ...\), """ +
-           """"STRING" \(`str1#\d+`\)""")),
+          (""""BIGINT" \(`long1`, `long2`\), """ +
+           """"INT" \(`int1`, `int2`, `int3`, ...\), """ +
+           """"STRING" \(`str1`\)""")),
       matchPVals = true)

Review Comment:
   `matchPVals = true` is not needed if you are sure that regexp is not needed, here. Just in case, could you explain why did you replace the regexps.



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -2094,6 +2095,23 @@ class Dataset[T] private[sql](
    * This is equivalent to calling `Dataset#unpivot(Array, Array, String, String)`
    * where `values` is set to all non-id columns that exist in the DataFrame.
    *
+   * Note: A column that is referenced by an id column expression is considered an id column itself.
+   * For instance `$"id" * 2` references column `id`, so `id` is considered an id column and not a
+   * value column:
+   * {{{
+   *   val df = Seq((1, 11, 12L), (2, 21, 22L)).toDF("id", "int", "long")
+   *   df.unpivot(Array($"id" * 2), "var", "val").show()
+   *   // output:
+   *   // +--------+--------+-----+
+   *   // |(id * 2)|variable|value|
+   *   // +--------+--------+-----+
+   *   // |       2|     int|   11|
+   *   // |       2|    long|   12|
+   *   // |       4|     int|   21|
+   *   // |       4|    long|   22|
+   *   // +--------+--------+-----+

Review Comment:
   Column names are different slightly:
   ```scala
   scala> df.unpivot(Array($"id" * 2), "var", "val").show()
   +--------+----+---+
   |(id * 2)| var|val|
   +--------+----+---+
   |       2| int| 11|
   |       2|long| 12|
   |       4| int| 21|
   |       4|long| 22|
   +--------+----+---+
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959961704


##########
docs/sql-ref-syntax-qry-select-unpivot.md:
##########
@@ -0,0 +1,142 @@
+---
+layout: global
+title: UNPIVOT Clause
+displayTitle: UNPIVOT Clause
+license: |
+  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.
+---
+
+### Description
+
+The `UNPIVOT` clause transforms multiple columns into multiple rows used in `SELECT` clause. The `UNPIVOT` clause can be specified after the table name or subquery.
+
+### Syntax
+
+```sql
+UNPIVOT [ { INCLUDE | EXCLUDE } NULLS ] (
+    { single_value_column_unpivot | multi_value_column_unpivot }
+) [[AS] alias]
+
+single_value_column_unpivot:
+    values_column
+    FOR name_column
+    IN (unpivot_column [[AS] alias] [, ...])
+
+multi_value_column_unpivot:
+    (values_column [, ...])
+    FOR name_column
+    IN ((unpivot_column [[AS] alias] [, ...]) [, ...])

Review Comment:
   You are right, fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972848355


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3082,6 +3082,13 @@ def unpivot(
         When no "id" columns are given, the unpivoted DataFrame consists of only the
         "variable" and "value" columns.
 
+        The `values` columns must not be empty so at least one value must be given to be unpivoted.
+        When `values` is `None`, all non-id columns will be unpivoted.
+
+        Note: A column that is referenced by an id column expression is considered an id column
+        itself. For instance `col("id") * 2` references column `id`, so `id` is considered an
+        id column and not a value column.

Review Comment:
   No, see https://github.com/apache/spark/pull/37407/files#r972844049.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r974925706


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1374,32 +1374,104 @@ case class Pivot(
   override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild)
 }
 
+/**
+ * Expression for [[Unpivot]] for one unpivot value column (one or more expressions)
+ * and an optional alias. This node itself is not evaluable and resolvable.
+ * Only its children are to be resolved.
+ *
+ * @param exprs expressions to unpivot
+ * @param alias optional alias
+ */
+case class UnpivotExpr(exprs: Seq[NamedExpression], alias: Option[String]) extends Unevaluable {
+  override val children: Seq[NamedExpression] = exprs
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  // override lazy val resolved = false
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[Expression]): Expression = {
+    // turn expressions into named expressions
+    copy(exprs = newChildren.map {
+      case ne: NamedExpression => ne
+      case e: Expression => UnresolvedAlias(e)
+    })
+  }
+}
+
 /**
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1"), None),
+ *       (Seq("val2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), None),
+ *       (Seq("val2.1", "val2.2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), Some("val1")),
+ *       (Seq("val2.1", "val2.2"), Some("val2"))
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
- * @param values             Value columns to unpivot
+ * @param values             Value column sets to unpivot with optional aliases
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[UnpivotExpr]],
     variableColumnName: String,
-    valueColumnName: String,
+    valueColumnNames: Seq[String],
     child: LogicalPlan) extends UnaryNode {
   override lazy val resolved = false  // Unpivot will be replaced after being resolved.

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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969627592


##########
docs/sql-ref-syntax-qry-select-unpivot.md:
##########
@@ -0,0 +1,142 @@
+---
+layout: global
+title: UNPIVOT Clause
+displayTitle: UNPIVOT Clause
+license: |
+  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.
+---
+
+### Description
+
+The `UNPIVOT` clause transforms multiple columns into multiple rows used in `SELECT` clause. The `UNPIVOT` clause can be specified after the table name or subquery.
+
+### Syntax
+
+```sql
+UNPIVOT [ { INCLUDE | EXCLUDE } NULLS ] (
+    { single_value_column_unpivot | multi_value_column_unpivot }
+) [[AS] alias]
+
+single_value_column_unpivot:
+    values_column
+    FOR name_column
+    IN (unpivot_column [, ...])
+
+multi_value_column_unpivot:
+    (values_column [, ...])
+    FOR name_column
+    IN ((unpivot_column [, ...]) [[AS] alias] [, ...])

Review Comment:
   ```suggestion
       IN ((unpivot_column [, ...]) [AS] alias [, ...])
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969645973


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.

Review Comment:
   since value columns cannot be Nil, can we use still use `values: Seq[NamedExpression]` and then set it as `Nil` to indicate that the value columns are to be inferred?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969633374


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3082,6 +3082,13 @@ def unpivot(
         When no "id" columns are given, the unpivoted DataFrame consists of only the
         "variable" and "value" columns.
 
+        The `values` columns must not be empty so at least one value must be given to be unpivoted.
+        When `values` is `None`, all non-id columns will be unpivoted.
+
+        Note: A column that is referenced by an id column expression is considered an id column
+        itself. For instance `col("id") * 2` references column `id`, so `id` is considered an
+        id column and not a value column.

Review Comment:
   @HyukjinKwon is this behavior the same as pandas?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980960360


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -2118,6 +2127,16 @@ class Dataset[T] private[sql](
       valueColumnName: String): DataFrame =
     unpivot(ids.toArray, values.toArray, variableColumnName, valueColumnName)
 
+  /**
+   * Called from Python as Seq[Column] are easier to create via py4j than Array[Column].
+   * We use Array[Column] for unpivot rather than Seq[Column] as those are Java-friendly.
+   */
+  private[sql] def unpivotWithSeq(

Review Comment:
   This method is not used.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980984200


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq

Review Comment:
   ```scala
   // up.child.output.intersect preserves order of columns
   up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
   ```
   does preserve the column order, but `filterNot` is more concise.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988501693


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -101,16 +101,45 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
         "pivotType" -> pivotCol.dataType.catalogString))
   }
 
+  def unpivotRequiresAttributes(given: String,
+                                empty: String,
+                                expressions: Seq[NamedExpression]): Throwable = {
+    val types = expressions.groupBy(_.getClass.getSimpleName)
+      .mapValues(exprs => exprs.map(expr => expr.toString.replaceAll("#\\d+", "")).sorted)
+      .mapValues(exprs => if (exprs.length > 3) exprs.take(3) :+ "..." else exprs)
+      .toList.sortBy(_._1)
+      .map { case (className, exprs) => s"$className (${exprs.mkString(", ")})" }

Review Comment:
   We should not expose too many internal information in the user-facing error message. Can we just put `expressions.filterNot(_.isInstanceOf[Attribute]).map(_.sql).mkString(", ")`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980878546


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq

Review Comment:
   @cloud-fan looks like `AttributeSet` treats `$"id"`, `$"id" * 2`, and `$"id".as("uid")` as equal.
   
   This is why `id` rows disappear from
   
   ```scala
   df.unpivot(Array($"id" * 2), "var", "val")
   ```
   https://github.com/apache/spark/pull/37407/files#diff-2a4366c38acddfb5e0f468e88411679ccb3c081369a457606b3460e0e78d29d6R165
   
   and
   
   ```scala
   df.unpivot(Array($"id".as("uid")), "var", "val")
   ```
   https://github.com/apache/spark/pull/37407/files#diff-2a4366c38acddfb5e0f468e88411679ccb3c081369a457606b3460e0e78d29d6R181
   
   I am using `AttributeSet -- AttributeSet` here because `up.child.output.diff(up.ids))` does not work as `up.child.output` contains different `AttributeReference` _instances_ than `up.ids`, though they got resolved to the same attributes. The `diff` does not remove any `ids` from `up.child.output`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r974929159


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3064,7 +3064,7 @@ def cube(self, *cols: "ColumnOrName") -> "GroupedData":  # type: ignore[misc]
 
     def unpivot(
         self,
-        ids: Optional[Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]]],
+        ids: Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]],

Review Comment:
   I think we can allow None as id columns in the pandas on Spark API, but not pyspark. cc @HyukjinKwon @xinrong-meng 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972808194


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3091,12 +3098,12 @@ def unpivot(
 
         Parameters
         ----------
-        ids : str, Column, tuple, list, optional
+        ids : str, Column, tuple, list
             Column(s) to use as identifiers. Can be a single column or column name,
             or a list or tuple for multiple columns.
         values : str, Column, tuple, list, optional
             Column(s) to unpivot. Can be a single column or column name, or a list or tuple
-            for multiple columns. If not specified or empty, uses all columns that
+            for multiple columns. Must not be empty. If None, uses all columns that

Review Comment:
   ```suggestion
               for multiple columns. If specified, must not be empty. If not specified, uses all columns that
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972860753


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1374,32 +1374,104 @@ case class Pivot(
   override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild)
 }
 
+/**
+ * Expression for [[Unpivot]] for one unpivot value column (one or more expressions)
+ * and an optional alias. This node itself is not evaluable and resolvable.
+ * Only its children are to be resolved.
+ *
+ * @param exprs expressions to unpivot
+ * @param alias optional alias
+ */
+case class UnpivotExpr(exprs: Seq[NamedExpression], alias: Option[String]) extends Unevaluable {
+  override val children: Seq[NamedExpression] = exprs
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+  // override lazy val resolved = false
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[Expression]): Expression = {
+    // turn expressions into named expressions
+    copy(exprs = newChildren.map {
+      case ne: NamedExpression => ne
+      case e: Expression => UnresolvedAlias(e)
+    })
+  }
+}
+
 /**
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1"), None),
+ *       (Seq("val2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), None),
+ *       (Seq("val2.1", "val2.2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), Some("val1")),
+ *       (Seq("val2.1", "val2.2"), Some("val2"))
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
- * @param values             Value columns to unpivot
+ * @param values             Value column sets to unpivot with optional aliases
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[UnpivotExpr]],
     variableColumnName: String,
-    valueColumnName: String,
+    valueColumnNames: Seq[String],
     child: LogicalPlan) extends UnaryNode {
   override lazy val resolved = false  // Unpivot will be replaced after being resolved.

Review Comment:
   let's add `assert(ids.isDefined || values.isDefined)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on PR #37407:
URL: https://github.com/apache/spark/pull/37407#issuecomment-1249106578

   @cloud-fan I have introduced expression `UnpivotExpr` to replace the `(Seq[NamedExpression], Option[String]])`, which makes code more readable.
   
   But, this introduces the following change in behaviour / deviation from projection behaviour:
   
   ```scala
   spark.range(5).select(struct($"id").as("an")).select($"an.id").show()
   ```
   
   "an.id" gets alias "id":
   ```
   +---+
   | id|
   +---+
   |  0|
   |  1|
   |  2|
   |  3|
   |  4|
   +---+
   ```
   
   ```
   Project(UnresolvedAttribute("an.id"), plan)
     --> ResolveReferences rule -->
   Project(Alias(GetStructField(an#2.id), "id"), plan)
   ```
   
   ```scala
   spark.range(5).select(struct($"id").as("an")).unpivot(Array($"an.id"), Array($"an.id"), "col", "val").show()
   ```
   
   before introducing `UnpivotExpr`, both ids and values get alias "id" (as in select / `Project`):
   ```
   +---+---+---+
   | id|col|val|
   +---+---+---+
   |  0| id|  0|
   |  1| id|  1|
   |  2| id|  2|
   |  3| id|  3|
   |  4| id|  4|
   +---+---+---+
   ```
   
   after introducing `UnpivotExpr`, id "str.id" gets alias "id", value "str.id" does not get an alias and hence gets name "an.id":
   ```
   +---+-----+---+
   | id|  col|val|
   +---+-----+---+
   |  0|an.id|  0|
   |  1|an.id|  1|
   |  2|an.id|  2|
   |  3|an.id|  3|
   |  4|an.id|  4|
   +---+-----+---+
   ```
   
   Now that `UnpivotExpr` is the top level expression, inner `UnresolvedAttribute` / `GetStructField` does not get an alias:
   
   ```
   Unpivot(Seq(UnresolvedAttribute("an.id")), Seq(UnpivotExpr(Seq(UnresolvedAttribute("an.id")), ...)), ..., plan)
     --> ResolveReferences -->
   Unpivot(Seq(Alias(GetStructField(an#2.id), "id")), Seq(UnpivotExpr(Seq(GetStructField(an#2.id)), ...)), ..., plan)
   ```
   
   https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L1770
   
   `CleanupAliases` rule is not the reason, the alias is being removed inside `ResolveReferences`.
   
   The only way to get to the old behaviour is a special treatment of `UnpivotExpr` in `QueryPlan.mapExpressions.recursiveTransform`:
   https://github.com/apache/spark/pull/37407/commits/9dd66b78ec817a53325d95900f18198dac9bc3b1#diff-ece55283a94dd23d3c04f8b9d8ae35937ccff67724be690ff30f76e9f8093c6eR211


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r958678393


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -310,9 +327,9 @@ class DatasetUnpivotSuite extends QueryTest
       errorClass = "UNPIVOT_VALUE_DATA_TYPE_MISMATCH",
       parameters = Map(
         "types" ->
-          (""""BIGINT" \(`long1#\d+L`, `long2#\d+L`\), """ +
-           """"INT" \(`int1#\d+`, `int2#\d+`, `int3#\d+`, ...\), """ +
-           """"STRING" \(`str1#\d+`\)""")),
+          (""""BIGINT" \(`long1`, `long2`\), """ +
+           """"INT" \(`int1`, `int2`, `int3`, ...\), """ +
+           """"STRING" \(`str1`\)""")),
       matchPVals = true)

Review Comment:
   good spot, this is not needed anymore, fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r977818035


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   @cloud-fan @HyukjinKwon
   
   Scala `Dataset.groupBy.pivot` allows for expressions, whereas PySpark SQL (`pyspark.sql.DataFrame`) and PySpark Pandas (`pyspark.pandas.DataFrame`) allow for column names only. So we are already mixing expressions and column names.
   
   Scala `Dataset.unpivot` should allow for (named) expressions (as it currently does), to mirror Scala `Dataset.groupBy.pivot`.
   SQL should allow for column names only, as all other databases do.
   I am fine with downgrading PySpark SQL from allowing `ColumnOrName`to `str` to mirror existing PySpark Pandas `pyspark.pandas.DataFrame.melt`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r983396003


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   Done in 70f986a.
   
   I think the error message needs some improvement, it is likely not clear to users what is meant with `Attribute`:
   
       "UNPIVOT requires given {given} to be Attributes when no {empty} are given: [<types>]"
   
   Maybe `column names` is better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r983393051


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   fine with me, closing #38020.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981083737


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),
+        None,
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    } else {
+      val unpivotClause = ctx.unpivotOperator().unpivotMultiValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val (unpivotColumns, unpivotAliases) =
+        unpivotClause.unpivotColumnSets.asScala.map(visitUnpivotColumnSet).toSeq.unzip
+
+      Unpivot(
+        None,
+        Some(unpivotColumns),
+        Some(unpivotAliases),
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    }
+
+    // exclude null values
+    val filtered = if (ctx.nullOperator != null && ctx.nullOperator.EXCLUDE() != null) {
+      Filter(IsNotNull(Coalesce(valueColumnNames.map(UnresolvedAttribute(_)))), unpivot)

Review Comment:
   [Oracle](https://www.oracletutorial.com/oracle-basics/oracle-unpivot/) is not specific about this, nor provides it an example that exemplifies NULL values with multiple unpivot columns:
   
       The EXCLUDE NULLS clause, on the other hand, eliminates null-valued rows from the returned result set.
   
   [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unpivot_operator)
   
       EXCLUDE NULLS: Do not add rows with NULL values to the result.
   
   From [here](https://stackoverflow.com/questions/10747355/oracle-11g-unpivot-multiple-columns-and-include-column-name) it looks like Oracle behaves the same.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980870200


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   This has been done for SQL:  b6b31a0, 4c01645



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981168595


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq

Review Comment:
   Done in bd33daad.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
+      case up: Unpivot if up.childrenResolved &&
+        up.values.exists(_.forall(_.forall(_.resolved))) && up.ids.isEmpty =>
+        val ids = (AttributeSet(up.child.output) -- AttributeSet(up.values.get.flatten)).toSeq

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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981171196


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -535,6 +548,98 @@ class DatasetUnpivotSuite extends QueryTest
         "val"),
       longStructDataRows)
   }
+
+  test("unpivot with struct expressions") {
+    checkAnswer(
+      wideDataDs.unpivot(
+        Array($"id"),
+        Array(
+          struct($"str1".as("str"), $"int1".cast(LongType).as("long")).as("str-int"),

Review Comment:
   Removed in 8ed5d460.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988700017


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up

Review Comment:
   Those parser suites parse the SQL statement and assert the logical plan, but the plan is not fully analyzed. Some of the situations tested in `unpivot.sql` require full analysis, so they cannot be covered by `UnpivotParserSuite`.
   
   I could add those to `DatasetUnpivotSuite` and assert the result of `spark.sql("...")`.
   
   I'll sketch that out so we can see how this looks like.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988687537


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,52 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given, and only AttributeReference are given
+      case up @Unpivot(Some(ids), None, _, _, _, _) if up.childrenResolved &&

Review Comment:
   fixed



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,52 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given, and only AttributeReference are given
+      case up @Unpivot(Some(ids), None, _, _, _, _) if up.childrenResolved &&
+        ids.forall(_.resolved) &&
+        ids.forall(_.isInstanceOf[AttributeReference]) =>
+        val idAttrs = AttributeSet(up.ids.get)
+        val values = up.child.output.filterNot(idAttrs.contains)
+        up.copy(values = Some(values.map(Seq(_))))
+      case up @Unpivot(None, Some(values), _, _, _, _) if up.childrenResolved &&

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax
URL: https://github.com/apache/spark/pull/37407


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988867034


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up

Review Comment:
   I have cleaned up `unpivot.sql`: 84a02b67
   
   Will add the removed tests to `UnpivotParserSuite` and `DatasetUnpivotSuite` in a follow-up PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959969544


##########
docs/sql-ref-syntax-qry-select.md:
##########
@@ -75,7 +76,7 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { [ [ named_expression | regex_column_
 
     An expression with an assigned name. In general, it denotes a column expression.
 
-    **Syntax:** `expression [AS] [alias]`
+    **Syntax:** `expression [[AS] alias]`

Review Comment:
   @cloud-fan this was incorrect I think



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969644167


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except

Review Comment:
   When can the `ids` be `Nonw`? Seems all the user-facing APIs require users to specify id 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969659692


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except

Review Comment:
   what if `ids` and `values` are both None?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969644167


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except

Review Comment:
   When can the `ids` be `Nonw`? Seems all the user-facing APIs require users to specify id 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r962557915


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1375,28 +1375,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],

Review Comment:
   Yes, the `ResolveUnpivot` rule in `Analyzer` resolves `ids` and `values` to some and then replaces the `Unpivot` with the equivalent `Expand`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980965661


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -137,48 +138,49 @@ class DatasetUnpivotSuite extends QueryTest
     assert(unpivoted.schema === longSchema)
     checkAnswer(unpivoted, longDataRows)
 
-    val unpivoted2 = wideDataDs.select($"id", $"str1", $"str2")
-      .unpivot(
-        Array($"id"),
-        Array.empty,
-        variableColumnName = "var",
-        valueColumnName = "val")
-    assert(unpivoted2.schema === longSchema)
-    checkAnswer(unpivoted2, longDataRows)
+    val e = intercept[AnalysisException] {
+      wideDataDs.select($"id", $"str1", $"str2")
+        .unpivot(
+          Array($"id"),
+          Array.empty,
+          variableColumnName = "var",
+          valueColumnName = "val")
+    }
+    checkError(
+      exception = e,
+      errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS",

Review Comment:
   This is arguable. Since value columns can't be `Nil`, it seems reasonable to pass `Array.empty` and expect Spark to infer the value 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982382485


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   For all APIs, so that we can put the check in the catalyst rule, not in these APIs. SQL is special because it can't specify id 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982515423


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   I am not sure what we want to support here:
   
   ```
   SELECT *
   FROM a.b
   UNPIVOT (
     val for col in (c)
   )
   ```
   when `a` is a database and `b` is a table.
   
   ```
   SELECT *
   FROM a.b.*
   UNPIVOT (
     val for col in (c)
   )
   ```
   when `a` is a table and `b` is a struct.
   
   In either way, `c` is an `AttributeReference`, right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982517306


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   I am confused. You are saying it would be OK for PySpark `unpivot` to take expressions as long as both, `ids` and `values` are given, even though `pivot` and `pandas.melt` take only column names?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982352630


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   Interesting, so only in this special case we restrict the expressions to be attributes, because this is the only case where we need this constraint.
   
   Are we speaking Scala API only or Scala, Python and SQL syntax?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981213360


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   How can we make sure these value columns will end up with `Attribute`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r974039852


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Then why don't we follow other databases? The previous code `up.child.output.diff(up.ids)` is corrected.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972828426


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   we need to justify this behavior change. Can you give some examples of other SQL databases?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r970593177


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],
+    aliases: Option[Seq[Option[String]]],

Review Comment:
   Yes, this then guarantees that values and aliases have the same length. Which again, currently does not happen in the code but as code changes, there is no assertion yet.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r974944845


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1374,33 +1374,110 @@ case class Pivot(
   override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild)
 }
 
+/**
+ * Expression for [[Unpivot]] for one unpivot value column (one or more expressions)
+ * and an optional alias. This node itself is not evaluable and resolvable.
+ * Only its children are to be resolved.
+ *
+ * @param exprs expressions to unpivot
+ * @param alias optional alias
+ */
+case class UnpivotExpr(exprs: Seq[NamedExpression], alias: Option[String]) extends Unevaluable {
+  override val children: Seq[NamedExpression] = exprs
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+
+  // this expression is never resolved, it gets replaced together with its parent Unpivot plan
+  override lazy val resolved = false
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[Expression]): Expression = {
+    // turn expressions into named expressions
+    copy(exprs = newChildren.map {
+      case ne: NamedExpression => ne
+      case e: Expression => UnresolvedAlias(e)
+    })
+  }
+}
+
 /**
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1"), None),
+ *       (Seq("val2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), None),
+ *       (Seq("val2.1", "val2.2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), Some("val1")),
+ *       (Seq("val2.1", "val2.2"), Some("val2"))
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
- * @param values             Value columns to unpivot
+ * @param values             Value column sets to unpivot with optional aliases
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[UnpivotExpr]],

Review Comment:
   Sorry for the back and forth, but after reading the source code more, the alias problem is a bit hard to solve. Today Spark has 2 frameworks to enforce the alias:
   1. allow a list of operators to hold `NamedExpression`s, such as `Project` and `Aggregate`. The rule `CleanupAliases` handles these operators specially to not remove top-level aliases, to keep `NamedExpression`
   2. allow a list of expressions to hold alias names, such as `ArraysZip` and `CreateStruct`. These expressions will contain `NamePlaceholder` first, and then the rule `ResolveExpressionsWithNamePlaceholders` will match these expressions and resolve `NamePlaceholder`.
   
   `Unpivot` is an operator and it fits the first framework better. So it seems simpler to use `values: Option[Seq[Seq[NamedExpression]]]` with an extra `aliases: Seq[Option[String]]` parameter.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988686782


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -101,16 +101,45 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
         "pivotType" -> pivotCol.dataType.catalogString))
   }
 
+  def unpivotRequiresAttributes(given: String,
+                                empty: String,
+                                expressions: Seq[NamedExpression]): Throwable = {
+    val types = expressions.groupBy(_.getClass.getSimpleName)
+      .mapValues(exprs => exprs.map(expr => expr.toString.replaceAll("#\\d+", "")).sorted)
+      .mapValues(exprs => if (exprs.length > 3) exprs.take(3) :+ "..." else exprs)
+      .toList.sortBy(_._1)
+      .map { case (className, exprs) => s"$className (${exprs.mkString(", ")})" }

Review Comment:
   Done in 2c8f53df, though `.map(toSQLExpr)` gives better results



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -564,6 +570,12 @@
     ],
     "sqlState" : "42000"
   },
+  "UNPIVOT_VALUE_SIZE_MISMATCH" : {
+    "message" : [
+      "All unpivot value columns must have the same size as there are value column names (<names>): [<sizes>]"

Review Comment:
   fixed in e6b1bcf2



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988686782


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -101,16 +101,45 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
         "pivotType" -> pivotCol.dataType.catalogString))
   }
 
+  def unpivotRequiresAttributes(given: String,
+                                empty: String,
+                                expressions: Seq[NamedExpression]): Throwable = {
+    val types = expressions.groupBy(_.getClass.getSimpleName)
+      .mapValues(exprs => exprs.map(expr => expr.toString.replaceAll("#\\d+", "")).sorted)
+      .mapValues(exprs => if (exprs.length > 3) exprs.take(3) :+ "..." else exprs)
+      .toList.sortBy(_._1)
+      .map { case (className, exprs) => s"$className (${exprs.mkString(", ")})" }

Review Comment:
   Done in 2c8f53df though `.map(toSQLExpr)` gives better results



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r977100211


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   I checked with @HyukjinKwon , even the pandas melt/unpivot API does not support expressions but only columns. Let's only allow `Attribute` for id and value columns, for both SQL and Scala API.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980968668


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -535,6 +548,98 @@ class DatasetUnpivotSuite extends QueryTest
         "val"),
       longStructDataRows)
   }
+
+  test("unpivot with struct expressions") {
+    checkAnswer(
+      wideDataDs.unpivot(
+        Array($"id"),
+        Array(
+          struct($"str1".as("str"), $"int1".cast(LongType).as("long")).as("str-int"),

Review Comment:
   Shall we remove this test? We will only allow attributes as id and value expressions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981083420


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),
+        None,
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    } else {
+      val unpivotClause = ctx.unpivotOperator().unpivotMultiValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val (unpivotColumns, unpivotAliases) =
+        unpivotClause.unpivotColumnSets.asScala.map(visitUnpivotColumnSet).toSeq.unzip
+
+      Unpivot(
+        None,
+        Some(unpivotColumns),
+        Some(unpivotAliases),
+        variableColumnName,
+        valueColumnNames,
+        query
+      )
+    }
+
+    // exclude null values
+    val filtered = if (ctx.nullOperator != null && ctx.nullOperator.EXCLUDE() != null) {

Review Comment:
   [Orcacle](https://www.oracletutorial.com/oracle-basics/oracle-unpivot/) and [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unpivot_operator) by default exclude NULL values, I am going to change that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981242591


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   Have have to come back to this:
   
   It looks like Oracle and BigQuery allow for these aliases:
   
   ```sql
   SELECT * FROM table UNPIVOT (
     val for var in (col1 AS alias1, col2 AS alias2)
   )
   ```
   and
   ```sql
   SELECT * FROM table UNPIVOT (
     (val1, val2) for var in ((col1, col2) AS alias1, (col3, col4) AS alias2)
   )
   ```
   
   but still not
   ```sql
   SELECT * FROM table UNPIVOT (
     (val1, val2) for var in ((col1 AS aliasA, col2 AS aliasB) AS alias1, (col3 AS aliasC, col4 AS aliasD) AS alias2)
   )
   ```
   (which was your original question).
   
   https://docs.oracle.com/cd/B28359_01/server.111/b28286/statements_10002.htm#SQLRF55133
   https://blogs.oracle.com/sql/post/how-to-convert-rows-to-columns-and-back-again-with-sql-aka-pivot-and-unpivot
   https://hevodata.com/learn/bigquery-columns-to-rows/#u1
   
   I'll add the former alias again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982503768


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   Added in 11cce9ef33.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988507447


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -552,6 +552,12 @@
       "Unable to acquire <requestedBytes> bytes of memory, got <receivedBytes>"
     ]
   },
+  "UNPIVOT_REQUIRES_ATTRIBUTES" : {
+    "message" : [
+      "UNPIVOT requires given {given} to be Attributes when no {empty} are given: [<types>]"

Review Comment:
   ```suggestion
         "UNPIVOT requires given {given} expressions to be columns when no {empty} expressions are given, but got: [<expressions>]"
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

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

   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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r975015432


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1374,33 +1374,110 @@ case class Pivot(
   override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild)
 }
 
+/**
+ * Expression for [[Unpivot]] for one unpivot value column (one or more expressions)
+ * and an optional alias. This node itself is not evaluable and resolvable.
+ * Only its children are to be resolved.
+ *
+ * @param exprs expressions to unpivot
+ * @param alias optional alias
+ */
+case class UnpivotExpr(exprs: Seq[NamedExpression], alias: Option[String]) extends Unevaluable {
+  override val children: Seq[NamedExpression] = exprs
+  override def dataType: DataType = throw new UnresolvedException("dataType")
+  override def nullable: Boolean = throw new UnresolvedException("nullable")
+
+  // this expression is never resolved, it gets replaced together with its parent Unpivot plan
+  override lazy val resolved = false
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[Expression]): Expression = {
+    // turn expressions into named expressions
+    copy(exprs = newChildren.map {
+      case ne: NamedExpression => ne
+      case e: Expression => UnresolvedAlias(e)
+    })
+  }
+}
+
 /**
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1"), None),
+ *       (Seq("val2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), None),
+ *       (Seq("val2.1", "val2.2"), None)
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       (Seq("val1.1", "val1.2"), Some("val1")),
+ *       (Seq("val2.1", "val2.2"), Some("val2"))
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
- * @param values             Value columns to unpivot
+ * @param values             Value column sets to unpivot with optional aliases
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[UnpivotExpr]],

Review Comment:
   I'd prefer the tuple, but it makes the code harder to read, so I revert back to the `aliases` field: cf0ec27



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on PR #37407:
URL: https://github.com/apache/spark/pull/37407#issuecomment-1229174797

   @cloud-fan @MaxGekk @HyukjinKwon @gengliangwang @zhengruifeng what do you think?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r962542605


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   It looks like other systems only allow for identifiers available in the `FROM`, not expressions as our unpivot supports. In those systems you would have to move the expressions into a subquery in `FROM` and reference the named expression.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r965771785


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   Saying `unpivotColumn` should be `namedExpression` instead of `expression (AS? identifier)?`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972833735


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Oh I see, it's to match the behavior of when id columns are not specified. It's better to make them consistent, but it's also important to match both Pandas behavior and SQL behavior.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972802281


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3064,7 +3064,7 @@ def cube(self, *cols: "ColumnOrName") -> "GroupedData":  # type: ignore[misc]
 
     def unpivot(
         self,
-        ids: Optional[Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]]],
+        ids: Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]],

Review Comment:
   Makes sense. The Scala API also requires specifying the id columns, and Python API should be consistent.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r958680962


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -2094,6 +2095,23 @@ class Dataset[T] private[sql](
    * This is equivalent to calling `Dataset#unpivot(Array, Array, String, String)`
    * where `values` is set to all non-id columns that exist in the DataFrame.
    *
+   * Note: A column that is referenced by an id column expression is considered an id column itself.
+   * For instance `$"id" * 2` references column `id`, so `id` is considered an id column and not a
+   * value column:
+   * {{{
+   *   val df = Seq((1, 11, 12L), (2, 21, 22L)).toDF("id", "int", "long")
+   *   df.unpivot(Array($"id" * 2), "var", "val").show()
+   *   // output:
+   *   // +--------+--------+-----+
+   *   // |(id * 2)|variable|value|
+   *   // +--------+--------+-----+
+   *   // |       2|     int|   11|
+   *   // |       2|    long|   12|
+   *   // |       4|     int|   21|
+   *   // |       4|    long|   22|
+   *   // +--------+--------+-----+

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r979697887


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   @EnricoMi Are you working on making this change?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r979850401


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Waiting for your reply on https://github.com/apache/spark/pull/37407#discussion_r977818035.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on PR #37407:
URL: https://github.com/apache/spark/pull/37407#issuecomment-1231853602

   > Can we write down the SQL spec for this syntax in the PR description? To make it easier for people to review the syntax and understand the semantic.
   
   I have added the syntax and examples from `docs/sql-ref-syntax-qry-select-unpivot.md` to the PR description.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959700764


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   do we allow `IN ((col1 AS a, col2 AS b) AS c)`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r960164566


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   is it also valid in other systems?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r966629689


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   just `expression`? people can still use `col AS alias` but we just treat it as a normal expression.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972790708


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala:
##########
@@ -208,6 +208,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
     }
 
     def recursiveTransform(arg: Any): AnyRef = arg match {
+      case ue: UnpivotExpr => ue.withNewChildren(ue.exprs.map(transformExpression))

Review Comment:
   This is a very special treatment in a very generic method. Not nice, but the best way I could find to make `mapExpressions` tranform `UnpivotExpr.exprs` as a top-level expression. See comment on top-level expression in PR discussion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972871859


##########
python/pyspark/sql/dataframe.py:
##########
@@ -3064,7 +3064,7 @@ def cube(self, *cols: "ColumnOrName") -> "GroupedData":  # type: ignore[misc]
 
     def unpivot(
         self,
-        ids: Optional[Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]]],
+        ids: Union["ColumnOrName", List["ColumnOrName"], Tuple["ColumnOrName", ...]],

Review Comment:
   However, `pyspark.pandas.frame.melt` allows for `None` for `ids`, having the meaning of `[]`, while `values` being `None` means magically take all non-id columns:
   
   ```python
       def melt(
           self,
           id_vars: Optional[Union[Name, List[Name]]] = None,
           value_vars: Optional[Union[Name, List[Name]]] = None,
           var_name: Optional[Union[str, List[str]]] = None,
           value_name: str = "value",
       ) -> "DataFrame":
           """
           ...
           Parameters
           ----------
           frame : DataFrame
           id_vars : tuple, list, or ndarray, optional
               Column(s) to use as identifier variables.
           value_vars : tuple, list, or ndarray, optional
               Column(s) to unpivot. If not specified, uses all columns that
               are not set as `id_vars`.
   ```
   
   Should Python API be consistent with Scala API or PySpark Pandas API?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r973273719


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   The former.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r977414048


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Alright, I'll create a separate PR for that, and rebase this afterwards.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980015454


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Will start with that change now.
   
   Downgrading PySpark SQL to column names will be done in a separate PR, downgrading SQL to column names is going to happen here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981213982


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   This has been done for PySpark in separate PR: #38020



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981106767


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -535,6 +548,98 @@ class DatasetUnpivotSuite extends QueryTest
         "val"),
       longStructDataRows)
   }
+
+  test("unpivot with struct expressions") {
+    checkAnswer(
+      wideDataDs.unpivot(
+        Array($"id"),
+        Array(
+          struct($"str1".as("str"), $"int1".cast(LongType).as("long")).as("str-int"),

Review Comment:
   I can move this into a separate Scala DataFrame PR. The Scala API does support expressions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982382485


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val idAttrs = AttributeSet(up.ids.get)

Review Comment:
   For all APIs. SQL is special because it can't specify id 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969645973


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.

Review Comment:
   since value columns cannot be Nil, can we use `values: Seq[Seq[NamedExpression]]` and then set it as `Nil` to indicate that the value columns are to be inferred?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988505530


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+  "messageParameters" : {
+    "columns" : "course, year, earningsYear",
+    "targetString" : "up"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 11,
+    "fragment" : "up.*"
+  } ]
+}
+
+
+-- !query
+create temporary view courseEarningsAndSales as select * from values
+  ("dotNET", 15000, 2, 48000, 1, 22500, 1),
+  ("Java", 20000, 1, 30000, 2, NULL, NULL)
+  as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) `2012`, (earnings2013, sales2013) `2013`, (earnings2014, sales2014) `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)

Review Comment:
   ditto, we can test optional `AS` keyword in `UnpivotParserSuite`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988509480


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -552,6 +552,12 @@
       "Unable to acquire <requestedBytes> bytes of memory, got <receivedBytes>"
     ]
   },
+  "UNPIVOT_REQUIRES_ATTRIBUTES" : {
+    "message" : [
+      "UNPIVOT requires given {given} to be Attributes when no {empty} are given: [<types>]"

Review Comment:
   then the caller side can just pass "id" and "value"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r970587522


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except

Review Comment:
   The Scala / Java API always sets ids.
   The Python API calls into the Scala API.
   The SQL API always sets values.
   
   There is no code path that creates `Unpivot` with both `ids` and `values` being None. That is why there is no rule in CheckAnalysis and no user facing error for this case.
   
   I could add an assert in Unpivot to handle this case and provide a useful error / comment in the code.
   
   The SQL API requires `Unpivot` to allow for `ids = None` as the SQL syntax does not allow to specify any ids columns. The select clause contains that information, but that is inaccessible when parsing the unpivot clause. So resolving ids is done in analysis phase, not in SQL parsing phase.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

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

   Can we write down the SQL spec for this syntax in the PR description? To make it easier for people to review the syntax and understand the semantic.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r959694541


##########
docs/sql-ref-syntax-qry-select-unpivot.md:
##########
@@ -0,0 +1,142 @@
+---
+layout: global
+title: UNPIVOT Clause
+displayTitle: UNPIVOT Clause
+license: |
+  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.
+---
+
+### Description
+
+The `UNPIVOT` clause transforms multiple columns into multiple rows used in `SELECT` clause. The `UNPIVOT` clause can be specified after the table name or subquery.
+
+### Syntax
+
+```sql
+UNPIVOT [ { INCLUDE | EXCLUDE } NULLS ] (
+    { single_value_column_unpivot | multi_value_column_unpivot }
+) [[AS] alias]
+
+single_value_column_unpivot:
+    values_column
+    FOR name_column
+    IN (unpivot_column [[AS] alias] [, ...])
+
+multi_value_column_unpivot:
+    (values_column [, ...])
+    FOR name_column
+    IN ((unpivot_column [[AS] alias] [, ...]) [, ...])

Review Comment:
   this seems incorrect, should be `IN ((unpivot_column [, ...]) [[AS] alias] [, ...])`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r960164704


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1375,28 +1375,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],

Review Comment:
   sorry typo, do they must be `Some` after the plan is analyzed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982503768


##########
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -618,6 +618,46 @@ pivotValue
     : expression (AS? identifier)?
     ;
 
+unpivotClause
+    : UNPIVOT nullOperator=unpivotNullClause? LEFT_PAREN
+        operator=unpivotOperator
+      RIGHT_PAREN (AS? identifier)?
+    ;
+
+unpivotNullClause
+    : (INCLUDE | EXCLUDE) NULLS
+    ;
+
+unpivotOperator
+    : (unpivotSingleValueColumnClause | unpivotMultiValueColumnClause)
+    ;
+
+unpivotSingleValueColumnClause
+    : unpivotValueColumn FOR unpivotNameColumn IN LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN
+    ;
+
+unpivotMultiValueColumnClause
+    : LEFT_PAREN unpivotValueColumns+=unpivotValueColumn (COMMA unpivotValueColumns+=unpivotValueColumn)* RIGHT_PAREN
+      FOR unpivotNameColumn
+      IN LEFT_PAREN unpivotColumnSets+=unpivotColumnSet (COMMA unpivotColumnSets+=unpivotColumnSet)* RIGHT_PAREN
+    ;
+
+unpivotColumnSet
+    : LEFT_PAREN unpivotColumns+=unpivotColumn (COMMA unpivotColumns+=unpivotColumn)* RIGHT_PAREN (AS? identifier)?

Review Comment:
   Added in 11cce9ef.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980967098


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -137,48 +138,49 @@ class DatasetUnpivotSuite extends QueryTest
     assert(unpivoted.schema === longSchema)
     checkAnswer(unpivoted, longDataRows)
 
-    val unpivoted2 = wideDataDs.select($"id", $"str1", $"str2")
-      .unpivot(
-        Array($"id"),
-        Array.empty,
-        variableColumnName = "var",
-        valueColumnName = "val")
-    assert(unpivoted2.schema === longSchema)
-    checkAnswer(unpivoted2, longDataRows)
+    val e = intercept[AnalysisException] {
+      wideDataDs.select($"id", $"str1", $"str2")
+        .unpivot(
+          Array($"id"),
+          Array.empty,
+          variableColumnName = "var",
+          valueColumnName = "val")
+    }
+    checkError(
+      exception = e,
+      errorClass = "UNPIVOT_REQUIRES_VALUE_COLUMNS",

Review Comment:
   nvm, people should call the other overload if they want Spark to infer value 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r980969164


##########
sql/core/src/test/scala/org/apache/spark/sql/DatasetUnpivotSuite.scala:
##########
@@ -535,6 +548,98 @@ class DatasetUnpivotSuite extends QueryTest
         "val"),
       longStructDataRows)
   }
+
+  test("unpivot with struct expressions") {
+    checkAnswer(
+      wideDataDs.unpivot(
+        Array($"id"),
+        Array(
+          struct($"str1".as("str"), $"int1".cast(LongType).as("long")).as("str-int"),

Review Comment:
   and this PR should focus on the SQL API, not DataFrame.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981080039


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
+      case up: Unpivot if up.childrenResolved &&
+        up.values.exists(_.forall(_.forall(_.resolved))) && up.ids.isEmpty =>
+        val ids = (AttributeSet(up.child.output) -- AttributeSet(up.values.get.flatten)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(ids = Some(up.child.output.intersect(ids)))
+
+      case up: Unpivot if !up.childrenResolved || !up.ids.exists(_.forall(_.resolved)) ||
+        !up.values.exists(_.nonEmpty) || !up.values.exists(_.forall(_.forall(_.resolved))) ||
+        !up.values.get.forall(_.length == up.valueColumnNames.length) ||
+        !up.valuesTypeCoercioned => up
 
       // TypeCoercionBase.UnpivotCoercion determines valueType
       // and casts values once values are set and resolved
-      case Unpivot(ids, values, variableColumnName, valueColumnName, child) =>
+      case Unpivot(Some(ids), Some(values), aliases, variableColumnName, valueColumnNames, child) =>
+
+        def toString(values: Seq[NamedExpression]): String =
+          "(" + values.map(v => quoteIdentifier(v.name)).mkString(",") + ")"

Review Comment:
   [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unpivot_operator) does `col1_col2`:
   
       row_value_alias: An optional alias for a column set that is displayed for the column set in name_column.
       If not specified, a string value for the column set is used and each column in the string is separated with
       an underscore (_). For example, `(col1, col2)` outputs `col1_col2`.
   
   [Oracle](https://www.oracletutorial.com/oracle-basics/oracle-unpivot/) and [T-SQL](https://learn.microsoft.com/en-us/sql/t-sql/queries/from-using-pivot-and-unpivot?view=sql-server-ver15#unpivot-example) are not specific about the string representation of those column name tuples.
   
   [Oracle seems to do the same thing](https://blogs.oracle.com/sql/post/how-to-convert-rows-to-columns-and-back-again-with-sql-aka-pivot-and-unpivot) as BigQuery _(given `(gold_medals, gold_sports)`)_:
   
       Without these medal column contains the concatenation of the source column names.
       For example GOLD_MEDALS_GOLD_SPORTS.
   
   I am going to change that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988504705


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up

Review Comment:
   nit: ideally the SQL test should focus on the end-user behavior, instead of tiny details like the optional `AS` keyword. We can have a `UnpivotParserSuite` to focus on these details. See `DDLParserSuite` as an example.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988504938


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()

Review Comment:
   ditto for this one, we can create a `UnpivotParserSuite` to test 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988506103


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+  "messageParameters" : {
+    "columns" : "course, year, earningsYear",
+    "targetString" : "up"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 11,
+    "fragment" : "up.*"
+  } ]
+}
+
+
+-- !query
+create temporary view courseEarningsAndSales as select * from values
+  ("dotNET", 15000, 2, 48000, 1, 22500, 1),
+  ("Java", 20000, 1, 30000, 2, NULL, NULL)
+  as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) `2012`, (earnings2013, sales2013) `2013`, (earnings2014, sales2014) `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012 as earnings, sales2012 as sales) as `2012`, (earnings2013 as earnings, sales2013 as sales) as `2013`, (earnings2014 as earnings, sales2014 as sales) as `2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "'as'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT EXCLUDE NULLS (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	earnings2012_sales2012	20000	1
+Java	earnings2013_sales2013	30000	2
+dotNET	earnings2012_sales2012	15000	2
+dotNET	earnings2013_sales2013	48000	1
+dotNET	earnings2014_sales2014	22500	1
+
+
+-- !query
+SELECT * FROM (
+  SELECT course,
+         named_struct('2012', earnings2012, '2013', earnings2013, '2014', earnings2014) AS `the.earnings`,
+         named_struct('2012', sales2012, '2013', sales2013, '2014', sales2014) AS `the.sales`
+  FROM courseEarningsAndSales
+)
+UNPIVOT (
+  (earnings, sales) FOR year IN (
+    (`the.earnings`.`2012`, `the.sales`.`2012`) `2012`,
+    (`the.earnings`.`2013`, `the.sales`.`2013`) `2013`,
+    (`the.earnings`.`2014`, `the.sales`.`2014`) `2014`
+  )
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS 2012, the.earnings.2013 AS 2013, the.earnings.2014 AS 2014, ...)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2013, sales2013), (earnings2014, sales2014))
+)
+-- !query schema
+struct<course:string,earnings2012:int,sales2012:int,year:string,earnings:int,sales:int>
+-- !query output
+Java	20000	1	earnings2013_sales2013	30000	2
+dotNET	15000	2	earnings2013_sales2013	48000	1
+dotNET	15000	2	earnings2014_sales2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  () FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))

Review Comment:
   ditto



##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+  "messageParameters" : {
+    "columns" : "course, year, earningsYear",
+    "targetString" : "up"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 11,
+    "fragment" : "up.*"
+  } ]
+}
+
+
+-- !query
+create temporary view courseEarningsAndSales as select * from values
+  ("dotNET", 15000, 2, 48000, 1, 22500, 1),
+  ("Java", 20000, 1, 30000, 2, NULL, NULL)
+  as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) `2012`, (earnings2013, sales2013) `2013`, (earnings2014, sales2014) `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012 as earnings, sales2012 as sales) as `2012`, (earnings2013 as earnings, sales2013 as sales) as `2013`, (earnings2014 as earnings, sales2014 as sales) as `2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "'as'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT EXCLUDE NULLS (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	earnings2012_sales2012	20000	1
+Java	earnings2013_sales2013	30000	2
+dotNET	earnings2012_sales2012	15000	2
+dotNET	earnings2013_sales2013	48000	1
+dotNET	earnings2014_sales2014	22500	1
+
+
+-- !query
+SELECT * FROM (
+  SELECT course,
+         named_struct('2012', earnings2012, '2013', earnings2013, '2014', earnings2014) AS `the.earnings`,
+         named_struct('2012', sales2012, '2013', sales2013, '2014', sales2014) AS `the.sales`
+  FROM courseEarningsAndSales
+)
+UNPIVOT (
+  (earnings, sales) FOR year IN (
+    (`the.earnings`.`2012`, `the.sales`.`2012`) `2012`,
+    (`the.earnings`.`2013`, `the.sales`.`2013`) `2013`,
+    (`the.earnings`.`2014`, `the.sales`.`2014`) `2014`
+  )
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS 2012, the.earnings.2013 AS 2013, the.earnings.2014 AS 2014, ...)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2013, sales2013), (earnings2014, sales2014))
+)
+-- !query schema
+struct<course:string,earnings2012:int,sales2012:int,year:string,earnings:int,sales:int>
+-- !query output
+Java	20000	1	earnings2013_sales2013	30000	2
+dotNET	15000	2	earnings2013_sales2013	48000	1
+dotNET	15000	2	earnings2014_sales2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  () FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ": extra input ')'"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ()

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988505872


##########
sql/core/src/test/resources/sql-tests/results/unpivot.sql.out:
##########
@@ -0,0 +1,409 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view courseEarnings as select * from values
+  ("dotNET", 15000, 48000, 22500),
+  ("Java", 20000, 30000, NULL)
+  as courseEarnings(course, `2012`, `2013`, `2014`)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+Java	2014	NULL
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT EXCLUDE NULLS (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, `the.earnings`.* FROM (
+    SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+    FROM courseEarnings
+  )
+)
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT * FROM (
+  SELECT course, struct(`2012`, `2013`, `2014`) AS `the.earnings`
+  FROM courseEarnings
+)
+UNPIVOT (
+  earningsYear FOR year IN (`the.earnings`.`2012`, `the.earnings`.`2013`, `the.earnings`.`2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNPIVOT_REQUIRES_ATTRIBUTES",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "empty" : "ids",
+    "given" : "values",
+    "types" : "Alias (the.earnings.2012 AS the.earnings.2012, the.earnings.2013 AS the.earnings.2013, the.earnings.2014 AS the.earnings.2014)"
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2013`, `2014`)
+)
+-- !query schema
+struct<course:string,2012:int,year:string,earningsYear:int>
+-- !query output
+Java	20000	2013	30000
+dotNET	15000	2013	48000
+dotNET	15000	2014	22500
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  values FOR year IN ()
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42000",
+  "messageParameters" : {
+    "error" : "')'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SELECT * FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+)
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	twenty-thirteen	30000
+Java	twenty-twelve	20000
+dotNET	twenty-fourteen	22500
+dotNET	twenty-thirteen	48000
+dotNET	twenty-twelve	15000
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+) AS up
+-- !query schema
+struct<course:string,year:string,earningsYear:int>
+-- !query output
+Java	2012	20000
+Java	2013	30000
+dotNET	2012	15000
+dotNET	2013	48000
+dotNET	2014	22500
+
+
+-- !query
+SELECT up.* FROM courseEarnings
+UNPIVOT (
+  earningsYear FOR year IN (`2012`, `2013`, `2014`)
+)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+  "messageParameters" : {
+    "columns" : "course, year, earningsYear",
+    "targetString" : "up"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 11,
+    "fragment" : "up.*"
+  } ]
+}
+
+
+-- !query
+create temporary view courseEarningsAndSales as select * from values
+  ("dotNET", 15000, 2, 48000, 1, 22500, 1),
+  ("Java", 20000, 1, 30000, 2, NULL, NULL)
+  as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) `2012`, (earnings2013, sales2013) `2013`, (earnings2014, sales2014) `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)
+)
+-- !query schema
+struct<course:string,year:string,earnings:int,sales:int>
+-- !query output
+Java	2012	20000	1
+Java	2013	30000	2
+dotNET	2012	15000	2
+dotNET	2013	48000	1
+dotNET	2014	22500	1
+
+
+-- !query
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+  (earnings, sales) FOR year IN ((earnings2012 as earnings, sales2012 as sales) as `2012`, (earnings2013 as earnings, sales2013 as sales) as `2013`, (earnings2014 as earnings, sales2014 as sales) as `2014`)

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988508520


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,52 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given, and only AttributeReference are given
+      case up @Unpivot(Some(ids), None, _, _, _, _) if up.childrenResolved &&
+        ids.forall(_.resolved) &&
+        ids.forall(_.isInstanceOf[AttributeReference]) =>
+        val idAttrs = AttributeSet(up.ids.get)
+        val values = up.child.output.filterNot(idAttrs.contains)
+        up.copy(values = Some(values.map(Seq(_))))
+      case up @Unpivot(None, Some(values), _, _, _, _) if up.childrenResolved &&

Review Comment:
   ```suggestion
         case up @ Unpivot(None, Some(values), _, _, _, _) if up.childrenResolved &&
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r988508833


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -101,16 +101,45 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
         "pivotType" -> pivotCol.dataType.catalogString))
   }
 
+  def unpivotRequiresAttributes(given: String,
+                                empty: String,
+                                expressions: Seq[NamedExpression]): Throwable = {

Review Comment:
   ```suggestion
     def unpivotRequiresAttributes(
         given: String,
         empty: String,
         expressions: Seq[NamedExpression]): Throwable = {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on PR #37407:
URL: https://github.com/apache/spark/pull/37407#issuecomment-1270014946

   All green: https://github.com/G-Research/spark/actions/runs/3196590413/jobs/5220447028
   
   Status update may have 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #37407: [SPARK-39876][SQL][WIP] Add UNPIVOT to SQL syntax

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

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r969660489


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],
+    aliases: Option[Seq[Option[String]]],

Review Comment:
   can we combine `values` and `aliases` since they are in the same clause? e.g. `values: Option[Seq[(Seq[NamedExpression], Option[String])]]`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972844049


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   I thought, the following would be annoying:
   ```
   scala> val df = spark.range(5).select($"id", ($"id"*10).as("val"))
   scala> df.show()
   +---+---+
   | id|val|
   +---+---+
   |  0|  0|
   |  1| 10|
   |  2| 20|
   |  3| 30|
   |  4| 40|
   +---+---+
   
   df.unpivot(Array($"id" * 2), "col", "val").show()
   +--------+---+---+
   |(id * 2)|col|val|
   +--------+---+---+
   |       0|id|  0|
   |       0|val|  0|
   |       2|id| 1|
   |       2|val| 10|
   |       4|id 2|
   |       4|val| 20|
   |       6|id 3|
   |       6|val| 30|
   |       8|id 4|
   |       8|val| 40|
   +--------+---+---+
   ```
   
   Of course, that id manipulation can be done before `unpivot` to "materialize" it as a reference:
   ```
   df.withColumn("id", $"id" * 2).unpivot(Array($"id"), "col", "val").show()
   +---+---+---+
   | id|col|val|
   +---+---+---+
   |  0|val|  0|
   |  2|val| 10|
   |  4|val| 20|
   |  6|val| 30|
   |  8|val| 40|
   +---+---+---+
   ```
   
   Happy to remove that complexity.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r972869981


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   Scala and Python API both require ids. SQL API does not allow to specify ids, which is the same in other SQL databases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r971407082


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:
##########
@@ -1378,28 +1378,84 @@ case class Pivot(
  * A constructor for creating an Unpivot, which will later be converted to an [[Expand]]
  * during the query analysis.
  *
- * An empty values array will be replaced during analysis with all resolved outputs of child except
+ * Either ids or values array must be set. The ids array can be empty,
+ * the values array must not be empty if not None.
+ *
+ * A None ids array will be replaced during analysis with all resolved outputs of child except
+ * the values. This expansion allows to easily select all non-value columns as id columns.
+ *
+ * A None values array will be replaced during analysis with all resolved outputs of child except
  * the ids. This expansion allows to easily unpivot all non-id columns.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveUnpivot`
  *
- * The type of the value column is derived from all value columns during analysis once all values
- * are resolved. All values' types have to be compatible, otherwise the result value column cannot
- * be assigned the individual values and an AnalysisException is thrown.
+ * Multiple columns can be unpivoted in one row by providing multiple value column names
+ * and the same number of unpivot value expressions:
+ * {{{
+ *   // one-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1"),
+ *       Seq("val2")
+ *     )),
+ *     None,
+ *     "var",
+ *     Seq("val")
+ *   )
+ *
+ *   // two-dimensional value columns
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     None,
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * The variable column will contain the name of the unpivot value while the value columns contain
+ * the unpivot values. Multi-dimensional unpivot values can be given `aliases`:
+ * }}}
+ *   // two-dimensional value columns with aliases
+ *   Unpivot(
+ *     Some(Seq("id")),
+ *     Some(Seq(
+ *       Seq("val1.1", "val1.2"),
+ *       Seq("val2.1", "val2.2"))
+ *     ),
+ *     Some(Seq(
+ *       Some("val1"),
+ *       Some("val2")
+ *     )),
+ *     "var",
+ *     Seq("val1", "val2")
+ *   )
+ * }}}
+ *
+ * All "value" columns must share a least common data type. Unless they are the same data type,
+ * all "value" columns are cast to the nearest common data type. For instance,
+ * types `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`
+ * do not have a common data type and `unpivot` fails with an `AnalysisException`.
  *
  * @see `org.apache.spark.sql.catalyst.analysis.TypeCoercionBase.UnpivotCoercion`
  *
  * @param ids                Id columns
  * @param values             Value columns to unpivot
+ * @param aliases            Optional aliases for values
  * @param variableColumnName Name of the variable column
- * @param valueColumnName    Name of the value column
+ * @param valueColumnNames   Names of the value columns
  * @param child              Child operator
  */
 case class Unpivot(
-    ids: Seq[NamedExpression],
-    values: Seq[NamedExpression],
+    ids: Option[Seq[NamedExpression]],
+    values: Option[Seq[Seq[NamedExpression]]],
+    aliases: Option[Seq[Option[String]]],

Review Comment:
   It's in `QueryPlan.mapExpressions`, Spark does not accept expressions in `Tuple2`...
   
   Another way is to create a `case class UnpivotValueColumns(children: Seq[Expression], alias: Option[String]) extends Expression`, and then use `values: Option[Seq[UnpivotValueColumns]]` here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r974938699


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +873,55 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        up.copy(values =
+          Some(
+            up.child.output.diff(up.ids.get.flatMap(_.references))

Review Comment:
   In contrast to other databases, the Spark implementation allows for expressions rather than column names:
   
   ```scala
   df.unpivot(
     Array($"id"),
     Array($"struct.val", $"struct.time".cast(DateType).as("date")),
     "column",
     "value"
   )
   ```
   
   ```SQL
   SELECT *
   FROM df
   UNPIVOT (
     value FOR column in (struct.val, CAST(struct.time AS DATE) AS date)
   )
   ```
   
   This complicates identifying the unpivoted (value) columns in order to derive the id columns. [Ignoring references here will add columns used in unpivot expressions to the id columns](https://github.com/apache/spark/pull/37407/commits/de19ba0bb33b6369a339a9578173e628102332ae#diff-f306d1dfcd20c4cdcdb5d631416ee9d92bf6893422b0a9683ca4493d7215e5e8):
   
   ```SQL
   SELECT * FROM courseEarnings
   UNPIVOT (
     earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
   )
   
   -- !query schema
   struct<course:string,2012:int,2013:int,2014:int,year:string,earningsYear:int>
   
   -- !query output
   Java	20000	30000	NULL	twenty-fourteen	NULL
   Java	20000	30000	NULL	twenty-thirteen	30000
   Java	20000	30000	NULL	twenty-twelve	20000
   dotNET	15000	48000	22500	twenty-fourteen	22500
   dotNET	15000	48000	22500	twenty-thirteen	48000
   dotNET	15000	48000	22500	twenty-twelve	15000
   ```
   
   This is not a problem for the Scala API as we always provide id columns there. But in the SQL API, this gets messy.
   
   Maybe we should disallow expressions for unpivot columns (ids and values). Those expression can be expressed in the FROM clause.
   
   But then the Scala API would also need to abandon expressions, to be like the SQL API. Or would you say they don't need to match in this sense?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981080039


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
+      case up: Unpivot if up.childrenResolved &&
+        up.values.exists(_.forall(_.forall(_.resolved))) && up.ids.isEmpty =>
+        val ids = (AttributeSet(up.child.output) -- AttributeSet(up.values.get.flatten)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(ids = Some(up.child.output.intersect(ids)))
+
+      case up: Unpivot if !up.childrenResolved || !up.ids.exists(_.forall(_.resolved)) ||
+        !up.values.exists(_.nonEmpty) || !up.values.exists(_.forall(_.forall(_.resolved))) ||
+        !up.values.get.forall(_.length == up.valueColumnNames.length) ||
+        !up.valuesTypeCoercioned => up
 
       // TypeCoercionBase.UnpivotCoercion determines valueType
       // and casts values once values are set and resolved
-      case Unpivot(ids, values, variableColumnName, valueColumnName, child) =>
+      case Unpivot(Some(ids), Some(values), aliases, variableColumnName, valueColumnNames, child) =>
+
+        def toString(values: Seq[NamedExpression]): String =
+          "(" + values.map(v => quoteIdentifier(v.name)).mkString(",") + ")"

Review Comment:
   [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unpivot_operator) does `col1_col2`:
   
       row_value_alias: An optional alias for a column set that is displayed for the column set in name_column.
       If not specified, a string value for the column set is used and each column in the string is separated with
       an underscore (_). For example, (col1, col2) outputs col1_col2.
   
   [Oracle](https://www.oracletutorial.com/oracle-basics/oracle-unpivot/) and [T-SQL](https://learn.microsoft.com/en-us/sql/t-sql/queries/from-using-pivot-and-unpivot?view=sql-server-ver15#unpivot-example) are not specific about the string representation of those column name tuples.
   
   [Oracle seems to do the same thing](https://blogs.oracle.com/sql/post/how-to-convert-rows-to-columns-and-back-again-with-sql-aka-pivot-and-unpivot) as BigQuery _(given `(gold_medals, gold_sports)`)_:
   
       Without these medal column contains the concatenation of the source column names.
       For example GOLD_MEDALS_GOLD_SPORTS.
   
   I am going to change that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] EnricoMi commented on a diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
EnricoMi commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r981169668


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -869,26 +869,50 @@ class Analyzer(override val catalogManager: CatalogManager)
     def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
       _.containsPattern(UNPIVOT), ruleId) {
 
-      // once children and ids are resolved, we can determine values, if non were given
-      case up: Unpivot if up.childrenResolved && up.ids.forall(_.resolved) && up.values.isEmpty =>
-        up.copy(values = up.child.output.diff(up.ids))
-
-      case up: Unpivot if !up.childrenResolved || !up.ids.forall(_.resolved) ||
-        up.values.isEmpty || !up.values.forall(_.resolved) || !up.valuesTypeCoercioned => up
+      // once children are resolved, we can determine values from ids and vice versa
+      // if only either is given
+      case up: Unpivot if up.childrenResolved &&
+        up.ids.exists(_.forall(_.resolved)) && up.values.isEmpty =>
+        val values = (AttributeSet(up.child.output) -- AttributeSet(up.ids.get)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(values = Some(up.child.output.intersect(values).map(Seq(_))))
+      case up: Unpivot if up.childrenResolved &&
+        up.values.exists(_.forall(_.forall(_.resolved))) && up.ids.isEmpty =>
+        val ids = (AttributeSet(up.child.output) -- AttributeSet(up.values.get.flatten)).toSeq
+        // up.child.output.intersect preserves order of columns
+        up.copy(ids = Some(up.child.output.intersect(ids)))
+
+      case up: Unpivot if !up.childrenResolved || !up.ids.exists(_.forall(_.resolved)) ||
+        !up.values.exists(_.nonEmpty) || !up.values.exists(_.forall(_.forall(_.resolved))) ||
+        !up.values.get.forall(_.length == up.valueColumnNames.length) ||
+        !up.valuesTypeCoercioned => up
 
       // TypeCoercionBase.UnpivotCoercion determines valueType
       // and casts values once values are set and resolved
-      case Unpivot(ids, values, variableColumnName, valueColumnName, child) =>
+      case Unpivot(Some(ids), Some(values), aliases, variableColumnName, valueColumnNames, child) =>
+
+        def toString(values: Seq[NamedExpression]): String =
+          "(" + values.map(v => quoteIdentifier(v.name)).mkString(",") + ")"

Review Comment:
   Done in 890d21f8.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37407: [SPARK-39876][SQL] Add UNPIVOT to SQL syntax

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on code in PR #37407:
URL: https://github.com/apache/spark/pull/37407#discussion_r982631012


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -1098,6 +1106,87 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
     }
   }
 
+  /**
+   * Add an [[Unpivot]] to a logical plan.
+   */
+  private def withUnpivot(
+      ctx: UnpivotClauseContext,
+      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
+    // this is needed to create unpivot and to filter unpivot for nulls further down
+    val valueColumnNames =
+      Option(ctx.unpivotOperator().unpivotSingleValueColumnClause())
+        .map(_.unpivotValueColumn().identifier().getText)
+        .map(Seq(_))
+      .getOrElse(
+        Option(ctx.unpivotOperator().unpivotMultiValueColumnClause())
+          .map(_.unpivotValueColumns.asScala.map(_.identifier().getText).toSeq)
+          .get
+      )
+
+    val unpivot = if (ctx.unpivotOperator().unpivotSingleValueColumnClause() != null) {
+      val unpivotClause = ctx.unpivotOperator().unpivotSingleValueColumnClause()
+      val variableColumnName = unpivotClause.unpivotNameColumn().identifier().getText
+      val unpivotColumns = unpivotClause.unpivotColumns.asScala.map(visitUnpivotColumn).toSeq
+
+      Unpivot(
+        None,
+        Some(unpivotColumns.map(Seq(_))),

Review Comment:
   I'm saying that it's impossible to enforce `AttributeReference` in the parser side. `UnresolvedAttribute` can be resolved to `AttributeReference` or `GetStructField`. Please refer to the rule `ResolveReferences`. We need to enforce `AttributeReference` in the analyzer rule.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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