[spark] branch master updated (7c3ec122636 -> 0d93bb2c0a4)

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 7c3ec122636 [SPARK-42268][CONNECT][PYTHON] Add UserDefinedType in 
protos
 add 0d93bb2c0a4 [SPARK-42093][SQL] Move JavaTypeInference to 
AgnosticEncoders

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/JavaTypeInference.scala | 565 ++---
 .../spark/sql/catalyst/ScalaReflection.scala   |  64 ++-
 .../sql/catalyst/encoders/AgnosticEncoder.scala|  13 +-
 .../sql/catalyst/encoders/ExpressionEncoder.scala  |  11 +-
 .../sql/catalyst/expressions/objects/objects.scala |   8 +-
 .../sql/catalyst/JavaTypeInferenceSuite.scala  | 203 +++-
 6 files changed, 418 insertions(+), 446 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-42115][SQL] Push down limit through Python UDFs

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 028232693f2 [SPARK-42115][SQL] Push down limit through Python UDFs
028232693f2 is described below

commit 028232693f24fa82813daf18e36881b23df3edc0
Author: Kelvin Jiang 
AuthorDate: Thu Feb 2 09:00:51 2023 +0800

[SPARK-42115][SQL] Push down limit through Python UDFs

### What changes were proposed in this pull request?

This PR adds cases in LimitPushDown to push limits through Python UDFs. In 
order to allow for this, we need to call LimitPushDown in SparkOptimizer after 
the "Extract Python UDFs" batch. We also add PushProjectionThroughLimit 
afterwards in order to plan CollectLimit.

### Why are the changes needed?

Right now, LimitPushdown does not push limits through Python UDFs, which 
means that expensive Python UDFs can be run on potentially large amounts of 
input. This PR adds this capability, while ensuring that a GlobalLimit - 
LocalLimit pattern stays at the top in order to trigger the CollectLimit code 
path.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added a UT.

Closes #39842 from kelvinjian-db/SPARK-42115-limit-through-python-udfs.

Authored-by: Kelvin Jiang 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 0fe361e3a1b5be04114402b78e62dd010703477b)
Signed-off-by: Wenchen Fan 
---
 .../sql/tests/pandas/test_pandas_udf_scalar.py |  1 +
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  9 +
 .../plans/logical/pythonLogicalOperators.scala |  3 ++
 .../spark/sql/catalyst/trees/TreePatterns.scala|  1 +
 .../spark/sql/execution/SparkOptimizer.scala   |  2 +
 .../execution/python/ExtractPythonUDFsSuite.scala  | 43 ++
 6 files changed, 59 insertions(+)

diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py 
b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
index cbb26e45d2f..33c957fac58 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
@@ -981,6 +981,7 @@ class ScalarPandasUDFTests(ReusedSQLTestCase):
 with self.assertRaisesRegex(Exception, "reached finally block"):
 self.spark.range(1).select(test_close(col("id"))).collect()
 
+@unittest.skip("LimitPushDown should push limits through Python UDFs so 
this won't occur")
 def test_scalar_iter_udf_close_early(self):
 tmp_dir = tempfile.mkdtemp()
 try:
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 627e3952480..1233f2207f5 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -766,6 +766,15 @@ object LimitPushDown extends Rule[LogicalPlan] {
 // Push down local limit 1 if join type is LeftSemiOrAnti and join 
condition is empty.
 case j @ Join(_, right, LeftSemiOrAnti(_), None, _) if 
!right.maxRows.exists(_ <= 1) =>
   j.copy(right = maybePushLocalLimit(Literal(1, IntegerType), right))
+// Push down limits through Python UDFs.
+case LocalLimit(le, udf: BatchEvalPython) =>
+  LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child)))
+case LocalLimit(le, p @ Project(_, udf: BatchEvalPython)) =>
+  LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, 
udf.child
+case LocalLimit(le, udf: ArrowEvalPython) =>
+  LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child)))
+case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) =>
+  LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, 
udf.child
   }
 }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
index e97ff7808f1..1ce6808be60 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.catalyst.plans.logical
 
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, 
Expression, PythonUDF}
+import org.apache.spark.sql.catalyst.trees.TreePattern._
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.s

[spark] branch master updated: [SPARK-42115][SQL] Push down limit through Python UDFs

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 0fe361e3a1b [SPARK-42115][SQL] Push down limit through Python UDFs
0fe361e3a1b is described below

commit 0fe361e3a1b5be04114402b78e62dd010703477b
Author: Kelvin Jiang 
AuthorDate: Thu Feb 2 09:00:51 2023 +0800

[SPARK-42115][SQL] Push down limit through Python UDFs

### What changes were proposed in this pull request?

This PR adds cases in LimitPushDown to push limits through Python UDFs. In 
order to allow for this, we need to call LimitPushDown in SparkOptimizer after 
the "Extract Python UDFs" batch. We also add PushProjectionThroughLimit 
afterwards in order to plan CollectLimit.

### Why are the changes needed?

Right now, LimitPushdown does not push limits through Python UDFs, which 
means that expensive Python UDFs can be run on potentially large amounts of 
input. This PR adds this capability, while ensuring that a GlobalLimit - 
LocalLimit pattern stays at the top in order to trigger the CollectLimit code 
path.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added a UT.

Closes #39842 from kelvinjian-db/SPARK-42115-limit-through-python-udfs.

Authored-by: Kelvin Jiang 
Signed-off-by: Wenchen Fan 
---
 .../sql/tests/pandas/test_pandas_udf_scalar.py |  1 +
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  9 +
 .../plans/logical/pythonLogicalOperators.scala |  3 ++
 .../spark/sql/catalyst/trees/TreePatterns.scala|  1 +
 .../spark/sql/execution/SparkOptimizer.scala   |  2 +
 .../execution/python/ExtractPythonUDFsSuite.scala  | 43 ++
 6 files changed, 59 insertions(+)

diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py 
b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
index cbb26e45d2f..33c957fac58 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_scalar.py
@@ -981,6 +981,7 @@ class ScalarPandasUDFTests(ReusedSQLTestCase):
 with self.assertRaisesRegex(Exception, "reached finally block"):
 self.spark.range(1).select(test_close(col("id"))).collect()
 
+@unittest.skip("LimitPushDown should push limits through Python UDFs so 
this won't occur")
 def test_scalar_iter_udf_close_early(self):
 tmp_dir = tempfile.mkdtemp()
 try:
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 627e3952480..1233f2207f5 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -766,6 +766,15 @@ object LimitPushDown extends Rule[LogicalPlan] {
 // Push down local limit 1 if join type is LeftSemiOrAnti and join 
condition is empty.
 case j @ Join(_, right, LeftSemiOrAnti(_), None, _) if 
!right.maxRows.exists(_ <= 1) =>
   j.copy(right = maybePushLocalLimit(Literal(1, IntegerType), right))
+// Push down limits through Python UDFs.
+case LocalLimit(le, udf: BatchEvalPython) =>
+  LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child)))
+case LocalLimit(le, p @ Project(_, udf: BatchEvalPython)) =>
+  LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, 
udf.child
+case LocalLimit(le, udf: ArrowEvalPython) =>
+  LocalLimit(le, udf.copy(child = maybePushLocalLimit(le, udf.child)))
+case LocalLimit(le, p @ Project(_, udf: ArrowEvalPython)) =>
+  LocalLimit(le, p.copy(child = udf.copy(child = maybePushLocalLimit(le, 
udf.child
   }
 }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
index e97ff7808f1..1ce6808be60 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.catalyst.plans.logical
 
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, 
Expression, PythonUDF}
+import org.apache.spark.sql.catalyst.trees.TreePattern._
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode}
 import org.apache.spark.sql.types.StructType
@@ -141,6 +142,8 @@ trait BaseEvalPython

[spark] branch branch-3.4 updated: [SPARK-41985][SQL] Centralize more column resolution rules

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 3927c5e17ab [SPARK-41985][SQL] Centralize more column resolution rules
3927c5e17ab is described below

commit 3927c5e17abbe7ea2a4fa8d4d25114a0fbf96788
Author: Wenchen Fan 
AuthorDate: Thu Feb 2 00:24:32 2023 +0800

[SPARK-41985][SQL] Centralize more column resolution rules

### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/3 .

When I search for all the matching of `UnresolvedAttribute`, I found that 
there are still a few rules doing column resolution:
1. ResolveAggAliasInGroupBy
2. ResolveGroupByAll
3. ResolveOrderByAll
4. ResolveDefaultColumns

This PR merges the first 3 into `ResolvedReferences`. The last one will be 
done with a separate PR, as it's more complicated.

To avoid making the rule `ResolvedReferences` bigger and bigger, this PR 
pulls out the resolution code for `Aggregate` to a separated virtual rule (only 
be used by `ResolvedReferences`). The same to `Sort`. We can refactor and add 
more virtual rules later.

### Why are the changes needed?

It's problematic to not centralize all the column resolution logic, as the 
execution order of the rules is not reliable. It actually leads to regression 
after https://github.com/apache/spark/pull/3  : `select a from t where 
exists (select 1 as a group by a)`. The `group by a` should be resolved as `1 
as a`, but now it's resolved as outer reference `a`. This is because 
`ResolveReferences` runs before `ResolveAggAliasInGroupBy`, and resolves outer 
references too early.

### Does this PR introduce _any_ user-facing change?

Fixes a bug, but the bug is not released yet.

### How was this patch tested?

new tests

Closes #39508 from cloud-fan/column.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 40ca27cd6e561a746c3a8f5653ccd525ddaada96)
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   |   5 +
 .../spark/sql/catalyst/analysis/Analyzer.scala | 464 ++---
 .../sql/catalyst/analysis/CheckAnalysis.scala  |   6 +-
 .../catalyst/analysis/ColumnResolutionHelper.scala | 365 
 .../sql/catalyst/analysis/ResolveGroupByAll.scala  | 119 --
 .../sql/catalyst/analysis/ResolveOrderByAll.scala  |  81 
 .../analysis/ResolveReferencesInAggregate.scala| 201 +
 .../analysis/ResolveReferencesInSort.scala |  84 
 .../inputs/column-resolution-aggregate.sql |  33 ++
 .../sql-tests/inputs/column-resolution-sort.sql|  20 +
 .../results/column-resolution-aggregate.sql.out| 129 ++
 .../results/column-resolution-sort.sql.out |  42 ++
 12 files changed, 918 insertions(+), 631 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 89a8c2a..9b083645b1b 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1521,6 +1521,11 @@
   "Referencing a lateral column alias  in the aggregate function 
."
 ]
   },
+  "LATERAL_COLUMN_ALIAS_IN_GROUP_BY" : {
+"message" : [
+  "Referencing a lateral column alias via GROUP BY alias/ALL is not 
supported yet."
+]
+  },
   "LATERAL_JOIN_USING" : {
 "message" : [
   "JOIN USING with LATERAL correlation."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index bf66afffdde..488399e00b7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -185,8 +185,8 @@ object AnalysisContext {
  * Provides a logical query plan analyzer, which translates 
[[UnresolvedAttribute]]s and
  * [[UnresolvedRelation]]s into fully typed objects using information in a 
[[SessionCatalog]].
  */
-class Analyzer(override val catalogManager: CatalogManager)
-  extends RuleExecutor[LogicalPlan] with CheckAnalysis with SQLConfHelper {
+class Analyzer(override val catalogManager: CatalogManager) extends 
RuleExecutor[LogicalPlan]
+  with CheckAnalysis with SQLConfHelper with ColumnResolutionHelper {
 
   private val v1SessionCatalog: SessionCatalog = 
catalogManager.v1SessionCatalog
 
@@ -295,10 +295,7 @@ class Analyzer(override val catalogManager: CatalogManager)
   ResolveGroupingAnalytics ::
  

[spark] branch master updated: [SPARK-41985][SQL] Centralize more column resolution rules

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 40ca27cd6e5 [SPARK-41985][SQL] Centralize more column resolution rules
40ca27cd6e5 is described below

commit 40ca27cd6e561a746c3a8f5653ccd525ddaada96
Author: Wenchen Fan 
AuthorDate: Thu Feb 2 00:24:32 2023 +0800

[SPARK-41985][SQL] Centralize more column resolution rules

### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/3 .

When I search for all the matching of `UnresolvedAttribute`, I found that 
there are still a few rules doing column resolution:
1. ResolveAggAliasInGroupBy
2. ResolveGroupByAll
3. ResolveOrderByAll
4. ResolveDefaultColumns

This PR merges the first 3 into `ResolvedReferences`. The last one will be 
done with a separate PR, as it's more complicated.

To avoid making the rule `ResolvedReferences` bigger and bigger, this PR 
pulls out the resolution code for `Aggregate` to a separated virtual rule (only 
be used by `ResolvedReferences`). The same to `Sort`. We can refactor and add 
more virtual rules later.

### Why are the changes needed?

It's problematic to not centralize all the column resolution logic, as the 
execution order of the rules is not reliable. It actually leads to regression 
after https://github.com/apache/spark/pull/3  : `select a from t where 
exists (select 1 as a group by a)`. The `group by a` should be resolved as `1 
as a`, but now it's resolved as outer reference `a`. This is because 
`ResolveReferences` runs before `ResolveAggAliasInGroupBy`, and resolves outer 
references too early.

### Does this PR introduce _any_ user-facing change?

Fixes a bug, but the bug is not released yet.

### How was this patch tested?

new tests

Closes #39508 from cloud-fan/column.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   |   5 +
 .../spark/sql/catalyst/analysis/Analyzer.scala | 464 ++---
 .../sql/catalyst/analysis/CheckAnalysis.scala  |   6 +-
 .../catalyst/analysis/ColumnResolutionHelper.scala | 365 
 .../sql/catalyst/analysis/ResolveGroupByAll.scala  | 119 --
 .../sql/catalyst/analysis/ResolveOrderByAll.scala  |  81 
 .../analysis/ResolveReferencesInAggregate.scala| 201 +
 .../analysis/ResolveReferencesInSort.scala |  84 
 .../inputs/column-resolution-aggregate.sql |  33 ++
 .../sql-tests/inputs/column-resolution-sort.sql|  20 +
 .../results/column-resolution-aggregate.sql.out| 129 ++
 .../results/column-resolution-sort.sql.out |  42 ++
 12 files changed, 918 insertions(+), 631 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 230b616800f..84af7b5d64f 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1532,6 +1532,11 @@
   "Referencing a lateral column alias  in the aggregate function 
."
 ]
   },
+  "LATERAL_COLUMN_ALIAS_IN_GROUP_BY" : {
+"message" : [
+  "Referencing a lateral column alias via GROUP BY alias/ALL is not 
supported yet."
+]
+  },
   "LATERAL_JOIN_USING" : {
 "message" : [
   "JOIN USING with LATERAL correlation."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ce273f01c7a..28ae09e123c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -185,8 +185,8 @@ object AnalysisContext {
  * Provides a logical query plan analyzer, which translates 
[[UnresolvedAttribute]]s and
  * [[UnresolvedRelation]]s into fully typed objects using information in a 
[[SessionCatalog]].
  */
-class Analyzer(override val catalogManager: CatalogManager)
-  extends RuleExecutor[LogicalPlan] with CheckAnalysis with SQLConfHelper {
+class Analyzer(override val catalogManager: CatalogManager) extends 
RuleExecutor[LogicalPlan]
+  with CheckAnalysis with SQLConfHelper with ColumnResolutionHelper {
 
   private val v1SessionCatalog: SessionCatalog = 
catalogManager.v1SessionCatalog
 
@@ -295,10 +295,7 @@ class Analyzer(override val catalogManager: CatalogManager)
   ResolveGroupingAnalytics ::
   ResolvePivot ::
   ResolveUnpivot ::
-  ResolveOrderByAll ::
-  ResolveGroupByAll ::
 

[spark] branch branch-3.4 updated: [SPARK-42278][SQL] DS V2 pushdown supports supports JDBC dialects compile `SortOrder` by themselves

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new f64f146b8c2 [SPARK-42278][SQL] DS V2 pushdown supports supports JDBC 
dialects compile `SortOrder` by themselves
f64f146b8c2 is described below

commit f64f146b8c22ed5ac98bb67115e6793c3f568ca3
Author: Jiaan Geng 
AuthorDate: Wed Feb 1 20:17:39 2023 +0800

[SPARK-42278][SQL] DS V2 pushdown supports supports JDBC dialects compile 
`SortOrder` by themselves

### What changes were proposed in this pull request?
Currently, DS V2 pushdown framework compiles the `SortOrder` with fixed 
code. These fixed code outputs the fixed syntax format, such as `ORDER BY col 
ASC NULLS FIRST`.
This is not flexible and friendly for some databases that do not support 
this syntax.
For example, `ORDER BY col ASC NULLS FIRST` is not supported by MS SQL 
Server who not recognize the syntax `NULLS FIRST`.

### Why are the changes needed?
This PR want compile the `SortOrder` with `V2ExpressionSQLBuilder`'s 
`visitSortOrder`, so that JDBC dialects could compile `SortOrder` by themselves.

### Does this PR introduce _any_ user-facing change?
'No'.
New feature.

### How was this patch tested?
Exists test cases.

Closes #39846 from beliefer/SPARK-42278.

Authored-by: Jiaan Geng 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 20eb54661e5bf6e2e350b8311007a63f7beabc7a)
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/connector/util/V2ExpressionSQLBuilder.java | 12 
 .../sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala  |  5 +
 2 files changed, 13 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
index fe16174586b..9ca0fe4787f 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
@@ -27,6 +27,9 @@ import org.apache.spark.sql.connector.expressions.Extract;
 import org.apache.spark.sql.connector.expressions.NamedReference;
 import org.apache.spark.sql.connector.expressions.GeneralScalarExpression;
 import org.apache.spark.sql.connector.expressions.Literal;
+import org.apache.spark.sql.connector.expressions.NullOrdering;
+import org.apache.spark.sql.connector.expressions.SortDirection;
+import org.apache.spark.sql.connector.expressions.SortOrder;
 import org.apache.spark.sql.connector.expressions.UserDefinedScalarFunc;
 import org.apache.spark.sql.connector.expressions.aggregate.Avg;
 import org.apache.spark.sql.connector.expressions.aggregate.Max;
@@ -56,6 +59,10 @@ public class V2ExpressionSQLBuilder {
 } else if (expr instanceof Extract) {
   Extract extract = (Extract) expr;
   return visitExtract(extract.field(), build(extract.source()));
+} else if (expr instanceof SortOrder) {
+  SortOrder sortOrder = (SortOrder) expr;
+  return visitSortOrder(
+build(sortOrder.expression()), sortOrder.direction(), 
sortOrder.nullOrdering());
 } else if (expr instanceof GeneralScalarExpression) {
   GeneralScalarExpression e = (GeneralScalarExpression) expr;
   String name = e.name();
@@ -368,6 +375,11 @@ public class V2ExpressionSQLBuilder {
 return "EXTRACT(" + field + " FROM " + source + ")";
   }
 
+  protected String visitSortOrder(
+  String sortKey, SortDirection sortDirection, NullOrdering nullOrdering) {
+return sortKey + " " + sortDirection + " " + nullOrdering;
+  }
+
   private String joinArrayToString(
   String[] inputs, CharSequence delimiter, CharSequence prefix, 
CharSequence suffix) {
 StringJoiner joiner = new StringJoiner(delimiter, prefix, suffix);
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
index 5fc2edc042c..4c62c4c1c4a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
@@ -158,10 +158,7 @@ case class JDBCScanBuilder(
   override def pushTopN(orders: Array[SortOrder], limit: Int): Boolean = {
 if (jdbcOptions.pushDownLimit) {
   val dialect = JdbcDialects.get(jdbcOptions.url)
-  val compiledOrders = orders.flatMap { order =>
-dialect.compileExpression(order.expression())
-  .map(sortKey => s"$sortKey ${order.direction()} 
${order.nullOrde

[spark] branch master updated: [SPARK-42278][SQL] DS V2 pushdown supports supports JDBC dialects compile `SortOrder` by themselves

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 20eb54661e5 [SPARK-42278][SQL] DS V2 pushdown supports supports JDBC 
dialects compile `SortOrder` by themselves
20eb54661e5 is described below

commit 20eb54661e5bf6e2e350b8311007a63f7beabc7a
Author: Jiaan Geng 
AuthorDate: Wed Feb 1 20:17:39 2023 +0800

[SPARK-42278][SQL] DS V2 pushdown supports supports JDBC dialects compile 
`SortOrder` by themselves

### What changes were proposed in this pull request?
Currently, DS V2 pushdown framework compiles the `SortOrder` with fixed 
code. These fixed code outputs the fixed syntax format, such as `ORDER BY col 
ASC NULLS FIRST`.
This is not flexible and friendly for some databases that do not support 
this syntax.
For example, `ORDER BY col ASC NULLS FIRST` is not supported by MS SQL 
Server who not recognize the syntax `NULLS FIRST`.

### Why are the changes needed?
This PR want compile the `SortOrder` with `V2ExpressionSQLBuilder`'s 
`visitSortOrder`, so that JDBC dialects could compile `SortOrder` by themselves.

### Does this PR introduce _any_ user-facing change?
'No'.
New feature.

### How was this patch tested?
Exists test cases.

Closes #39846 from beliefer/SPARK-42278.

Authored-by: Jiaan Geng 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/connector/util/V2ExpressionSQLBuilder.java | 12 
 .../sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala  |  5 +
 2 files changed, 13 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
index fe16174586b..9ca0fe4787f 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
@@ -27,6 +27,9 @@ import org.apache.spark.sql.connector.expressions.Extract;
 import org.apache.spark.sql.connector.expressions.NamedReference;
 import org.apache.spark.sql.connector.expressions.GeneralScalarExpression;
 import org.apache.spark.sql.connector.expressions.Literal;
+import org.apache.spark.sql.connector.expressions.NullOrdering;
+import org.apache.spark.sql.connector.expressions.SortDirection;
+import org.apache.spark.sql.connector.expressions.SortOrder;
 import org.apache.spark.sql.connector.expressions.UserDefinedScalarFunc;
 import org.apache.spark.sql.connector.expressions.aggregate.Avg;
 import org.apache.spark.sql.connector.expressions.aggregate.Max;
@@ -56,6 +59,10 @@ public class V2ExpressionSQLBuilder {
 } else if (expr instanceof Extract) {
   Extract extract = (Extract) expr;
   return visitExtract(extract.field(), build(extract.source()));
+} else if (expr instanceof SortOrder) {
+  SortOrder sortOrder = (SortOrder) expr;
+  return visitSortOrder(
+build(sortOrder.expression()), sortOrder.direction(), 
sortOrder.nullOrdering());
 } else if (expr instanceof GeneralScalarExpression) {
   GeneralScalarExpression e = (GeneralScalarExpression) expr;
   String name = e.name();
@@ -368,6 +375,11 @@ public class V2ExpressionSQLBuilder {
 return "EXTRACT(" + field + " FROM " + source + ")";
   }
 
+  protected String visitSortOrder(
+  String sortKey, SortDirection sortDirection, NullOrdering nullOrdering) {
+return sortKey + " " + sortDirection + " " + nullOrdering;
+  }
+
   private String joinArrayToString(
   String[] inputs, CharSequence delimiter, CharSequence prefix, 
CharSequence suffix) {
 StringJoiner joiner = new StringJoiner(delimiter, prefix, suffix);
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
index 5fc2edc042c..4c62c4c1c4a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
@@ -158,10 +158,7 @@ case class JDBCScanBuilder(
   override def pushTopN(orders: Array[SortOrder], limit: Int): Boolean = {
 if (jdbcOptions.pushDownLimit) {
   val dialect = JdbcDialects.get(jdbcOptions.url)
-  val compiledOrders = orders.flatMap { order =>
-dialect.compileExpression(order.expression())
-  .map(sortKey => s"$sortKey ${order.direction()} 
${order.nullOrdering()}")
-  }
+  val compiledOrders = orders.flatMap(dialect.compileExpression(_))
   if (orders.length

[spark] branch branch-3.2 updated: [SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new e9734962442 [SPARK-42259][SQL] ResolveGroupingAnalytics should take 
care of Python UDAF
e9734962442 is described below

commit e97349624421afee07a68fda38027cfe7b796597
Author: Wenchen Fan 
AuthorDate: Wed Feb 1 17:36:14 2023 +0800

[SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

This is a long-standing correctness issue with Python UDAF and grouping 
analytics. The rule `ResolveGroupingAnalytics` should take care of Python UDAF 
when matching aggregate expressions.

bug fix

Yes, the query result was wrong before

existing tests

Closes #39824 from cloud-fan/python.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 1219c8492376e038894111cd5d99260482e7)
Signed-off-by: Wenchen Fan 
---
 .../main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 4d1ea95d0e7..8c304d8a400 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -610,7 +610,7 @@ class Analyzer(override val catalogManager: CatalogManager)
 // AggregateExpression should be computed on the unmodified value of 
its argument
 // expressions, so we should not replace any references to grouping 
expression
 // inside it.
-case e: AggregateExpression =>
+case e if AggregateExpression.isAggregate(e) =>
   aggsBuffer += e
   e
 case e if isPartOfAggregation(e) => e


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.3 updated: [SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 80e8df11d7e [SPARK-42259][SQL] ResolveGroupingAnalytics should take 
care of Python UDAF
80e8df11d7e is described below

commit 80e8df11d7e2c135ef707c1c1626b976a8dc09a0
Author: Wenchen Fan 
AuthorDate: Wed Feb 1 17:36:14 2023 +0800

[SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

This is a long-standing correctness issue with Python UDAF and grouping 
analytics. The rule `ResolveGroupingAnalytics` should take care of Python UDAF 
when matching aggregate expressions.

bug fix

Yes, the query result was wrong before

existing tests

Closes #39824 from cloud-fan/python.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 1219c8492376e038894111cd5d99260482e7)
Signed-off-by: Wenchen Fan 
---
 .../main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 84aa06baaff..881f2cc2078 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -617,7 +617,7 @@ class Analyzer(override val catalogManager: CatalogManager)
 // AggregateExpression should be computed on the unmodified value of 
its argument
 // expressions, so we should not replace any references to grouping 
expression
 // inside it.
-case e: AggregateExpression =>
+case e if AggregateExpression.isAggregate(e) =>
   aggsBuffer += e
   e
 case e if isPartOfAggregation(e) => e


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 5e9ad2e71f9 [SPARK-42259][SQL] ResolveGroupingAnalytics should take 
care of Python UDAF
5e9ad2e71f9 is described below

commit 5e9ad2e71f9a22d2f1b2d47b2e6bee323b354eb7
Author: Wenchen Fan 
AuthorDate: Wed Feb 1 17:36:14 2023 +0800

[SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

### What changes were proposed in this pull request?

This is a long-standing correctness issue with Python UDAF and grouping 
analytics. The rule `ResolveGroupingAnalytics` should take care of Python UDAF 
when matching aggregate expressions.

### Why are the changes needed?

bug fix

### Does this PR introduce _any_ user-facing change?

Yes, the query result was wrong before

### How was this patch tested?

existing tests

Closes #39824 from cloud-fan/python.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 1219c8492376e038894111cd5d99260482e7)
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  2 +-
 .../results/udaf/udaf-group-analytics.sql.out  | 58 +++---
 2 files changed, 30 insertions(+), 30 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 48ea0460725..7a92c46577d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -622,7 +622,7 @@ class Analyzer(override val catalogManager: CatalogManager)
 // AggregateExpression should be computed on the unmodified value of 
its argument
 // expressions, so we should not replace any references to grouping 
expression
 // inside it.
-case e: AggregateExpression =>
+case e if AggregateExpression.isAggregate(e) =>
   aggsBuffer += e
   e
 case e if isPartOfAggregation(e) => e
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
index b8c94b19d81..f0be6f43642 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
@@ -15,18 +15,18 @@ SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, 
b WITH CUBE
 struct<(a + b):int,b:int,udaf((a - b)):int>
 -- !query output
 2  1   1
-2  NULL0
+2  NULL1
 3  1   1
 3  2   1
-3  NULL0
+3  NULL2
 4  1   1
 4  2   1
-4  NULL0
+4  NULL2
 5  2   1
-5  NULL0
+5  NULL1
 NULL   1   3
 NULL   2   3
-NULL   NULL0
+NULL   NULL6
 
 
 -- !query
@@ -36,16 +36,16 @@ struct
 -- !query output
 1  1   1
 1  2   1
-1  NULL0
+1  NULL2
 2  1   1
 2  2   1
-2  NULL0
+2  NULL2
 3  1   1
 3  2   1
-3  NULL0
+3  NULL2
 NULL   1   3
 NULL   2   3
-NULL   NULL0
+NULL   NULL6
 
 
 -- !query
@@ -54,16 +54,16 @@ SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, 
b WITH ROLLUP
 struct<(a + b):int,b:int,udaf((a - b)):int>
 -- !query output
 2  1   1
-2  NULL0
+2  NULL1
 3  1   1
 3  2   1
-3  NULL0
+3  NULL2
 4  1   1
 4  2   1
-4  NULL0
+4  NULL2
 5  2   1
-5  NULL0
-NULL   NULL0
+5  NULL1
+NULL   NULL6
 
 
 -- !query
@@ -73,14 +73,14 @@ struct
 -- !query output
 1  1   1
 1  2   1
-1  NULL0
+1  NULL2
 2  1   1
 2  2   1
-2  NULL0
+2  NULL2
 3  1   1
 3  2   1
-3  NULL0
-NULL   NULL0
+3  NULL2
+NULL   NULL6
 
 
 -- !query
@@ -416,14 +416,14 @@ GROUP BY course, earnings GROUPING SETS((), (course), 
(course, earnings)) ORDER
 -- !query schema
 struct
 -- !query output
-NULL   0
-Java   0
+NULL   5
 Java   1
 Java   1
-dotNET 0
+Java   2
 dotNET 1
 dotNET 1
 dotNET 1
+dotNET 3
 
 
 -- !query
@@ -432,14 +432,14 @@ GROUP BY course, earnings GROUPING SETS((), (course), 
(course, earnings)) ORDER
 -- !query schema
 struct
 -- !query output
-NULL   0   3
-Java   0   1
+NULL   5   3
 Java   1   0
 Java   1   0
-dotNET 0   1
+Java   2   1
 dotNET 1   0
 dotNET 1   0
 dotNET 1   0
+dotNET 3   1
 
 

[spark] branch master updated: [SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

2023-02-01 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 1219c849237 [SPARK-42259][SQL] ResolveGroupingAnalytics should take 
care of Python UDAF
1219c849237 is described below

commit 1219c8492376e038894111cd5d99260482e7
Author: Wenchen Fan 
AuthorDate: Wed Feb 1 17:36:14 2023 +0800

[SPARK-42259][SQL] ResolveGroupingAnalytics should take care of Python UDAF

### What changes were proposed in this pull request?

This is a long-standing correctness issue with Python UDAF and grouping 
analytics. The rule `ResolveGroupingAnalytics` should take care of Python UDAF 
when matching aggregate expressions.

### Why are the changes needed?

bug fix

### Does this PR introduce _any_ user-facing change?

Yes, the query result was wrong before

### How was this patch tested?

existing tests

Closes #39824 from cloud-fan/python.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  2 +-
 .../results/udaf/udaf-group-analytics.sql.out  | 58 +++---
 2 files changed, 30 insertions(+), 30 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 8f6028b0993..ce273f01c7a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -622,7 +622,7 @@ class Analyzer(override val catalogManager: CatalogManager)
 // AggregateExpression should be computed on the unmodified value of 
its argument
 // expressions, so we should not replace any references to grouping 
expression
 // inside it.
-case e: AggregateExpression =>
+case e if AggregateExpression.isAggregate(e) =>
   aggsBuffer += e
   e
 case e if isPartOfAggregation(e) => e
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
index b8c94b19d81..f0be6f43642 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-analytics.sql.out
@@ -15,18 +15,18 @@ SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, 
b WITH CUBE
 struct<(a + b):int,b:int,udaf((a - b)):int>
 -- !query output
 2  1   1
-2  NULL0
+2  NULL1
 3  1   1
 3  2   1
-3  NULL0
+3  NULL2
 4  1   1
 4  2   1
-4  NULL0
+4  NULL2
 5  2   1
-5  NULL0
+5  NULL1
 NULL   1   3
 NULL   2   3
-NULL   NULL0
+NULL   NULL6
 
 
 -- !query
@@ -36,16 +36,16 @@ struct
 -- !query output
 1  1   1
 1  2   1
-1  NULL0
+1  NULL2
 2  1   1
 2  2   1
-2  NULL0
+2  NULL2
 3  1   1
 3  2   1
-3  NULL0
+3  NULL2
 NULL   1   3
 NULL   2   3
-NULL   NULL0
+NULL   NULL6
 
 
 -- !query
@@ -54,16 +54,16 @@ SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, 
b WITH ROLLUP
 struct<(a + b):int,b:int,udaf((a - b)):int>
 -- !query output
 2  1   1
-2  NULL0
+2  NULL1
 3  1   1
 3  2   1
-3  NULL0
+3  NULL2
 4  1   1
 4  2   1
-4  NULL0
+4  NULL2
 5  2   1
-5  NULL0
-NULL   NULL0
+5  NULL1
+NULL   NULL6
 
 
 -- !query
@@ -73,14 +73,14 @@ struct
 -- !query output
 1  1   1
 1  2   1
-1  NULL0
+1  NULL2
 2  1   1
 2  2   1
-2  NULL0
+2  NULL2
 3  1   1
 3  2   1
-3  NULL0
-NULL   NULL0
+3  NULL2
+NULL   NULL6
 
 
 -- !query
@@ -416,14 +416,14 @@ GROUP BY course, earnings GROUPING SETS((), (course), 
(course, earnings)) ORDER
 -- !query schema
 struct
 -- !query output
-NULL   0
-Java   0
+NULL   5
 Java   1
 Java   1
-dotNET 0
+Java   2
 dotNET 1
 dotNET 1
 dotNET 1
+dotNET 3
 
 
 -- !query
@@ -432,14 +432,14 @@ GROUP BY course, earnings GROUPING SETS((), (course), 
(course, earnings)) ORDER
 -- !query schema
 struct
 -- !query output
-NULL   0   3
-Java   0   1
+NULL   5   3
 Java   1   0
 Java   1   0
-dotNET 0   1
+Java   2   1
 dotNET 1   0
 dotNET 1   0
 dotNET 1   0
+dotNET 3   1
 
 
 -- !query
@@ -468,16 +468,16 @@ SELECT a + b AS k, b, udaf(a - b) FROM testData GROUP BY 
ROLLUP(k, b)
 struct
 -- !q

[spark] branch master updated (4d37e7816ce -> b0ac0612494)

2023-01-31 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 4d37e7816ce [SPARK-42253][PYTHON] Add test for detecting duplicated 
error class
 add b0ac0612494 [SPARK-42191][SQL] Support udf 'luhn_check'

No new revisions were added by this update.

Summary of changes:
 .../catalyst/expressions/ExpressionImplUtils.java  |  29 +
 .../sql/catalyst/analysis/FunctionRegistry.scala   |   1 +
 .../catalyst/expressions/stringExpressions.scala   |  40 ++
 .../sql-functions/sql-expression-schema.md |   1 +
 .../sql-tests/inputs/string-functions.sql  |  26 
 .../results/ansi/string-functions.sql.out  | 144 +
 .../sql-tests/results/string-functions.sql.out | 144 +
 7 files changed, 385 insertions(+)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

2023-01-31 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new b122436564c [SPARK-40086][SPARK-42049][SQL] Improve 
AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all 
aliases into account
b122436564c is described below

commit b122436564cad31987e6dda6eb15e0f95516a74a
Author: Peter Toth 
AuthorDate: Tue Jan 31 23:07:57 2023 +0800

[SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and 
AliasAwareQueryOutputOrdering to take all aliases into account

### What changes were proposed in this pull request?
Currently `AliasAwareOutputPartitioning` and `AliasAwareQueryOutputOrdering`
takes only the last alias by aliased expressions into account. We could 
avoid some extra shuffles and sorts with better alias handling.

### Why are the changes needed?
Performance improvement and this also fix the issue in 
https://github.com/apache/spark/pull/39475.

### Does this PR introduce _any_ user-facing change?
Yes, this PR fixes the issue in https://github.com/apache/spark/pull/39475.

### How was this patch tested?
Added new UT.

Closes #37525 from peter-toth/SPARK-40086-fix-aliasawareoutputexpression.

Lead-authored-by: Peter Toth 
Co-authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 6341b06f66bc8f919d086341d5b15157ada3b5e0)
Signed-off-by: Wenchen Fan 
---
 .../catalyst/expressions/stringExpressions.scala   |  24 +++
 .../plans/AliasAwareOutputExpression.scala | 120 ++
 .../spark/sql/catalyst/plans/QueryPlan.scala   |   7 +
 .../sql/catalyst/plans/logical/LogicalPlan.scala   |  13 +-
 .../plans/logical/basicLogicalOperators.scala  |   1 +
 .../org/apache/spark/sql/internal/SQLConf.scala|  10 ++
 .../sql/execution/AliasAwareOutputExpression.scala |  75 +++--
 .../org/apache/spark/sql/execution/SparkPlan.scala |   3 -
 .../execution/aggregate/BaseAggregateExec.scala|   4 +-
 .../execution/aggregate/SortAggregateExec.scala|   4 +-
 .../sql/execution/basicPhysicalOperators.scala |   4 +-
 .../execution/datasources/FileFormatWriter.scala   |   3 +-
 .../spark/sql/execution/datasources/V1Writes.scala |  29 +---
 .../org/apache/spark/sql/execution/limit.scala |   2 +-
 .../scala/org/apache/spark/sql/ExplainSuite.scala  |   2 +-
 .../execution/CoalesceShufflePartitionsSuite.scala |   8 +-
 .../apache/spark/sql/execution/PlannerSuite.scala  |  67 +++-
 .../ProjectedOrderingAndPartitioningSuite.scala| 180 +
 .../datasources/V1WriteCommandSuite.scala  |  27 ++--
 19 files changed, 464 insertions(+), 119 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 590582eee07..c1ca86b356e 100755
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -3024,3 +3024,27 @@ case class SplitPart (
   partNum = newChildren.apply(2))
   }
 }
+
+/**
+ * A internal function that converts the empty string to null for partition 
values.
+ * This function should be only used in V1Writes.
+ */
+case class Empty2Null(child: Expression) extends UnaryExpression with 
String2StringExpression {
+  override def convert(v: UTF8String): UTF8String = if (v.numBytes() == 0) 
null else v
+
+  override def nullable: Boolean = true
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+nullSafeCodeGen(ctx, ev, c => {
+  s"""if ($c.numBytes() == 0) {
+ |  ${ev.isNull} = true;
+ |  ${ev.value} = null;
+ |} else {
+ |  ${ev.value} = $c;
+ |}""".stripMargin
+})
+  }
+
+  override protected def withNewChildInternal(newChild: Expression): 
Empty2Null =
+copy(child = newChild)
+}
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala
new file mode 100644
index 000..4d9d69d14fe
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala
@@ -0,0 +1,120 @@
+/*
+ * 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

[spark] branch master updated: [SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all aliases into account

2023-01-31 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6341b06f66b [SPARK-40086][SPARK-42049][SQL] Improve 
AliasAwareOutputPartitioning and AliasAwareQueryOutputOrdering to take all 
aliases into account
6341b06f66b is described below

commit 6341b06f66bc8f919d086341d5b15157ada3b5e0
Author: Peter Toth 
AuthorDate: Tue Jan 31 23:07:57 2023 +0800

[SPARK-40086][SPARK-42049][SQL] Improve AliasAwareOutputPartitioning and 
AliasAwareQueryOutputOrdering to take all aliases into account

### What changes were proposed in this pull request?
Currently `AliasAwareOutputPartitioning` and `AliasAwareQueryOutputOrdering`
takes only the last alias by aliased expressions into account. We could 
avoid some extra shuffles and sorts with better alias handling.

### Why are the changes needed?
Performance improvement and this also fix the issue in 
https://github.com/apache/spark/pull/39475.

### Does this PR introduce _any_ user-facing change?
Yes, this PR fixes the issue in https://github.com/apache/spark/pull/39475.

### How was this patch tested?
Added new UT.

Closes #37525 from peter-toth/SPARK-40086-fix-aliasawareoutputexpression.

Lead-authored-by: Peter Toth 
Co-authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../catalyst/expressions/stringExpressions.scala   |  24 +++
 .../plans/AliasAwareOutputExpression.scala | 120 ++
 .../spark/sql/catalyst/plans/QueryPlan.scala   |   7 +
 .../sql/catalyst/plans/logical/LogicalPlan.scala   |  13 +-
 .../plans/logical/basicLogicalOperators.scala  |   1 +
 .../org/apache/spark/sql/internal/SQLConf.scala|  10 ++
 .../sql/execution/AliasAwareOutputExpression.scala |  75 +++--
 .../org/apache/spark/sql/execution/SparkPlan.scala |   3 -
 .../execution/aggregate/BaseAggregateExec.scala|   4 +-
 .../execution/aggregate/SortAggregateExec.scala|   4 +-
 .../sql/execution/basicPhysicalOperators.scala |   4 +-
 .../execution/datasources/FileFormatWriter.scala   |   3 +-
 .../spark/sql/execution/datasources/V1Writes.scala |  29 +---
 .../org/apache/spark/sql/execution/limit.scala |   2 +-
 .../scala/org/apache/spark/sql/ExplainSuite.scala  |   2 +-
 .../execution/CoalesceShufflePartitionsSuite.scala |   8 +-
 .../apache/spark/sql/execution/PlannerSuite.scala  |  67 +++-
 .../ProjectedOrderingAndPartitioningSuite.scala| 180 +
 .../datasources/V1WriteCommandSuite.scala  |  27 ++--
 19 files changed, 464 insertions(+), 119 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 590582eee07..c1ca86b356e 100755
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -3024,3 +3024,27 @@ case class SplitPart (
   partNum = newChildren.apply(2))
   }
 }
+
+/**
+ * A internal function that converts the empty string to null for partition 
values.
+ * This function should be only used in V1Writes.
+ */
+case class Empty2Null(child: Expression) extends UnaryExpression with 
String2StringExpression {
+  override def convert(v: UTF8String): UTF8String = if (v.numBytes() == 0) 
null else v
+
+  override def nullable: Boolean = true
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+nullSafeCodeGen(ctx, ev, c => {
+  s"""if ($c.numBytes() == 0) {
+ |  ${ev.isNull} = true;
+ |  ${ev.value} = null;
+ |} else {
+ |  ${ev.value} = $c;
+ |}""".stripMargin
+})
+  }
+
+  override protected def withNewChildInternal(newChild: Expression): 
Empty2Null =
+copy(child = newChild)
+}
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala
new file mode 100644
index 000..4d9d69d14fe
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/AliasAwareOutputExpression.scala
@@ -0,0 +1,120 @@
+/*
+ * 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
+ *
+ *   

[spark] branch master updated (11a75371705 -> 2c104c3bdde)

2023-01-31 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 11a75371705 [SPARK-42229][CORE] Migrate `SparkCoreErrors` into error 
classes
 add 2c104c3bdde [SPARK-41488][SQL] Assign name to _LEGACY_ERROR_TEMP_1176 
(and 1177)

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/error-classes.json   |  17 ++--
 .../spark/sql/catalyst/analysis/Analyzer.scala |   2 +-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  11 +--
 .../sql-tests/inputs/columnresolution-negative.sql |  14 +++
 .../results/columnresolution-negative.sql.out  | 100 +
 .../spark/sql/execution/SQLViewTestSuite.scala |   6 +-
 6 files changed, 130 insertions(+), 20 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-41219][SQL] IntegralDivide use decimal(1, 0) to represent 0

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new e3a02a1b6e1 [SPARK-41219][SQL] IntegralDivide use decimal(1, 0) to 
represent 0
e3a02a1b6e1 is described below

commit e3a02a1b6e148622e1d763702ef9ae16e8e1f0af
Author: ulysses-you 
AuthorDate: Tue Jan 31 11:58:14 2023 +0800

[SPARK-41219][SQL] IntegralDivide use decimal(1, 0) to represent 0

### What changes were proposed in this pull request?

0 is a special case for decimal which data type can be Decimal(0, 0), to be 
safe we should use decimal(1, 0) to represent 0.

### Why are the changes needed?

fix data correctness for regression.

We do not promote the decimal precision since we refactor decimal binary 
operater in https://github.com/apache/spark/pull/36698. However, it causes the 
intermediate decimal type of `IntegralDivide` returns decimal(0, 0). It's 
dangerous that Spark does not actually support decimal(0, 0). e.g.
```sql
-- work with in-memory catalog
create table t (c decimal(0, 0)) using parquet;
-- fail with parquet
-- java.lang.IllegalArgumentException: Invalid DECIMAL precision: 0
--  at 
org.apache.parquet.Preconditions.checkArgument(Preconditions.java:57)
insert into table t values(0);

-- fail with hive catalog
-- Caused by: java.lang.IllegalArgumentException: Decimal precision out of 
allowed range [1,38]
--  at 
org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils.validateParameter(HiveDecimalUtils.java:44)
create table t (c decimal(0, 0)) using parquet;
```
And decimal(0, 0) means the data is 0, so to be safe we use decimal(1, 0) 
to represent 0 for `IntegralDivide`.

### Does this PR introduce _any_ user-facing change?

yes, bug fix

### How was this patch tested?

add test

Closes #38760 from ulysses-you/SPARK-41219.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
(cherry picked from commit a056f694386cc074219914debf7023f5935bc37d)
Signed-off-by: Wenchen Fan 
---
 .../scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala | 4 +++-
 sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala| 5 +
 2 files changed, 8 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 69730f4b8d6..d5694e58cc9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -865,7 +865,9 @@ case class IntegralDivide(
 // This follows division rule
 val intDig = p1 - s1 + s2
 // No precision loss can happen as the result scale is 0.
-DecimalType.bounded(intDig, 0)
+// If intDig is 0 that means the result data is 0, to be safe we use 
decimal(1, 0)
+// to represent 0.
+DecimalType.bounded(if (intDig == 0) 1 else intDig, 0)
   }
 
   override def sqlOperator: String = "div"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 72e459ca198..12eb52bc168 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -3584,6 +3584,11 @@ class DataFrameSuite extends QueryTest
   assert(row.getInt(0).toString == row.getString(3))
 }
   }
+
+  test("SPARK-41219: IntegralDivide use decimal(1, 0) to represent 0") {
+val df = Seq("0.5944910").toDF("a")
+checkAnswer(df.selectExpr("cast(a as decimal(7,7)) div 100"), Row(0))
+  }
 }
 
 case class GroupByKey(a: Int, b: Int)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41219][SQL] IntegralDivide use decimal(1, 0) to represent 0

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new a056f694386 [SPARK-41219][SQL] IntegralDivide use decimal(1, 0) to 
represent 0
a056f694386 is described below

commit a056f694386cc074219914debf7023f5935bc37d
Author: ulysses-you 
AuthorDate: Tue Jan 31 11:58:14 2023 +0800

[SPARK-41219][SQL] IntegralDivide use decimal(1, 0) to represent 0

### What changes were proposed in this pull request?

0 is a special case for decimal which data type can be Decimal(0, 0), to be 
safe we should use decimal(1, 0) to represent 0.

### Why are the changes needed?

fix data correctness for regression.

We do not promote the decimal precision since we refactor decimal binary 
operater in https://github.com/apache/spark/pull/36698. However, it causes the 
intermediate decimal type of `IntegralDivide` returns decimal(0, 0). It's 
dangerous that Spark does not actually support decimal(0, 0). e.g.
```sql
-- work with in-memory catalog
create table t (c decimal(0, 0)) using parquet;
-- fail with parquet
-- java.lang.IllegalArgumentException: Invalid DECIMAL precision: 0
--  at 
org.apache.parquet.Preconditions.checkArgument(Preconditions.java:57)
insert into table t values(0);

-- fail with hive catalog
-- Caused by: java.lang.IllegalArgumentException: Decimal precision out of 
allowed range [1,38]
--  at 
org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils.validateParameter(HiveDecimalUtils.java:44)
create table t (c decimal(0, 0)) using parquet;
```
And decimal(0, 0) means the data is 0, so to be safe we use decimal(1, 0) 
to represent 0 for `IntegralDivide`.

### Does this PR introduce _any_ user-facing change?

yes, bug fix

### How was this patch tested?

add test

Closes #38760 from ulysses-you/SPARK-41219.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala | 4 +++-
 sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala| 5 +
 2 files changed, 8 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 69730f4b8d6..d5694e58cc9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -865,7 +865,9 @@ case class IntegralDivide(
 // This follows division rule
 val intDig = p1 - s1 + s2
 // No precision loss can happen as the result scale is 0.
-DecimalType.bounded(intDig, 0)
+// If intDig is 0 that means the result data is 0, to be safe we use 
decimal(1, 0)
+// to represent 0.
+DecimalType.bounded(if (intDig == 0) 1 else intDig, 0)
   }
 
   override def sqlOperator: String = "div"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 72e459ca198..12eb52bc168 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -3584,6 +3584,11 @@ class DataFrameSuite extends QueryTest
   assert(row.getInt(0).toString == row.getString(3))
 }
   }
+
+  test("SPARK-41219: IntegralDivide use decimal(1, 0) to represent 0") {
+val df = Seq("0.5944910").toDF("a")
+checkAnswer(df.selectExpr("cast(a as decimal(7,7)) div 100"), Row(0))
+  }
 }
 
 case class GroupByKey(a: Int, b: Int)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-42231][SQL] Turn `MISSING_STATIC_PARTITION_COLUMN` into `internalError`

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 29b28186001 [SPARK-42231][SQL] Turn `MISSING_STATIC_PARTITION_COLUMN` 
into `internalError`
29b28186001 is described below

commit 29b28186001671d7280261b041cf7d1975578bec
Author: itholic 
AuthorDate: Tue Jan 31 10:27:17 2023 +0800

[SPARK-42231][SQL] Turn `MISSING_STATIC_PARTITION_COLUMN` into 
`internalError`

### What changes were proposed in this pull request?

This PR proposes to turn `MISSING_STATIC_PARTITION_COLUMN` into 
`internalError`.

### Why are the changes needed?

`MISSING_STATIC_PARTITION_COLUMN` is not reproducible from user space, so 
it should not be exposed into error classes.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

The existing CI should pass.

Closes #39797 from itholic/MISSING_STATIC_PARTITION_COLUMN.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
(cherry picked from commit e5933448ac0f99cb2b0b2b594807bb7a22404e18)
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json| 6 --
 .../scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala  | 4 +---
 2 files changed, 1 insertion(+), 9 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index af5e17d56d4..d881e48d604 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -958,12 +958,6 @@
 ],
 "sqlState" : "42803"
   },
-  "MISSING_STATIC_PARTITION_COLUMN" : {
-"message" : [
-  "Unknown static partition column: ."
-],
-"sqlState" : "42000"
-  },
   "MULTI_UDF_INTERFACE_ERROR" : {
 "message" : [
   "Not allowed to implement multiple UDF interfaces, UDF class 
."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index eddd30839eb..c43806449a8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -157,9 +157,7 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase {
   }
 
   def missingStaticPartitionColumn(staticName: String): Throwable = {
-new AnalysisException(
-  errorClass = "MISSING_STATIC_PARTITION_COLUMN",
-  messageParameters = Map("columnName" -> staticName))
+SparkException.internalError(s"Unknown static partition column: 
$staticName.")
   }
 
   def staticPartitionInUserSpecifiedColumnsError(staticName: String): 
Throwable = {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-42231][SQL] Turn `MISSING_STATIC_PARTITION_COLUMN` into `internalError`

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new e5933448ac0 [SPARK-42231][SQL] Turn `MISSING_STATIC_PARTITION_COLUMN` 
into `internalError`
e5933448ac0 is described below

commit e5933448ac0f99cb2b0b2b594807bb7a22404e18
Author: itholic 
AuthorDate: Tue Jan 31 10:27:17 2023 +0800

[SPARK-42231][SQL] Turn `MISSING_STATIC_PARTITION_COLUMN` into 
`internalError`

### What changes were proposed in this pull request?

This PR proposes to turn `MISSING_STATIC_PARTITION_COLUMN` into 
`internalError`.

### Why are the changes needed?

`MISSING_STATIC_PARTITION_COLUMN` is not reproducible from user space, so 
it should not be exposed into error classes.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

The existing CI should pass.

Closes #39797 from itholic/MISSING_STATIC_PARTITION_COLUMN.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json| 6 --
 .../scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala  | 4 +---
 2 files changed, 1 insertion(+), 9 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 172308a8713..e589ea22bf2 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -986,12 +986,6 @@
 ],
 "sqlState" : "42803"
   },
-  "MISSING_STATIC_PARTITION_COLUMN" : {
-"message" : [
-  "Unknown static partition column: ."
-],
-"sqlState" : "42000"
-  },
   "MULTI_UDF_INTERFACE_ERROR" : {
 "message" : [
   "Not allowed to implement multiple UDF interfaces, UDF class 
."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 7530008093f..6eb98db0d77 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -157,9 +157,7 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase {
   }
 
   def missingStaticPartitionColumn(staticName: String): Throwable = {
-new AnalysisException(
-  errorClass = "MISSING_STATIC_PARTITION_COLUMN",
-  messageParameters = Map("columnName" -> staticName))
+SparkException.internalError(s"Unknown static partition column: 
$staticName.")
   }
 
   def staticPartitionInUserSpecifiedColumnsError(staticName: String): 
Throwable = {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-42163][SQL] Fix schema pruning for non-foldable array index or map key

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 1d6b2a07fe5 [SPARK-42163][SQL] Fix schema pruning for non-foldable 
array index or map key
1d6b2a07fe5 is described below

commit 1d6b2a07fe5d66832e740c4e9d669f9cfd51dab1
Author: cashmand 
AuthorDate: Tue Jan 31 10:16:15 2023 +0800

[SPARK-42163][SQL] Fix schema pruning for non-foldable array index or map 
key

### What changes were proposed in this pull request?

In parquet schema pruning, we use SelectedField to try to extract the field 
that is used in a struct. It looks through GetArrayItem/GetMapItem, but when 
doing so, it ignores the index/key, which may itself be a struct field. If it 
is a struct field that is not selected in some other expression, and another 
field of the same attribute is selected, then pruning will drop the field, 
resulting in an optimizer error.

This change modifies SelectedField to only look through 
GetArrayItem/GetMapItem if the index/key argument is foldable. The equivalent 
code for `ElementAt` was already doing the same thing, so this just makes them 
consistent.

In principle, we could continue to traverse through these expressions, we'd 
just need to make sure that the index/key expression was also surfaced to 
column pruning as an expression that needs to be examined. But this seems like 
a fairly non-trivial change to the design of the SelectedField class.

There is some risk that the current approach could result in a regression 
e.g. if there is an existing GetArrayItem that is being successfully pruned, 
where a non-foldable index argument happens to not trigger an error (because it 
is not a struct field, or it is preserved due to some other expression).

### Why are the changes needed?

Allows queries that previously would fail in the optimizer to pass.

### Does this PR introduce _any_ user-facing change?

Yes, as described above, there could be a performance regression if a query 
was previously pruning through a GetArrayItem/GetMapItem, and happened to not 
fail.

### How was this patch tested?

Unit test included in patch, fails without the patch and passes with it.

Closes #39718 from cashmand/fix_selected_field.

Authored-by: cashmand 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 16cfa092d2c23a1cca45acd62e2095a72cb27d86)
Signed-off-by: Wenchen Fan 
---
 .../sql/catalyst/expressions/SelectedField.scala   |  7 +++--
 .../execution/datasources/SchemaPruningSuite.scala | 30 ++
 2 files changed, 35 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
index ee2b4969222..820dc452d7e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
@@ -96,9 +96,10 @@ object SelectedField {
 }
 val newField = StructField(field.name, newFieldDataType, 
field.nullable)
 selectField(child, Option(ArrayType(struct(newField), containsNull)))
-  case GetMapValue(child, _) =>
+  case GetMapValue(child, key) if key.foldable =>
 // GetMapValue does not select a field from a struct (i.e. prune the 
struct) so it can't be
 // the top-level extractor. However it can be part of an extractor 
chain.
+// See comment on GetArrayItem regarding the need for key.foldable
 val MapType(keyType, _, valueContainsNull) = child.dataType
 val opt = dataTypeOpt.map(dt => MapType(keyType, dt, 
valueContainsNull))
 selectField(child, opt)
@@ -124,9 +125,11 @@ object SelectedField {
 throw QueryCompilationErrors.dataTypeUnsupportedByClassError(x, 
"MapKeys")
 }
 selectField(child, opt)
-  case GetArrayItem(child, _, _) =>
+  case GetArrayItem(child, index, _) if index.foldable =>
 // GetArrayItem does not select a field from a struct (i.e. prune the 
struct) so it can't be
 // the top-level extractor. However it can be part of an extractor 
chain.
+// If index is not foldable, we'd need to also return the field 
selected by index, which
+// the SelectedField interface doesn't support, so only allow a 
foldable index for now.
 val ArrayType(_, containsNull) = child.dataType
 val opt = dataTypeOpt.map(dt => ArrayType(dt, containsNull))
 selectField(child, opt)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
 
b/sql/core/src/tes

[spark] branch master updated: [SPARK-42163][SQL] Fix schema pruning for non-foldable array index or map key

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 16cfa092d2c [SPARK-42163][SQL] Fix schema pruning for non-foldable 
array index or map key
16cfa092d2c is described below

commit 16cfa092d2c23a1cca45acd62e2095a72cb27d86
Author: cashmand 
AuthorDate: Tue Jan 31 10:16:15 2023 +0800

[SPARK-42163][SQL] Fix schema pruning for non-foldable array index or map 
key

### What changes were proposed in this pull request?

In parquet schema pruning, we use SelectedField to try to extract the field 
that is used in a struct. It looks through GetArrayItem/GetMapItem, but when 
doing so, it ignores the index/key, which may itself be a struct field. If it 
is a struct field that is not selected in some other expression, and another 
field of the same attribute is selected, then pruning will drop the field, 
resulting in an optimizer error.

This change modifies SelectedField to only look through 
GetArrayItem/GetMapItem if the index/key argument is foldable. The equivalent 
code for `ElementAt` was already doing the same thing, so this just makes them 
consistent.

In principle, we could continue to traverse through these expressions, we'd 
just need to make sure that the index/key expression was also surfaced to 
column pruning as an expression that needs to be examined. But this seems like 
a fairly non-trivial change to the design of the SelectedField class.

There is some risk that the current approach could result in a regression 
e.g. if there is an existing GetArrayItem that is being successfully pruned, 
where a non-foldable index argument happens to not trigger an error (because it 
is not a struct field, or it is preserved due to some other expression).

### Why are the changes needed?

Allows queries that previously would fail in the optimizer to pass.

### Does this PR introduce _any_ user-facing change?

Yes, as described above, there could be a performance regression if a query 
was previously pruning through a GetArrayItem/GetMapItem, and happened to not 
fail.

### How was this patch tested?

Unit test included in patch, fails without the patch and passes with it.

Closes #39718 from cashmand/fix_selected_field.

Authored-by: cashmand 
Signed-off-by: Wenchen Fan 
---
 .../sql/catalyst/expressions/SelectedField.scala   |  7 +++--
 .../execution/datasources/SchemaPruningSuite.scala | 30 ++
 2 files changed, 35 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
index ee2b4969222..820dc452d7e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala
@@ -96,9 +96,10 @@ object SelectedField {
 }
 val newField = StructField(field.name, newFieldDataType, 
field.nullable)
 selectField(child, Option(ArrayType(struct(newField), containsNull)))
-  case GetMapValue(child, _) =>
+  case GetMapValue(child, key) if key.foldable =>
 // GetMapValue does not select a field from a struct (i.e. prune the 
struct) so it can't be
 // the top-level extractor. However it can be part of an extractor 
chain.
+// See comment on GetArrayItem regarding the need for key.foldable
 val MapType(keyType, _, valueContainsNull) = child.dataType
 val opt = dataTypeOpt.map(dt => MapType(keyType, dt, 
valueContainsNull))
 selectField(child, opt)
@@ -124,9 +125,11 @@ object SelectedField {
 throw QueryCompilationErrors.dataTypeUnsupportedByClassError(x, 
"MapKeys")
 }
 selectField(child, opt)
-  case GetArrayItem(child, _, _) =>
+  case GetArrayItem(child, index, _) if index.foldable =>
 // GetArrayItem does not select a field from a struct (i.e. prune the 
struct) so it can't be
 // the top-level extractor. However it can be part of an extractor 
chain.
+// If index is not foldable, we'd need to also return the field 
selected by index, which
+// the SelectedField interface doesn't support, so only allow a 
foldable index for now.
 val ArrayType(_, containsNull) = child.dataType
 val opt = dataTypeOpt.map(dt => ArrayType(dt, containsNull))
 selectField(child, opt)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
index f9a8c67fc9f..bd9c79e5b96 100644
--

[spark] branch branch-3.4 updated: [SPARK-42066][SQL] The DATATYPE_MISMATCH error class contains inappropriate and duplicating subclasses

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 4abeafe0c36 [SPARK-42066][SQL] The DATATYPE_MISMATCH error class 
contains inappropriate and duplicating subclasses
4abeafe0c36 is described below

commit 4abeafe0c36842ee9c2cd44f6dc84ba1c0d3d033
Author: itholic 
AuthorDate: Mon Jan 30 18:55:07 2023 +0800

[SPARK-42066][SQL] The DATATYPE_MISMATCH error class contains inappropriate 
and duplicating subclasses

### What changes were proposed in this pull request?

This PR proposes to remove `DATATYPE_MISMATCH.WRONG_NUM_ARGS` and 
`DATATYPE_MISMATCH.WRONG_NUM_ARGS_WITH_SUGGESTION`from sub-class of 
`DATATYPE_MISMATCH`, and integrate into `WRONG_NUM_ARGS.WITH_SUGGESTION` and 
`WRONG_NUM_ARGS.WITHOUT_SUGGESTION`.

### Why are the changes needed?

Because `WRONG_NUM_ARGS` is logically not a part of `DATATYPE_MISMATCH` 
error, and duplicated.

We should use appropriate name for all error classes.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

The existing UT is fixed, so CI should pass.

Closes #39625 from itholic/SPARK-42066.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 8fefb6218812ae3395fde9c172c966a60dab78e6)
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   | 15 +---
 .../sql/catalyst/analysis/FunctionRegistry.scala   |  6 +-
 .../expressions/CallMethodViaReflection.scala  | 11 +--
 .../catalyst/expressions/aggregate/Average.scala   |  2 +-
 .../sql/catalyst/expressions/aggregate/Count.scala | 21 ++---
 .../sql/catalyst/expressions/aggregate/Sum.scala   |  2 +-
 .../sql/catalyst/expressions/arithmetic.scala  | 20 ++---
 .../catalyst/expressions/complexTypeCreator.scala  | 19 ++---
 .../catalyst/expressions/datetimeExpressions.scala | 16 ++--
 .../sql/catalyst/expressions/generators.scala  | 10 +--
 .../spark/sql/catalyst/expressions/hash.scala  | 10 +--
 .../sql/catalyst/expressions/jsonExpressions.scala |  9 +-
 .../sql/catalyst/expressions/mathExpressions.scala |  2 +-
 .../sql/catalyst/expressions/nullExpressions.scala | 12 +--
 .../catalyst/expressions/stringExpressions.scala   | 33 ++--
 .../sql/catalyst/expressions/urlExpressions.scala  | 12 +--
 .../spark/sql/errors/QueryCompilationErrors.scala  | 49 ++-
 .../analysis/ExpressionTypeCheckingSuite.scala | 18 ++--
 .../expressions/CallMethodViaReflectionSuite.scala | 35 
 .../catalyst/expressions/ComplexTypeSuite.scala| 35 
 .../expressions/GeneratorExpressionSuite.scala | 37 +
 .../expressions/StringExpressionsSuite.scala   | 79 ++
 .../org/apache/spark/sql/UDFRegistration.scala | 96 +++---
 .../sql/internal/BaseSessionStateBuilder.scala |  2 +-
 .../resources/sql-tests/results/ansi/date.sql.out  |  2 +-
 .../results/ansi/string-functions.sql.out  | 36 +++-
 .../results/ceil-floor-with-scale-param.sql.out|  4 +-
 .../test/resources/sql-tests/results/count.sql.out | 16 +---
 .../sql-tests/results/csv-functions.sql.out|  2 +-
 .../test/resources/sql-tests/results/date.sql.out  |  2 +-
 .../sql-tests/results/datetime-legacy.sql.out  |  2 +-
 .../sql-tests/results/json-functions.sql.out   |  8 +-
 .../results/sql-compatibility-functions.sql.out|  2 +-
 .../sql-tests/results/string-functions.sql.out | 36 +++-
 .../results/table-valued-functions.sql.out |  4 +-
 .../sql-tests/results/timestamp-ntz.sql.out|  2 +-
 .../resources/sql-tests/results/udaf/udaf.sql.out  |  2 +-
 .../sql-tests/results/udf/udf-udaf.sql.out |  2 +-
 .../apache/spark/sql/DataFrameFunctionsSuite.scala | 74 ++---
 .../org/apache/spark/sql/DateFunctionsSuite.scala  |  2 +-
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  2 +-
 .../apache/spark/sql/StringFunctionsSuite.scala|  2 +-
 .../test/scala/org/apache/spark/sql/UDFSuite.scala |  4 +-
 .../sql/errors/QueryCompilationErrorsSuite.scala   |  5 +-
 .../spark/sql/hive/execution/HiveUDAFSuite.scala   |  2 +-
 45 files changed, 324 insertions(+), 438 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index b9bef3b7b1d..8975fe279c2 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -433,17 +433,6 @@
   "The  must be between  (current value = 
)."
 ]
   },
-  "WRONG_NUM_ARGS" : {
-"message" : [
-  "The  requires  parameters but the actual 
number is ."
-]
-  },
-  "WRONG_NU

[spark] branch master updated (8b8a5a84453 -> 8fefb621881)

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 8b8a5a84453 [SPARK-41735][SQL] Use MINIMAL instead of STANDARD for 
SparkListenerSQLExecutionEnd
 add 8fefb621881 [SPARK-42066][SQL] The DATATYPE_MISMATCH error class 
contains inappropriate and duplicating subclasses

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/error-classes.json   | 15 +---
 .../sql/catalyst/analysis/FunctionRegistry.scala   |  6 +-
 .../expressions/CallMethodViaReflection.scala  | 11 +--
 .../catalyst/expressions/aggregate/Average.scala   |  2 +-
 .../sql/catalyst/expressions/aggregate/Count.scala | 21 ++---
 .../sql/catalyst/expressions/aggregate/Sum.scala   |  2 +-
 .../sql/catalyst/expressions/arithmetic.scala  | 20 ++---
 .../catalyst/expressions/complexTypeCreator.scala  | 19 ++---
 .../catalyst/expressions/datetimeExpressions.scala | 16 ++--
 .../sql/catalyst/expressions/generators.scala  | 10 +--
 .../spark/sql/catalyst/expressions/hash.scala  | 10 +--
 .../sql/catalyst/expressions/jsonExpressions.scala |  9 +-
 .../sql/catalyst/expressions/mathExpressions.scala |  2 +-
 .../sql/catalyst/expressions/nullExpressions.scala | 12 +--
 .../catalyst/expressions/stringExpressions.scala   | 33 ++--
 .../sql/catalyst/expressions/urlExpressions.scala  | 12 +--
 .../spark/sql/errors/QueryCompilationErrors.scala  | 49 ++-
 .../analysis/ExpressionTypeCheckingSuite.scala | 18 ++--
 .../expressions/CallMethodViaReflectionSuite.scala | 35 
 .../catalyst/expressions/ComplexTypeSuite.scala| 35 
 .../expressions/GeneratorExpressionSuite.scala | 37 +
 .../expressions/StringExpressionsSuite.scala   | 79 ++
 .../org/apache/spark/sql/UDFRegistration.scala | 96 +++---
 .../sql/internal/BaseSessionStateBuilder.scala |  2 +-
 .../resources/sql-tests/results/ansi/date.sql.out  |  2 +-
 .../results/ansi/string-functions.sql.out  | 36 +++-
 .../results/ceil-floor-with-scale-param.sql.out|  4 +-
 .../test/resources/sql-tests/results/count.sql.out | 16 +---
 .../sql-tests/results/csv-functions.sql.out|  2 +-
 .../test/resources/sql-tests/results/date.sql.out  |  2 +-
 .../sql-tests/results/datetime-legacy.sql.out  |  2 +-
 .../sql-tests/results/json-functions.sql.out   |  8 +-
 .../results/sql-compatibility-functions.sql.out|  2 +-
 .../sql-tests/results/string-functions.sql.out | 36 +++-
 .../results/table-valued-functions.sql.out |  4 +-
 .../sql-tests/results/timestamp-ntz.sql.out|  2 +-
 .../resources/sql-tests/results/udaf/udaf.sql.out  |  2 +-
 .../sql-tests/results/udf/udf-udaf.sql.out |  2 +-
 .../apache/spark/sql/DataFrameFunctionsSuite.scala | 74 ++---
 .../org/apache/spark/sql/DateFunctionsSuite.scala  |  2 +-
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  2 +-
 .../apache/spark/sql/StringFunctionsSuite.scala|  2 +-
 .../test/scala/org/apache/spark/sql/UDFSuite.scala |  4 +-
 .../sql/errors/QueryCompilationErrorsSuite.scala   |  5 +-
 .../spark/sql/hive/execution/HiveUDAFSuite.scala   |  2 +-
 45 files changed, 324 insertions(+), 438 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-41735][SQL] Use MINIMAL instead of STANDARD for SparkListenerSQLExecutionEnd

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 4573fdfdf61 [SPARK-41735][SQL] Use MINIMAL instead of STANDARD for 
SparkListenerSQLExecutionEnd
4573fdfdf61 is described below

commit 4573fdfdf615221de671aebe5ecafa179f5cd778
Author: ulysses-you 
AuthorDate: Mon Jan 30 16:01:29 2023 +0800

[SPARK-41735][SQL] Use MINIMAL instead of STANDARD for 
SparkListenerSQLExecutionEnd

### What changes were proposed in this pull request?

Use MINIMAL instead of STANDARD for SparkListenerSQLExecutionEnd. 
`SparkThrowable` is not an internal api, so we can not assume all exceptions 
are defined in Spark internal-json file.

### Why are the changes needed?

Improve developers error msg with `SparkThrowable`.

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

N/A

Closes #39794 from ulysses-you/SPARK-41735.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 8b8a5a8445346eda14b8f7b8acd52bf78c3cf983)
Signed-off-by: Wenchen Fan 
---
 .../src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
index 90468b18a99..56fc9d946df 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
@@ -124,7 +124,7 @@ object SQLExecution {
   val endTime = System.nanoTime()
   val errorMessage = ex.map {
 case e: SparkThrowable =>
-  SparkThrowableHelper.getMessage(e, ErrorMessageFormat.STANDARD)
+  SparkThrowableHelper.getMessage(e, ErrorMessageFormat.MINIMAL)
 case e =>
   // unexpected behavior
   SparkThrowableHelper.getMessage(e)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41735][SQL] Use MINIMAL instead of STANDARD for SparkListenerSQLExecutionEnd

2023-01-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 8b8a5a84453 [SPARK-41735][SQL] Use MINIMAL instead of STANDARD for 
SparkListenerSQLExecutionEnd
8b8a5a84453 is described below

commit 8b8a5a8445346eda14b8f7b8acd52bf78c3cf983
Author: ulysses-you 
AuthorDate: Mon Jan 30 16:01:29 2023 +0800

[SPARK-41735][SQL] Use MINIMAL instead of STANDARD for 
SparkListenerSQLExecutionEnd

### What changes were proposed in this pull request?

Use MINIMAL instead of STANDARD for SparkListenerSQLExecutionEnd. 
`SparkThrowable` is not an internal api, so we can not assume all exceptions 
are defined in Spark internal-json file.

### Why are the changes needed?

Improve developers error msg with `SparkThrowable`.

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

N/A

Closes #39794 from ulysses-you/SPARK-41735.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
index 90468b18a99..56fc9d946df 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
@@ -124,7 +124,7 @@ object SQLExecution {
   val endTime = System.nanoTime()
   val errorMessage = ex.map {
 case e: SparkThrowable =>
-  SparkThrowableHelper.getMessage(e, ErrorMessageFormat.STANDARD)
+  SparkThrowableHelper.getMessage(e, ErrorMessageFormat.MINIMAL)
 case e =>
   // unexpected behavior
   SparkThrowableHelper.getMessage(e)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.4 updated: [SPARK-38591][SQL][FOLLOW-UP] Fix ambiguous references for sorted cogroups

2023-01-29 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 0cb0fa31397 [SPARK-38591][SQL][FOLLOW-UP] Fix ambiguous references for 
sorted cogroups
0cb0fa31397 is described below

commit 0cb0fa313979e1b82ddd711a05d8c4e78cf6c9f5
Author: Enrico Minack 
AuthorDate: Mon Jan 30 10:06:07 2023 +0800

[SPARK-38591][SQL][FOLLOW-UP] Fix ambiguous references for sorted cogroups

Sort order for left and right cogroups must be resolved against left and 
right plan, respectively. Otherwise, ambiguous reference exception can be 
thrown.

```Scala
leftGroupedDf.cogroup(rightGroupedDf)($"time")($"time") { ... }
```

Grouped DataFrames `leftGroupedDf` and `rightGroupedDf` both contain column 
`"time"`. Left and right sort order `$"time"` is ambiguous when resolved 
against all children. They must be resolved against left or right child, 
exclusively.

This fixes errors like

[AMBIGUOUS_REFERENCE] Reference `time` is ambiguous, could be: [`time`, 
`time`].

Tested in `AnalysisSuite` on `Analyzer` level, and E2E in `DatasetSuite`.

Closes #39744 from EnricoMi/branch-sorted-groups-ambiguous-reference.

Authored-by: Enrico Minack 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 607e753f9cf390cce293cef22a682e8a2d63e86b)
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  31 +
 .../sql/catalyst/analysis/AnalysisSuite.scala  |  46 +++-
 .../scala/org/apache/spark/sql/DatasetSuite.scala  | 127 ++---
 3 files changed, 186 insertions(+), 18 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index dd1743f4554..48ea0460725 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -1593,6 +1593,37 @@ class Analyzer(override val catalogManager: 
CatalogManager)
   Generate(newG.asInstanceOf[Generator], join, outer, qualifier, 
output, child)
 }
 
+  case mg: MapGroups if mg.dataOrder.exists(!_.resolved) =>
+// Resolve against `AppendColumns`'s children, instead of 
`AppendColumns`,
+// because `AppendColumns`'s serializer might produce conflict 
attribute
+// names leading to ambiguous references exception.
+val planForResolve = mg.child match {
+  case appendColumns: AppendColumns => appendColumns.child
+  case plan => plan
+}
+val resolvedOrder = mg.dataOrder
+.map(resolveExpressionByPlanOutput(_, 
planForResolve).asInstanceOf[SortOrder])
+mg.copy(dataOrder = resolvedOrder)
+
+  // Left and right sort expression have to be resolved against the 
respective child plan only
+  case cg: CoGroup if cg.leftOrder.exists(!_.resolved) || 
cg.rightOrder.exists(!_.resolved) =>
+// Resolve against `AppendColumns`'s children, instead of 
`AppendColumns`,
+// because `AppendColumns`'s serializer might produce conflict 
attribute
+// names leading to ambiguous references exception.
+val (leftPlanForResolve, rightPlanForResolve) = Seq(cg.left, 
cg.right).map {
+  case appendColumns: AppendColumns => appendColumns.child
+  case plan => plan
+} match {
+  case Seq(left, right) => (left, right)
+}
+
+val resolvedLeftOrder = cg.leftOrder
+  .map(resolveExpressionByPlanOutput(_, 
leftPlanForResolve).asInstanceOf[SortOrder])
+val resolvedRightOrder = cg.rightOrder
+  .map(resolveExpressionByPlanOutput(_, 
rightPlanForResolve).asInstanceOf[SortOrder])
+
+cg.copy(leftOrder = resolvedLeftOrder, rightOrder = resolvedRightOrder)
+
   // Skips plan which contains deserializer expressions, as they should be 
resolved by another
   // rule: ResolveDeserializer.
   case plan if containsDeserializer(plan.expressions) => plan
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 6dfbf12bbd7..e6cd0699468 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -28,7 +28,7 @@ import org.scalatest.matchers.must.Matchers
 
 import org.apache.spark.api.python.PythonEvalType
 import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.{AliasId

[spark] branch master updated: [SPARK-38591][SQL][FOLLOW-UP] Fix ambiguous references for sorted cogroups

2023-01-29 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 607e753f9cf [SPARK-38591][SQL][FOLLOW-UP] Fix ambiguous references for 
sorted cogroups
607e753f9cf is described below

commit 607e753f9cf390cce293cef22a682e8a2d63e86b
Author: Enrico Minack 
AuthorDate: Mon Jan 30 10:06:07 2023 +0800

[SPARK-38591][SQL][FOLLOW-UP] Fix ambiguous references for sorted cogroups

### What changes were proposed in this pull request?
Sort order for left and right cogroups must be resolved against left and 
right plan, respectively. Otherwise, ambiguous reference exception can be 
thrown.

### Why are the changes needed?
#39640 added sorted groups for `flatMapGroups` and `cogroup`. Sort order 
for the `cogroup` can be ambiguous when resolved against all children of 
`CoGroup`:

```Scala
leftGroupedDf.cogroup(rightGroupedDf)($"time")($"time") { ... }
```

Grouped DataFrames `leftGroupedDf` and `rightGroupedDf` both contain column 
`"time"`. Left and right sort order `$"time"` is ambiguous when resolved 
against all children. They must be resolved against left or right child, 
exclusively.

### Does this PR introduce _any_ user-facing change?
This fixes errors like

[AMBIGUOUS_REFERENCE] Reference `time` is ambiguous, could be: [`time`, 
`time`].

### How was this patch tested?
Tested in `AnalysisSuite` on `Analyzer` level, and E2E in `DatasetSuite`.

Closes #39744 from EnricoMi/branch-sorted-groups-ambiguous-reference.

Authored-by: Enrico Minack 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  31 +
 .../sql/catalyst/analysis/AnalysisSuite.scala  |  46 +++-
 .../scala/org/apache/spark/sql/DatasetSuite.scala  | 127 ++---
 3 files changed, 186 insertions(+), 18 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 87342528927..154587eeff7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -1593,6 +1593,37 @@ class Analyzer(override val catalogManager: 
CatalogManager)
   Generate(newG.asInstanceOf[Generator], join, outer, qualifier, 
output, child)
 }
 
+  case mg: MapGroups if mg.dataOrder.exists(!_.resolved) =>
+// Resolve against `AppendColumns`'s children, instead of 
`AppendColumns`,
+// because `AppendColumns`'s serializer might produce conflict 
attribute
+// names leading to ambiguous references exception.
+val planForResolve = mg.child match {
+  case appendColumns: AppendColumns => appendColumns.child
+  case plan => plan
+}
+val resolvedOrder = mg.dataOrder
+.map(resolveExpressionByPlanOutput(_, 
planForResolve).asInstanceOf[SortOrder])
+mg.copy(dataOrder = resolvedOrder)
+
+  // Left and right sort expression have to be resolved against the 
respective child plan only
+  case cg: CoGroup if cg.leftOrder.exists(!_.resolved) || 
cg.rightOrder.exists(!_.resolved) =>
+// Resolve against `AppendColumns`'s children, instead of 
`AppendColumns`,
+// because `AppendColumns`'s serializer might produce conflict 
attribute
+// names leading to ambiguous references exception.
+val (leftPlanForResolve, rightPlanForResolve) = Seq(cg.left, 
cg.right).map {
+  case appendColumns: AppendColumns => appendColumns.child
+  case plan => plan
+} match {
+  case Seq(left, right) => (left, right)
+}
+
+val resolvedLeftOrder = cg.leftOrder
+  .map(resolveExpressionByPlanOutput(_, 
leftPlanForResolve).asInstanceOf[SortOrder])
+val resolvedRightOrder = cg.rightOrder
+  .map(resolveExpressionByPlanOutput(_, 
rightPlanForResolve).asInstanceOf[SortOrder])
+
+cg.copy(leftOrder = resolvedLeftOrder, rightOrder = resolvedRightOrder)
+
   // Skips plan which contains deserializer expressions, as they should be 
resolved by another
   // rule: ResolveDeserializer.
   case plan if containsDeserializer(plan.expressions) => plan
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 0a3023a7af8..5c2878be69f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sq

[spark] branch branch-3.4 updated: [SPARK-42081][SQL] Improve the plan change validation

2023-01-29 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 99f81ccbc43 [SPARK-42081][SQL] Improve the plan change validation
99f81ccbc43 is described below

commit 99f81ccbc43ef73c4b3f54ed98f1191407c4896d
Author: Wenchen Fan 
AuthorDate: Mon Jan 30 10:00:29 2023 +0800

[SPARK-42081][SQL] Improve the plan change validation

### What changes were proposed in this pull request?

Today, we validate the plan change after each rule execution, which is very 
useful to find issues in the rule. However, it has 2 problems:
1. it can only be used in the tests
2. it just returns a boolean, making it hard to known what's wrong with the 
rule

This PR addresses these 2 problems:
1. adds a new config to allow people to turn on it to debug production 
workloads
2. throws an exception immediately if a violation is found.

### Why are the changes needed?

To make it easier to debug queries using plan change validation.

### Does this PR introduce _any_ user-facing change?

No, the new config is internal.

### How was this patch tested?

updated tests

Closes #39592 from cloud-fan/validate.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
(cherry picked from commit 7b426acb5902960f70f1e9e3c336fd76fb8e3009)
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   | 20 +++---
 .../spark/sql/catalyst/analysis/Analyzer.scala | 10 +--
 .../spark/sql/catalyst/optimizer/Optimizer.scala   | 15 +
 .../sql/catalyst/plans/logical/LogicalPlan.scala   | 76 --
 .../spark/sql/catalyst/rules/RuleExecutor.scala| 49 +-
 .../spark/sql/errors/QueryExecutionErrors.scala| 17 -
 .../org/apache/spark/sql/internal/SQLConf.scala|  8 +++
 .../OptimizerStructuralIntegrityCheckerSuite.scala | 17 ++---
 .../plans/logical/LogicalPlanIntegritySuite.scala  | 18 ++---
 .../sql/catalyst/trees/RuleExecutorSuite.scala | 35 +-
 .../sql/execution/adaptive/AQEOptimizer.scala  | 12 ++--
 11 files changed, 155 insertions(+), 122 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 5d2e184874a..b9bef3b7b1d 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1123,6 +1123,16 @@
 ],
 "sqlState" : "42K09"
   },
+  "PLAN_VALIDATION_FAILED_RULE_EXECUTOR" : {
+"message" : [
+  "The input plan of  is invalid: "
+]
+  },
+  "PLAN_VALIDATION_FAILED_RULE_IN_BATCH" : {
+"message" : [
+  "Rule  in batch  generated an invalid plan: "
+]
+  },
   "PROTOBUF_DEPENDENCY_NOT_FOUND" : {
 "message" : [
   "Could not find dependency: ."
@@ -4357,16 +4367,6 @@
   "."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2173" : {
-"message" : [
-  "The structural integrity of the input plan is broken in ."
-]
-  },
-  "_LEGACY_ERROR_TEMP_2174" : {
-"message" : [
-  "After applying rule  in batch , the structural 
integrity of the plan is broken."
-]
-  },
   "_LEGACY_ERROR_TEMP_2175" : {
 "message" : [
   "Rule id not found for . Please modify RuleIdCollection.scala 
if you are adding a new rule."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index f0c22471afa..dd1743f4554 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -56,7 +56,6 @@ import org.apache.spark.sql.internal.connector.V1Function
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.types.DayTimeIntervalType.DAY
 import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils}
-import org.apache.spark.util.Utils
 import org.apache.spark.util.collection.{Utils => CUtils}
 
 /**
@@ -191,13 +190,10 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 
   private val v1SessionCatalog: SessionCatalog = 
catalogManager.v1SessionCatalog
 
-  override protected def isPlanIntegral(
+  override protected def validatePlanChanges(
   previousPlan: LogicalPlan,
-  currentPlan: LogicalPlan): Boolean = {
-import org.apache.spark.sql.catalyst.util._
-!Utils.isTesting || 
(LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) &&
-  (!LogicalPlanIntegrity.canGetOutputA

[spark] branch master updated: [SPARK-42081][SQL] Improve the plan change validation

2023-01-29 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7b426acb590 [SPARK-42081][SQL] Improve the plan change validation
7b426acb590 is described below

commit 7b426acb5902960f70f1e9e3c336fd76fb8e3009
Author: Wenchen Fan 
AuthorDate: Mon Jan 30 10:00:29 2023 +0800

[SPARK-42081][SQL] Improve the plan change validation

### What changes were proposed in this pull request?

Today, we validate the plan change after each rule execution, which is very 
useful to find issues in the rule. However, it has 2 problems:
1. it can only be used in the tests
2. it just returns a boolean, making it hard to known what's wrong with the 
rule

This PR addresses these 2 problems:
1. adds a new config to allow people to turn on it to debug production 
workloads
2. throws an exception immediately if a violation is found.

### Why are the changes needed?

To make it easier to debug queries using plan change validation.

### Does this PR introduce _any_ user-facing change?

No, the new config is internal.

### How was this patch tested?

updated tests

Closes #39592 from cloud-fan/validate.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   | 20 +++---
 .../spark/sql/catalyst/analysis/Analyzer.scala | 10 +--
 .../spark/sql/catalyst/optimizer/Optimizer.scala   | 15 +
 .../sql/catalyst/plans/logical/LogicalPlan.scala   | 76 --
 .../spark/sql/catalyst/rules/RuleExecutor.scala| 49 +-
 .../spark/sql/errors/QueryExecutionErrors.scala| 17 -
 .../org/apache/spark/sql/internal/SQLConf.scala|  8 +++
 .../OptimizerStructuralIntegrityCheckerSuite.scala | 17 ++---
 .../plans/logical/LogicalPlanIntegritySuite.scala  | 18 ++---
 .../sql/catalyst/trees/RuleExecutorSuite.scala | 35 +-
 .../sql/execution/adaptive/AQEOptimizer.scala  | 12 ++--
 11 files changed, 155 insertions(+), 122 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 936f996f3a4..62264a90afe 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1151,6 +1151,16 @@
 ],
 "sqlState" : "42K09"
   },
+  "PLAN_VALIDATION_FAILED_RULE_EXECUTOR" : {
+"message" : [
+  "The input plan of  is invalid: "
+]
+  },
+  "PLAN_VALIDATION_FAILED_RULE_IN_BATCH" : {
+"message" : [
+  "Rule  in batch  generated an invalid plan: "
+]
+  },
   "PROTOBUF_DEPENDENCY_NOT_FOUND" : {
 "message" : [
   "Could not find dependency: ."
@@ -4380,16 +4390,6 @@
   "."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2173" : {
-"message" : [
-  "The structural integrity of the input plan is broken in ."
-]
-  },
-  "_LEGACY_ERROR_TEMP_2174" : {
-"message" : [
-  "After applying rule  in batch , the structural 
integrity of the plan is broken."
-]
-  },
   "_LEGACY_ERROR_TEMP_2175" : {
 "message" : [
   "Rule id not found for . Please modify RuleIdCollection.scala 
if you are adding a new rule."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6f27c97ddf9..87342528927 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -56,7 +56,6 @@ import org.apache.spark.sql.internal.connector.V1Function
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.types.DayTimeIntervalType.DAY
 import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils}
-import org.apache.spark.util.Utils
 import org.apache.spark.util.collection.{Utils => CUtils}
 
 /**
@@ -191,13 +190,10 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 
   private val v1SessionCatalog: SessionCatalog = 
catalogManager.v1SessionCatalog
 
-  override protected def isPlanIntegral(
+  override protected def validatePlanChanges(
   previousPlan: LogicalPlan,
-  currentPlan: LogicalPlan): Boolean = {
-import org.apache.spark.sql.catalyst.util._
-!Utils.isTesting || 
(LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) &&
-  (!LogicalPlanIntegrity.canGetOutputAttrs(currentPlan) ||
-!currentPlan.output.exists(_.qualifiedAccessOnly)))
+  currentPlan: LogicalPlan)

[spark] branch master updated: [SPARK-41976][SQL] Improve error message for `INDEX_NOT_FOUND`

2023-01-19 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 9c5384595c8 [SPARK-41976][SQL] Improve error message for 
`INDEX_NOT_FOUND`
9c5384595c8 is described below

commit 9c5384595c8e19afba109b6d38a27856e50c1bc1
Author: itholic 
AuthorDate: Fri Jan 20 14:53:53 2023 +0800

[SPARK-41976][SQL] Improve error message for `INDEX_NOT_FOUND`

### What changes were proposed in this pull request?

This PR proposes to improve error message for `INDEX_NOT_FOUND`.

### Why are the changes needed?

Make the error message more clear and proper.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Fix UT and `./build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39498 from itholic/INDEX_NOT_FOUND.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 .../test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala  | 11 +++
 core/src/main/resources/error/error-classes.json  |  2 +-
 .../spark/sql/catalyst/analysis/NoSuchItemException.scala |  4 ++--
 .../src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala  |  6 +-
 .../main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala   |  6 +-
 .../scala/org/apache/spark/sql/jdbc/PostgresDialect.scala |  7 ++-
 .../test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala|  4 +---
 7 files changed, 27 insertions(+), 13 deletions(-)

diff --git 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
index 5bedcbd172e..f16d9b507d5 100644
--- 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
+++ 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
@@ -278,10 +278,13 @@ private[v2] trait V2JDBCTest extends SharedSparkSession 
with DockerIntegrationFu
 // This should pass without exception
 sql(s"DROP index IF EXISTS i1 ON $catalogName.new_table")
 
-m = intercept[NoSuchIndexException] {
-  sql(s"DROP index i1 ON $catalogName.new_table")
-}.getMessage
-assert(m.contains("Failed to drop index i1 in new_table"))
+checkError(
+  exception = intercept[NoSuchIndexException] {
+sql(s"DROP index i1 ON $catalogName.new_table")
+  },
+  errorClass = "INDEX_NOT_FOUND",
+  parameters = Map("indexName" -> "i1", "tableName" -> "new_table")
+)
   }
 }
   }
diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 993d1d084e7..5340ba2abc2 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -659,7 +659,7 @@
   },
   "INDEX_NOT_FOUND" : {
 "message" : [
-  "Cannot find the index. ."
+  "Cannot find the index  on table ."
 ],
 "sqlState" : "42704"
   },
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
index f6624126e94..8dd46c06e76 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
@@ -131,6 +131,6 @@ class NoSuchPartitionsException(errorClass: String, 
messageParameters: Map[Strin
 class NoSuchTempFunctionException(func: String)
   extends AnalysisException(errorClass = "ROUTINE_NOT_FOUND", 
Map("routineName" -> s"`$func`"))
 
-class NoSuchIndexException(message: String, cause: Option[Throwable] = None)
+class NoSuchIndexException(indexName: String, tableName: String, cause: 
Option[Throwable] = None)
   extends AnalysisException(errorClass = "INDEX_NOT_FOUND",
-Map("message" -> message), cause)
+Map("indexName" -> indexName, "tableName" -> tableName), cause)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
index eac3dab4f6b..5ede793f6d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
@@ -216,7 +216,11 @@ private[sql] object H2Dialect extends JdbcDialect {
  

[spark] branch master updated: [SPARK-38591][SQL] Add flatMapSortedGroups and cogroupSorted

2023-01-19 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6382a3be8f6 [SPARK-38591][SQL] Add flatMapSortedGroups and 
cogroupSorted
6382a3be8f6 is described below

commit 6382a3be8f6a937412c4e23a92ab440f7ba80bdf
Author: Enrico Minack 
AuthorDate: Fri Jan 20 13:21:29 2023 +0800

[SPARK-38591][SQL] Add flatMapSortedGroups and cogroupSorted

### What changes were proposed in this pull request?
This adds a sorted version of `Dataset.groupByKey(…).flatMapGroups(…)` and 
`Dataset.groupByKey(…).cogroup(…)`.

### Why are the changes needed?
The existing methods `KeyValueGroupedDataset.flatMapGroups` and 
`KeyValueGroupedDataset.cogroup` provide iterators of rows for each group key.

Sorting entire groups inside `flatMapGroups` / `cogroup` requires 
materialising all rows, which is against the idea of an iterator in the first 
place. Methods `flatMapGroups` and `cogroup` have the great advantage that they 
work with groups that are _too large to fit into memory of one executor_. 
Sorting them in the user function breaks this property.


[org.apache.spark.sql.KeyValueGroupedDataset](https://github.com/apache/spark/blob/47485a3c2df3201c838b939e82d5b26332e2d858/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala#L134-L137):
> Internally, the implementation will spill to disk if any given group is 
too large to fit into
> memory.  However, users must take care to avoid materializing the whole 
iterator for a group
> (for example, by calling `toList`) unless they are sure that this is 
possible given the memory
> constraints of their cluster.

The implementations of `KeyValueGroupedDataset.flatMapGroups` and 
`KeyValueGroupedDataset.cogroup` already sort each partition according to the 
group key. By additionally sorting by some data columns, the iterator can be 
guaranteed to provide some order.

### Does this PR introduce _any_ user-facing change?
This adds `KeyValueGroupedDataset.flatMapSortedGroups` and 
`KeyValueGroupedDataset.cogroupSorted`, which guarantees order of group 
iterators.

### How was this patch tested?
Tests have been added to `DatasetSuite` and `JavaDatasetSuite`.

Closes #39640 from EnricoMi/branch-sorted-groups-and-cogroups.

Authored-by: Enrico Minack 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/dsl/package.scala|   6 +-
 .../spark/sql/catalyst/plans/logical/object.scala  |  12 +-
 .../apache/spark/sql/KeyValueGroupedDataset.scala  | 148 -
 .../spark/sql/execution/SparkStrategies.scala  |  11 +-
 .../org/apache/spark/sql/execution/objects.scala   |  15 ++-
 .../streaming/FlatMapGroupsWithStateExec.scala |   6 +-
 .../org/apache/spark/sql/JavaDatasetSuite.java |  47 +--
 .../scala/org/apache/spark/sql/DatasetSuite.scala  |  68 ++
 8 files changed, 291 insertions(+), 22 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index 86d85abc6f3..ecd1ed94ffd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -425,7 +425,9 @@ package object dsl {
   leftGroup: Seq[Attribute],
   rightGroup: Seq[Attribute],
   leftAttr: Seq[Attribute],
-  rightAttr: Seq[Attribute]
+  rightAttr: Seq[Attribute],
+  leftOrder: Seq[SortOrder] = Nil,
+  rightOrder: Seq[SortOrder] = Nil
 ): LogicalPlan = {
 CoGroup.apply[Key, Left, Right, Result](
   func,
@@ -433,6 +435,8 @@ package object dsl {
   rightGroup,
   leftAttr,
   rightAttr,
+  leftOrder,
+  rightOrder,
   logicalPlan,
   otherPlan)
   }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
index e5fe07e2d95..b27c650cfb2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala
@@ -389,6 +389,7 @@ object MapGroups {
   func: (K, Iterator[T]) => TraversableOnce[U],
   groupingAttributes: Seq[Attribute],
   dataAttributes: Seq[Attribute],
+  dataOrder: Seq[SortOrder],
   child: LogicalPlan): LogicalPlan = {
 val mapped = new MapGroups(
   func.asInstanceOf[(Any, Iterator[Any]) => TraversableOnce[Any]],
@@ -396,6 +397,7 @@ object MapGroups {
   UnresolvedDeserializer(encoderF

[spark] branch master updated: [SPARK-40599][SQL] Relax multiTransform rule type to allow alternatives to be any kinds of Seq

2023-01-19 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c953610deaf [SPARK-40599][SQL] Relax multiTransform rule type to allow 
alternatives to be any kinds of Seq
c953610deaf is described below

commit c953610deafda769feb85fbb936591ffc4448f8e
Author: Peter Toth 
AuthorDate: Thu Jan 19 23:54:06 2023 +0800

[SPARK-40599][SQL] Relax multiTransform rule type to allow alternatives to 
be any kinds of Seq

### What changes were proposed in this pull request?
This is a follow-up PR to https://github.com/apache/spark/pull/38034. It 
relaxes `multiTransformDown()`'s `rule` parameter type to accept any kinds of 
`Seq` and make `MultiTransform.generateCartesianProduct()` helper public.

### Why are the changes needed?
API mprovement.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Existing UTs.

Closes #39652 from peter-toth/SPARK-40599-multitransform-follow-up.

Authored-by: Peter Toth 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/trees/TreeNode.scala | 70 +-
 .../spark/sql/catalyst/trees/TreeNodeSuite.scala   | 31 +-
 2 files changed, 57 insertions(+), 44 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index dc64e5e2560..c8df2086a72 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -626,7 +626,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
* @return the stream of alternatives
*/
   def multiTransformDown(
-  rule: PartialFunction[BaseType, Stream[BaseType]]): Stream[BaseType] = {
+  rule: PartialFunction[BaseType, Seq[BaseType]]): Stream[BaseType] = {
 multiTransformDownWithPruning(AlwaysProcess.fn, UnknownRuleId)(rule)
   }
 
@@ -639,10 +639,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
* as a lazy `Stream` to be able to limit the number of alternatives 
generated at the caller side
* as needed.
*
-   * The rule should not apply or can return a one element stream of original 
node to indicate that
+   * The purpose of this function to access the returned alternatives by the 
rule only if they are
+   * needed so the rule can return a `Stream` whose elements are also lazily 
calculated.
+   * E.g. `multiTransform*` calls can be nested with the help of
+   * `MultiTransform.generateCartesianProduct()`.
+   *
+   * The rule should not apply or can return a one element `Seq` of original 
node to indicate that
* the original node without any transformation is a valid alternative.
*
-   * The rule can return `Stream.empty` to indicate that the original node 
should be pruned. In this
+   * The rule can return `Seq.empty` to indicate that the original node should 
be pruned. In this
* case `multiTransform()` returns an empty `Stream`.
*
* Please consider the following examples of 
`input.multiTransformDown(rule)`:
@@ -652,9 +657,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
*
* 1.
* We have a simple rule:
-   *   `a` => `Stream(1, 2)`
-   *   `b` => `Stream(10, 20)`
-   *   `Add(a, b)` => `Stream(11, 12, 21, 22)`
+   *   `a` => `Seq(1, 2)`
+   *   `b` => `Seq(10, 20)`
+   *   `Add(a, b)` => `Seq(11, 12, 21, 22)`
*
* The output is:
*   `Stream(11, 12, 21, 22)`
@@ -662,9 +667,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
* 2.
* In the previous example if we want to generate alternatives of `a` and 
`b` too then we need to
* explicitly add the original `Add(a, b)` expression to the rule:
-   *   `a` => `Stream(1, 2)`
-   *   `b` => `Stream(10, 20)`
-   *   `Add(a, b)` => `Stream(11, 12, 21, 22, Add(a, b))`
+   *   `a` => `Seq(1, 2)`
+   *   `b` => `Seq(10, 20)`
+   *   `Add(a, b)` => `Seq(11, 12, 21, 22, Add(a, b))`
*
* The output is:
*   `Stream(11, 12, 21, 22, Add(1, 10), Add(2, 10), Add(1, 20), Add(2, 20))`
@@ -683,25 +688,25 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
   def multiTransformDownWithPruning(
   cond: TreePatternBits => Boolean,
   ruleId: RuleId = UnknownRuleId
-)(rule: PartialFunction[BaseType, Stream[BaseType]]): Stream[BaseType] = {
+)(rule: PartialFunction[BaseType, Seq[BaseType]]): Stream[BaseType] = {
 if (!cond.apply(this) || isRuleIneffective(ruleId)) {
   return Stream(this)
 }
 
-// We c

[spark] branch master updated: [SPARK-42108][SQL] Make Analyzer transform `Count(*)` into `Count(1)`

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 53c99a4270e [SPARK-42108][SQL] Make Analyzer transform `Count(*)` into 
`Count(1)`
53c99a4270e is described below

commit 53c99a4270eeaa431e18cfd7f053d8e0ae98734d
Author: Ruifeng Zheng 
AuthorDate: Thu Jan 19 10:25:07 2023 +0800

[SPARK-42108][SQL] Make Analyzer transform `Count(*)` into `Count(1)`

### What changes were proposed in this pull request?
Make Analyzer transform `Count(*)` into `Count(1)`

### Why are the changes needed?
Existing `Count(*) -> Count(1)` transformation happens in 
[`AstBuilder.visitFunctionCall`](https://github.com/apache/spark/blob/97a6955278c55fa02cb9f039ae45e49e6f0f2bfd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala#L2105-L2132).

The Analyzer requires the `Count(*)` had already been converted to 
`Count(1)` in Parser, and for a given `Count(*)` expression, the Analyzer 
itself can not correctly handle it and cause correctness issue in Spark Connect 
(see https://issues.apache.org/jira/browse/SPARK-41845)

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
added UT, manually test with Spark Connect

Closes #39636 from zhengruifeng/sql_move_count_star.

Authored-by: Ruifeng Zheng 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala  |  5 +
 .../spark/sql/catalyst/analysis/AnalysisSuite.scala| 18 ++
 2 files changed, 23 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ba2c2759e2d..f0c22471afa 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -1953,6 +1953,11 @@ class Analyzer(override val catalogManager: 
CatalogManager)
  */
 def expandStarExpression(expr: Expression, child: LogicalPlan): Expression 
= {
   expr.transformUp {
+case f0: UnresolvedFunction if !f0.isDistinct &&
+  f0.nameParts.map(_.toLowerCase(Locale.ROOT)) == Seq("count") &&
+  f0.arguments == Seq(UnresolvedStar(None)) =>
+  // Transform COUNT(*) into COUNT(1).
+  f0.copy(nameParts = Seq("count"), arguments = Seq(Literal(1)))
 case f1: UnresolvedFunction if containsStar(f1.arguments) =>
   // SPECIAL CASE: We want to block count(tblName.*) because in spark, 
count(tblName.*) will
   // be expanded while count(*) will be converted to count(1). They 
will produce different
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index a91a0a44dd3..6dfbf12bbd7 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -120,6 +120,24 @@ class AnalysisSuite extends AnalysisTest with Matchers {
   caseSensitive = false)
   }
 
+  test("SPARK-42108: transform count(*) to count(1)") {
+val a = testRelation.output(0)
+
+checkAnalysis(
+  Project(
+Alias(UnresolvedFunction("count" :: Nil,
+  UnresolvedStar(None) :: Nil, isDistinct = false), "x")() :: Nil, 
testRelation),
+  Aggregate(Nil, count(Literal(1)).as("x") :: Nil, testRelation))
+
+checkAnalysis(
+  Project(
+Alias(UnresolvedFunction("count" :: Nil,
+  UnresolvedStar(None) :: Nil, isDistinct = false), "x")() ::
+  Alias(UnresolvedFunction("count" :: Nil,
+UnresolvedAttribute("a") :: Nil, isDistinct = false), "y")() :: 
Nil, testRelation),
+  Aggregate(Nil, count(Literal(1)).as("x") :: count(a).as("y") :: Nil, 
testRelation))
+  }
+
   test("resolve sort references - filter/limit") {
 val a = testRelation2.output(0)
 val b = testRelation2.output(1)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41970] Introduce SparkPath for typesafety

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new faedcd91d55 [SPARK-41970] Introduce SparkPath for typesafety
faedcd91d55 is described below

commit faedcd91d554a00fc76116a0c188752cf036f907
Author: David Lewis 
AuthorDate: Thu Jan 19 10:05:51 2023 +0800

[SPARK-41970] Introduce SparkPath for typesafety

### What changes were proposed in this pull request?
This PR proposes a strongly typed `SparkPath` that encapsulates a 
url-encoded string. It has helper methods for creating hadoop paths, uris, and 
uri-encoded strings.
The intent is to identify and fix various bugs in the way that Spark 
handles these paths. To do this we introduced the SparkPath type to 
`PartitionFile` (a widely used class), and then started fixing compile errors. 
In doing so we fixed various bugs.

### Why are the changes needed?

Given `val str = "s3://bucket/path with space/a"` There is a difference 
between `new Path(str)` and `new Path(new URI(str))`, and thus a difference 
between `new URI(str)` and `new Path(str).toUri`.
Both `URI` and `Path` are symmetric in construction and `toString`, but are 
not interchangeable. Spark confuses these two paths (uri-encoded vs not). This 
PR attempts to use types to disambiguate them.

### Does this PR introduce _any_ user-facing change?

This PR proposes changing the public API of `PartitionedFile`, and various 
other methods in the name of type safety. It needs to be clear to callers of an 
API what type of path string is expected.

### How was this patch tested?

We rely on existing tests, and update the default temp path creation to 
include paths with spaces.

Closes #39488 from databricks-david-lewis/SPARK_PATH.

Authored-by: David Lewis 
Signed-off-by: Wenchen Fan 
---
 .../org/apache/spark/sql/avro/AvroFileFormat.scala |  5 +-
 .../sql/v2/avro/AvroPartitionReaderFactory.scala   |  7 +--
 .../apache/spark/sql/avro/AvroRowReaderSuite.scala |  6 +--
 .../org/apache/spark/sql/avro/AvroSuite.scala  |  3 +-
 .../apache/spark/deploy/worker/WorkerWatcher.scala |  2 +-
 .../scala/org/apache/spark/paths/SparkPath.scala   | 55 ++
 .../scala/org/apache/spark/rpc/RpcAddress.scala|  2 +-
 .../spark/ml/source/image/ImageFileFormat.scala|  6 +--
 scalastyle-config.xml  |  8 
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  7 +--
 .../spark/sql/execution/DataSourceScanExec.scala   |  4 +-
 .../apache/spark/sql/execution/FileRelation.scala  |  4 +-
 .../spark/sql/execution/PartitionedFileUtil.scala  |  5 +-
 .../execution/datasources/CatalogFileIndex.scala   |  3 +-
 .../sql/execution/datasources/DataSource.scala |  3 +-
 .../sql/execution/datasources/FileIndex.scala  |  3 +-
 .../sql/execution/datasources/FileScanRDD.scala| 21 ++---
 .../datasources/HadoopFileLinesReader.scala|  4 +-
 .../datasources/HadoopFileWholeTextReader.scala|  4 +-
 .../execution/datasources/HadoopFsRelation.scala   |  3 +-
 .../datasources/PartitioningAwareFileIndex.scala   |  5 +-
 .../datasources/binaryfile/BinaryFileFormat.scala  |  3 +-
 .../execution/datasources/csv/CSVDataSource.scala  |  3 +-
 .../execution/datasources/csv/CSVFileFormat.scala  |  2 +-
 .../datasources/json/JsonDataSource.scala  |  5 +-
 .../execution/datasources/orc/OrcFileFormat.scala  |  3 +-
 .../datasources/parquet/ParquetFileFormat.scala|  4 +-
 .../datasources/v2/FilePartitionReader.scala   |  8 ++--
 .../sql/execution/datasources/v2/FileScan.scala|  2 +-
 .../v2/csv/CSVPartitionReaderFactory.scala |  2 +-
 .../v2/orc/OrcPartitionReaderFactory.scala | 10 ++--
 .../v2/parquet/ParquetPartitionReaderFactory.scala | 11 ++---
 .../execution/streaming/FileStreamSinkLog.scala| 17 ---
 .../sql/execution/streaming/FileStreamSource.scala | 35 --
 .../spark/sql/FileBasedDataSourceSuite.scala   |  3 +-
 .../scala/org/apache/spark/sql/SubquerySuite.scala |  2 +-
 .../datasources/FileSourceStrategySuite.scala  |  9 ++--
 .../datasources/HadoopFileLinesReaderSuite.scala   |  7 ++-
 .../binaryfile/BinaryFileFormatSuite.scala |  4 +-
 .../spark/sql/streaming/FileStreamSinkSuite.scala  |  9 ++--
 .../sql/streaming/FileStreamSourceSuite.scala  | 45 +-
 .../apache/spark/sql/hive/orc/OrcFileFormat.scala  |  5 +-
 42 files changed, 216 insertions(+), 133 deletions(-)

diff --git 
a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala 
b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
index a13e0624f35..3e16e121081 100755
--- 
a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
+++ 
b/connector/avro/src/

[spark] branch branch-3.3 updated: [SPARK-42084][SQL] Avoid leaking the qualified-access-only restriction

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 9a8b652e872 [SPARK-42084][SQL] Avoid leaking the qualified-access-only 
restriction
9a8b652e872 is described below

commit 9a8b652e872d97334e01c2e16408be1277b54780
Author: Wenchen Fan 
AuthorDate: Wed Jan 18 18:43:55 2023 +0800

[SPARK-42084][SQL] Avoid leaking the qualified-access-only restriction

This is a better fix than https://github.com/apache/spark/pull/39077 and 
https://github.com/apache/spark/pull/38862

The special attribute metadata `__qualified_access_only` is very risky, as 
it breaks normal column resolution. The aforementioned 2 PRs remove the 
restriction in `SubqueryAlias` and `Alias`, but it's not good enough as we may 
forget to do the same thing for new logical plans/expressions in the future. 
It's also problematic if advanced users manipulate logical plans and 
expressions directly, when there is no `SubqueryAlias` and `Alias` to remove 
the restriction.

To be safe, we should only apply this restriction when resolving join 
hidden columns, which means the plan node right above `Project(Join(using or 
natural join))`. This PR simply removes the restriction when a column is 
resolved from a sequence of `Attributes`, or from star expansion, and also when 
adding the `Project` hidden columns to its output. This makes sure that the 
qualified-access-only restriction will not be leaked to normal column 
resolution, but only metadata column resolution.

To make the join hidden column feature more robust

No

existing tests

Closes #39596 from cloud-fan/join.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala  |  9 ++---
 .../spark/sql/catalyst/analysis/unresolved.scala   |  6 ++
 .../sql/catalyst/expressions/namedExpressions.scala|  7 ++-
 .../spark/sql/catalyst/expressions/package.scala   |  7 ++-
 .../spark/sql/catalyst/plans/logical/LogicalPlan.scala |  2 +-
 .../catalyst/plans/logical/basicLogicalOperators.scala | 10 +-
 .../org/apache/spark/sql/catalyst/util/package.scala   | 18 --
 .../resources/sql-tests/results/natural-join.sql.out   |  2 +-
 .../resources/sql-tests/results/using-join.sql.out |  2 +-
 9 files changed, 36 insertions(+), 27 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e55d504653f..84aa06baaff 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -193,7 +193,10 @@ class Analyzer(override val catalogManager: CatalogManager)
   override protected def isPlanIntegral(
   previousPlan: LogicalPlan,
   currentPlan: LogicalPlan): Boolean = {
-!Utils.isTesting || 
LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan)
+import org.apache.spark.sql.catalyst.util._
+!Utils.isTesting || 
(LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) &&
+  (!LogicalPlanIntegrity.canGetOutputAttrs(currentPlan) ||
+!currentPlan.output.exists(_.qualifiedAccessOnly)))
   }
 
   override def isView(nameParts: Seq[String]): Boolean = 
v1SessionCatalog.isView(nameParts)
@@ -919,7 +922,6 @@ class Analyzer(override val catalogManager: CatalogManager)
* projecting away metadata columns prematurely.
*/
   object AddMetadataColumns extends Rule[LogicalPlan] {
-
 import org.apache.spark.sql.catalyst.util._
 
 def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsDownWithPruning(
@@ -970,7 +972,8 @@ class Analyzer(override val catalogManager: CatalogManager)
   case s: ExposesMetadataColumns => s.withMetadataColumns()
   case p: Project =>
 val newProj = p.copy(
-  projectList = p.projectList ++ p.metadataOutput,
+  // Do not leak the qualified-access-only restriction to normal plan 
outputs.
+  projectList = p.projectList ++ 
p.metadataOutput.map(_.markAsAllowAnyAccess()),
   child = addMetadataCol(p.child))
 newProj.copyTagsFrom(p)
 newProj
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index 677e9844cac..04b66ed3275 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
@@ -364,6 +364,12 @@ case class UnresolvedStar(target: Option[Seq[String]])

[spark] branch master updated: [SPARK-42084][SQL] Avoid leaking the qualified-access-only restriction

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 8fde8bd68ae [SPARK-42084][SQL] Avoid leaking the qualified-access-only 
restriction
8fde8bd68ae is described below

commit 8fde8bd68ae51757be29f4b586556eb25b3aa2b7
Author: Wenchen Fan 
AuthorDate: Wed Jan 18 18:43:55 2023 +0800

[SPARK-42084][SQL] Avoid leaking the qualified-access-only restriction

### What changes were proposed in this pull request?

This is a better fix than https://github.com/apache/spark/pull/39077 and 
https://github.com/apache/spark/pull/38862

The special attribute metadata `__qualified_access_only` is very risky, as 
it breaks normal column resolution. The aforementioned 2 PRs remove the 
restriction in `SubqueryAlias` and `Alias`, but it's not good enough as we may 
forget to do the same thing for new logical plans/expressions in the future. 
It's also problematic if advanced users manipulate logical plans and 
expressions directly, when there is no `SubqueryAlias` and `Alias` to remove 
the restriction.

To be safe, we should only apply this restriction when resolving join 
hidden columns, which means the plan node right above `Project(Join(using or 
natural join))`. This PR simply removes the restriction when a column is 
resolved from a sequence of `Attributes`, or from star expansion, and also when 
adding the `Project` hidden columns to its output. This makes sure that the 
qualified-access-only restriction will not be leaked to normal column 
resolution, but only metadata column resolution.

### Why are the changes needed?

To make the join hidden column feature more robust

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

existing tests

Closes #39596 from cloud-fan/join.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala  |  9 ++---
 .../spark/sql/catalyst/analysis/unresolved.scala   |  6 ++
 .../sql/catalyst/expressions/namedExpressions.scala|  7 ++-
 .../spark/sql/catalyst/expressions/package.scala   |  7 ++-
 .../spark/sql/catalyst/plans/logical/LogicalPlan.scala |  2 +-
 .../catalyst/plans/logical/basicLogicalOperators.scala | 10 +-
 .../org/apache/spark/sql/catalyst/util/package.scala   | 18 --
 7 files changed, 34 insertions(+), 25 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index d6b68a45e77..ba2c2759e2d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -194,7 +194,10 @@ class Analyzer(override val catalogManager: CatalogManager)
   override protected def isPlanIntegral(
   previousPlan: LogicalPlan,
   currentPlan: LogicalPlan): Boolean = {
-!Utils.isTesting || 
LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan)
+import org.apache.spark.sql.catalyst.util._
+!Utils.isTesting || 
(LogicalPlanIntegrity.checkIfExprIdsAreGloballyUnique(currentPlan) &&
+  (!LogicalPlanIntegrity.canGetOutputAttrs(currentPlan) ||
+!currentPlan.output.exists(_.qualifiedAccessOnly)))
   }
 
   override def isView(nameParts: Seq[String]): Boolean = 
v1SessionCatalog.isView(nameParts)
@@ -984,7 +987,6 @@ class Analyzer(override val catalogManager: CatalogManager)
* projecting away metadata columns prematurely.
*/
   object AddMetadataColumns extends Rule[LogicalPlan] {
-
 import org.apache.spark.sql.catalyst.util._
 
 def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsDownWithPruning(
@@ -1039,7 +1041,8 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 s.withMetadataColumns()
   case p: Project if p.metadataOutput.exists(a => 
requiredAttrIds.contains(a.exprId)) =>
 val newProj = p.copy(
-  projectList = p.projectList ++ p.metadataOutput,
+  // Do not leak the qualified-access-only restriction to normal plan 
outputs.
+  projectList = p.projectList ++ 
p.metadataOutput.map(_.markAsAllowAnyAccess()),
   child = addMetadataCol(p.child, requiredAttrIds))
 newProj.copyTagsFrom(p)
 newProj
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index 4a4028dc4c4..5e20f12747b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ 
b/sql/catalyst/src/main/scala/org/ap

[spark] branch master updated: [SPARK-41983][SQL] Rename & improve error message for `NULL_COMPARISON_RESULT`

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new f3a82b32a00 [SPARK-41983][SQL] Rename & improve error message for 
`NULL_COMPARISON_RESULT`
f3a82b32a00 is described below

commit f3a82b32a00fea5f0068306764b887f729d18d8c
Author: itholic 
AuthorDate: Wed Jan 18 17:24:40 2023 +0800

[SPARK-41983][SQL] Rename & improve error message for 
`NULL_COMPARISON_RESULT`

### What changes were proposed in this pull request?

This PR proposes to rename & improve error message for 
`NULL_COMPARISON_RESULT`

### Why are the changes needed?

We should have proper name and error message for error classes.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"`

Closes #39506 from itholic/NULL_COMPARISON_RESULT.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json  | 11 +--
 .../spark/sql/catalyst/expressions/higherOrderFunctions.scala |  2 +-
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala|  6 +++---
 .../sql/catalyst/expressions/HigherOrderFunctionsSuite.scala  |  7 +--
 4 files changed, 14 insertions(+), 12 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index e3904c1fe5a..78b1bf0743d 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -134,6 +134,11 @@
 ],
 "sqlState" : "42703"
   },
+  "COMPARATOR_RETURNS_NULL" : {
+"message" : [
+  "The comparator has returned a NULL for a comparison between 
 and . It should return a positive integer for 
\"greater than\", 0 for \"equal\" and a negative integer for \"less than\". To 
revert to deprecated behavior where NULL is treated as 0 (equal), you must set 
\"spark.sql.legacy.allowNullComparisonResultInArraySort\" to \"true\"."
+]
+  },
   "CONCURRENT_QUERY" : {
 "message" : [
   "Another instance of this query was just started by a concurrent 
session."
@@ -1043,12 +1048,6 @@
 ],
 "sqlState" : "42000"
   },
-  "NULL_COMPARISON_RESULT" : {
-"message" : [
-  "The comparison result is null. If you want to handle null as 0 (equal), 
you can set \"spark.sql.legacy.allowNullComparisonResultInArraySort\" to 
\"true\"."
-],
-"sqlState" : "560A9"
-  },
   "NULL_MAP_KEY" : {
 "message" : [
   "Cannot use null as map key."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
index da57538a8b4..c2db38bae45 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
@@ -452,7 +452,7 @@ case class ArraySort(
   secondElemVar.value.set(o2)
   val cmp = f.eval(inputRow)
   if (!allowNullComparisonResult && cmp == null) {
-throw QueryExecutionErrors.nullComparisonResultError()
+throw QueryExecutionErrors.comparatorReturnsNull(o1.toString, 
o1.toString)
   }
   cmp.asInstanceOf[Int]
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index b89c624870e..8128c460602 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -2756,10 +2756,10 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
   summary = getSummary(context))
   }
 
-  def nullComparisonResultError(): Throwable = {
+  def comparatorReturnsNull(firstValue: String, secondValue: String): 
Throwable = {
 new SparkException(
-  errorClass = "NULL_COMPARISON_RESULT",
-  messageParameters = Map.empty,
+  errorClass = "COMPARATOR_RETURNS_NULL",
+  messageParameters = Map("firstValue" -> firstValue, "secondValue" -> 
secondValue),
   cause = null)
   }
 
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala
 
b

[spark] branch master updated: [SPARK-41960][SQL] Assign name to _LEGACY_ERROR_TEMP_1056

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 912caf4264c [SPARK-41960][SQL] Assign name to _LEGACY_ERROR_TEMP_1056
912caf4264c is described below

commit 912caf4264cccedfb96dfd0a5d37fb635973ee83
Author: itholic 
AuthorDate: Wed Jan 18 17:23:12 2023 +0800

[SPARK-41960][SQL] Assign name to _LEGACY_ERROR_TEMP_1056

### What changes were proposed in this pull request?

This PR proposes to assign name to _LEGACY_ERROR_TEMP_1056, 
"WRONG_COMMAND_FOR_OBJECT_TYPE".

### Why are the changes needed?

We should assign proper name to _LEGACY_ERROR_TEMP_*

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39480 from itholic/LEGACY_1056.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   | 10 
 .../spark/sql/errors/QueryCompilationErrors.scala  | 20 
 .../apache/spark/sql/execution/command/ddl.scala   | 16 +++--
 .../spark/sql/execution/command/DDLSuite.scala | 12 --
 .../spark/sql/hive/execution/HiveDDLSuite.scala| 28 +-
 5 files changed, 67 insertions(+), 19 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 3f0abe77d0b..e3904c1fe5a 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1732,6 +1732,11 @@
   "`writeStream` can be called only on streaming Dataset/DataFrame."
 ]
   },
+  "WRONG_COMMAND_FOR_OBJECT_TYPE" : {
+"message" : [
+  "The operation  requires a . But  
is a . Use  instead."
+]
+  },
   "WRONG_NUM_ARGS" : {
 "message" : [
   "Invalid number of arguments for the function ."
@@ -2257,11 +2262,6 @@
   "The database name is not valid: ."
 ]
   },
-  "_LEGACY_ERROR_TEMP_1056" : {
-"message" : [
-  "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead."
-]
-  },
   "_LEGACY_ERROR_TEMP_1057" : {
 "message" : [
   "SHOW COLUMNS with conflicting databases: '' != ''."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 91412b760bd..5fe141e7286 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -777,10 +777,22 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase {
   messageParameters = Map("database" -> quoted))
   }
 
-  def cannotDropViewWithDropTableError(): Throwable = {
-new AnalysisException(
-  errorClass = "_LEGACY_ERROR_TEMP_1056",
-  messageParameters = Map.empty)
+  def wrongCommandForObjectTypeError(
+  operation: String,
+  requiredType: String,
+  objectName: String,
+  foundType: String,
+  alternative: String): Throwable = {
+new AnalysisException(
+  errorClass = "WRONG_COMMAND_FOR_OBJECT_TYPE",
+  messageParameters = Map(
+"operation" -> operation,
+"requiredType" -> requiredType,
+"objectName" -> objectName,
+"foundType" -> foundType,
+"alternative" -> alternative
+  )
+)
   }
 
   def showColumnsWithConflictDatabasesError(
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
index 1f71a104707..fdd4f10c793 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
@@ -226,9 +226,21 @@ case class DropTableCommand(
   // issue an exception.
   catalog.getTableMetadata(tableName).tableType match {
 case CatalogTableType.VIEW if !isView =>
-  throw QueryCompilationErrors.cannotDropViewWithDropTableError()
+  throw QueryCompilationErrors.wrongCommandForObjectTypeError(
+operation = "DROP TABLE",
+requiredType = s"${CatalogTableType.EXTERNAL.name} or 
${CatalogTableType.MANAGED.name}",
+objectName = catalog.getTableMetadata(tableName).qualifiedName,
+foundType = catalog.getTableMetadata(tableNa

[spark] branch master updated: [SPARK-41572][SQL] Assign name to _LEGACY_ERROR_TEMP_2149

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6ed03dceafd [SPARK-41572][SQL] Assign name to _LEGACY_ERROR_TEMP_2149
6ed03dceafd is described below

commit 6ed03dceafd5b677b083ae87c44d2c2d2f8c94d3
Author: itholic 
AuthorDate: Wed Jan 18 17:21:57 2023 +0800

[SPARK-41572][SQL] Assign name to _LEGACY_ERROR_TEMP_2149

### What changes were proposed in this pull request?

This PR proposes to assign name to _LEGACY_ERROR_TEMP_2149, 
"MALFORMED_CSV_RECORD".

### Why are the changes needed?

We should assign proper name to _LEGACY_ERROR_TEMP_*

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Fix UT & run `./build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39258 from itholic/LEGACY_2149.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   | 10 ++---
 .../spark/sql/catalyst/csv/UnivocityParser.scala   |  8 ++--
 .../spark/sql/errors/QueryExecutionErrors.scala|  6 +--
 .../sql/execution/datasources/csv/CSVSuite.scala   | 48 --
 4 files changed, 57 insertions(+), 15 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 465aa162981..3f0abe77d0b 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -941,6 +941,11 @@
 ],
 "sqlState" : "42710"
   },
+  "MALFORMED_CSV_RECORD" : {
+"message" : [
+  "Malformed CSV record: "
+]
+  },
   "MALFORMED_PROTOBUF_MESSAGE" : {
 "message" : [
   "Malformed Protobuf messages are detected in message deserialization. 
Parse Mode: . To process malformed protobuf message as null 
result, try setting the option 'mode' as 'PERMISSIVE'."
@@ -4226,11 +4231,6 @@
   "null value found but field  is not nullable."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2149" : {
-"message" : [
-  "Malformed CSV record"
-]
-  },
   "_LEGACY_ERROR_TEMP_2150" : {
 "message" : [
   "Due to Scala's limited support of tuple, tuple with more than 22 
elements are not supported."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
index 8464e394ab5..42e03630b14 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
@@ -319,15 +319,17 @@ class UnivocityParser(
   throw BadRecordException(
 () => getCurrentInput,
 () => None,
-QueryExecutionErrors.malformedCSVRecordError())
+QueryExecutionErrors.malformedCSVRecordError(""))
 }
 
+val currentInput = getCurrentInput
+
 var badRecordException: Option[Throwable] = if (tokens.length != 
parsedSchema.length) {
   // If the number of tokens doesn't match the schema, we should treat it 
as a malformed record.
   // However, we still have chance to parse some of the tokens. It 
continues to parses the
   // tokens normally and sets null when `ArrayIndexOutOfBoundsException` 
occurs for missing
   // tokens.
-  Some(QueryExecutionErrors.malformedCSVRecordError())
+  Some(QueryExecutionErrors.malformedCSVRecordError(currentInput.toString))
 } else None
 // When the length of the returned tokens is identical to the length of 
the parsed schema,
 // we just need to:
@@ -360,7 +362,7 @@ class UnivocityParser(
 } else {
   if (badRecordException.isDefined) {
 throw BadRecordException(
-  () => getCurrentInput, () => requiredRow.headOption, 
badRecordException.get)
+  () => currentInput, () => requiredRow.headOption, 
badRecordException.get)
   } else {
 requiredRow
   }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 8634f60e34e..b89c624870e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -1528,10 +1528,10 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
 "name" -> name))
   }
 
-  def malformedCSVRecordError(): SparkRuntimeException = {
+

[spark] branch master updated: [SPARK-41574][SQL] Update `_LEGACY_ERROR_TEMP_2009` as `INTERNAL_ERROR`

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 62ca7647ed5 [SPARK-41574][SQL] Update `_LEGACY_ERROR_TEMP_2009` as 
`INTERNAL_ERROR`
62ca7647ed5 is described below

commit 62ca7647ed5a9b67ad0a873cd511367adf4ccb4b
Author: itholic 
AuthorDate: Wed Jan 18 17:20:40 2023 +0800

[SPARK-41574][SQL] Update `_LEGACY_ERROR_TEMP_2009` as `INTERNAL_ERROR`

### What changes were proposed in this pull request?

This PR proposes to update `_LEGACY_ERROR_TEMP_2136` as `INTERNAL_ERROR`.

### Why are the changes needed?

We should turn error class into INTERNAL_ERROR when it's not triggered by 
user space.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39389 from itholic/SPARK-41574.

Lead-authored-by: itholic 
Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com>
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json| 5 -
 .../apache/spark/sql/catalyst/expressions/windowExpressions.scala   | 6 +++---
 .../scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala  | 5 +
 .../scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala| 6 --
 4 files changed, 8 insertions(+), 14 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 2570ffeba3b..465aa162981 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -3581,11 +3581,6 @@
   "Find an invalid url string . If necessary set  to 
false to bypass this error."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2009" : {
-"message" : [
-  "dataType"
-]
-  },
   "_LEGACY_ERROR_TEMP_2010" : {
 "message" : [
   "Window Functions do not support merging."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index c32bf4d4d45..6fed9714002 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -27,7 +27,7 @@ import 
org.apache.spark.sql.catalyst.expressions.Cast.{toSQLExpr, toSQLType}
 import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateFunction, 
DeclarativeAggregate, NoOp}
 import org.apache.spark.sql.catalyst.trees.{BinaryLike, LeafLike, TernaryLike, 
UnaryLike}
 import org.apache.spark.sql.catalyst.trees.TreePattern.{TreePattern, 
UNRESOLVED_WINDOW_EXPRESSION, WINDOW_EXPRESSION}
-import org.apache.spark.sql.errors.{QueryErrorsBase, QueryExecutionErrors}
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase, 
QueryExecutionErrors}
 import org.apache.spark.sql.types._
 
 /**
@@ -62,7 +62,7 @@ case class WindowSpecDefinition(
   checkInputDataTypes().isSuccess
 
   override def nullable: Boolean = true
-  override def dataType: DataType = throw 
QueryExecutionErrors.dataTypeOperationUnsupportedError
+  override def dataType: DataType = throw 
QueryCompilationErrors.dataTypeOperationUnsupportedError
 
   override def checkInputDataTypes(): TypeCheckResult = {
 frameSpecification match {
@@ -182,7 +182,7 @@ case object CurrentRow extends SpecialFrameBoundary {
  * Represents a window frame.
  */
 sealed trait WindowFrame extends Expression with Unevaluable {
-  override def dataType: DataType = throw 
QueryExecutionErrors.dataTypeOperationUnsupportedError
+  override def dataType: DataType = throw 
QueryCompilationErrors.dataTypeOperationUnsupportedError
   override def nullable: Boolean = false
 }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 83b69d700ac..91412b760bd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -3401,4 +3401,9 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase {
   )
 )
   }
+
+  def dataTypeOperationUnsupportedError(): Throwable = {
+SparkException.internalError(
+  "The operation `dataType` is not supported.")
+  }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/o

[spark] branch master updated: [SPARK-41579][SQL] Assign name to _LEGACY_ERROR_TEMP_1249

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 13d9dd11a84 [SPARK-41579][SQL] Assign name to _LEGACY_ERROR_TEMP_1249
13d9dd11a84 is described below

commit 13d9dd11a842494f9306bde83a867231798e872c
Author: itholic 
AuthorDate: Wed Jan 18 17:19:38 2023 +0800

[SPARK-41579][SQL] Assign name to _LEGACY_ERROR_TEMP_1249

### What changes were proposed in this pull request?

This PR proposes to assign name to _LEGACY_ERROR_TEMP_1249, 
"NOT_A_PARTITIONED_TABLE".

### Why are the changes needed?

We should assign proper name to _LEGACY_ERROR_TEMP_*

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39260 from itholic/LEGACY_1249.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json | 15 ++-
 .../apache/spark/sql/errors/QueryCompilationErrors.scala | 12 ++--
 .../command/v1/AlterTableRecoverPartitionsSuite.scala| 16 
 3 files changed, 32 insertions(+), 11 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 39c23054945..2570ffeba3b 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1006,6 +1006,11 @@
 ],
 "sqlState" : "42000"
   },
+  "NOT_A_PARTITIONED_TABLE" : {
+"message" : [
+  "Operation  is not allowed for  because it 
is not a partitioned table."
+]
+  },
   "NO_HANDLER_FOR_UDAF" : {
 "message" : [
   "No handler for UDAF ''. Use 
sparkSession.udf.register(...) instead."
@@ -3062,11 +3067,6 @@
   "Operation not allowed: ALTER TABLE SET SERDE is not supported for 
tables created with the datasource API."
 ]
   },
-  "_LEGACY_ERROR_TEMP_1249" : {
-"message" : [
-  "Operation not allowed:  only works on partitioned tables: 
."
-]
-  },
   "_LEGACY_ERROR_TEMP_1250" : {
 "message" : [
   " is not allowed on  since filesource partition 
management is disabled (spark.sql.hive.manageFilesourcePartitions = false)."
@@ -5293,5 +5293,10 @@
 "message" : [
   "grouping() can only be used with GroupingSets/Cube/Rollup"
 ]
+  },
+  "_LEGACY_ERROR_TEMP_2446" : {
+"message" : [
+  "Operation not allowed:  only works on table with location 
provided: "
+]
   }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 69128d748ce..83b69d700ac 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -2393,22 +2393,22 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase {
   messageParameters = Map.empty)
   }
 
-  def cmdOnlyWorksOnPartitionedTablesError(cmd: String, tableIdentWithDB: 
String): Throwable = {
+  def cmdOnlyWorksOnPartitionedTablesError(
+  operation: String,
+  tableIdentWithDB: String): Throwable = {
 new AnalysisException(
-  errorClass = "_LEGACY_ERROR_TEMP_1249",
+  errorClass = "NOT_A_PARTITIONED_TABLE",
   messageParameters = Map(
-"cmd" -> cmd,
+"operation" -> toSQLStmt(operation),
 "tableIdentWithDB" -> tableIdentWithDB))
   }
 
   def cmdOnlyWorksOnTableWithLocationError(cmd: String, tableIdentWithDB: 
String): Throwable = {
 new AnalysisException(
-  errorClass = "_LEGACY_ERROR_TEMP_1249",
+  errorClass = "_LEGACY_ERROR_TEMP_2446",
   messageParameters = Map(
 "cmd" -> cmd,
 "tableIdentWithDB" -> tableIdentWithDB))
-new AnalysisException(s"Operation not allowed: $cmd only works on table 
with " +
-  s"location provided: $tableIdentWithDB")
   }
 
   def actionNotAllowedOnTableWithFilesourcePartitionManagementDisabledError(
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRecoverPartitionsSuite.scala
index 32963f5fc2d..b219e21a3d8 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1

[spark] branch master updated: [SPARK-41984][SQL] Rename & improve error message for `RESET_PERMISSION_TO_ORIGINAL`

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 757e5448c4c [SPARK-41984][SQL] Rename & improve error message for 
`RESET_PERMISSION_TO_ORIGINAL`
757e5448c4c is described below

commit 757e5448c4cf9be0728766ed02cd05bd0926e215
Author: itholic 
AuthorDate: Wed Jan 18 17:17:06 2023 +0800

[SPARK-41984][SQL] Rename & improve error message for 
`RESET_PERMISSION_TO_ORIGINAL`

### What changes were proposed in this pull request?

This PR proposes to rename & improve error message for 
`RESET_PERMISSION_TO_ORIGINAL`

### Why are the changes needed?

We should have proper name and error message for error classes.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"`

Closes #39507 from itholic/RESET_PERMISSION_TO_ORIGINAL.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   | 10 +-
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 10 +++---
 .../scala/org/apache/spark/sql/execution/command/tables.scala  |  4 ++--
 .../apache/spark/sql/errors/QueryExecutionErrorsSuite.scala|  7 +++
 4 files changed, 13 insertions(+), 18 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 9db0dc74562..39c23054945 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -93,6 +93,11 @@
 ],
 "sqlState" : "22007"
   },
+  "CANNOT_RESTORE_PERMISSIONS_FOR_PATH" : {
+"message" : [
+  "Failed to set permissions on created path  back to ."
+]
+  },
   "CANNOT_UP_CAST_DATATYPE" : {
 "message" : [
   "Cannot up cast  from  to .",
@@ -1149,11 +1154,6 @@
 ],
 "sqlState" : "42K03"
   },
-  "RESET_PERMISSION_TO_ORIGINAL" : {
-"message" : [
-  "Failed to set original permission  back to the created 
path: . Exception: "
-]
-  },
   "ROUTINE_ALREADY_EXISTS" : {
 "message" : [
   "Cannot create the function  because it already exists.",
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index a957fffc97c..0473736d018 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -2245,16 +2245,12 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
   cause = null)
   }
 
-  def failToSetOriginalPermissionBackError(
-  permission: FsPermission,
-  path: Path,
-  e: Throwable): Throwable = {
+  def cannotRestorePermissionsForPathError(permission: FsPermission, path: 
Path): Throwable = {
 new SparkSecurityException(
-  errorClass = "RESET_PERMISSION_TO_ORIGINAL",
+  errorClass = "CANNOT_RESTORE_PERMISSIONS_FOR_PATH",
   messageParameters = Map(
 "permission" -> permission.toString,
-"path" -> path.toString,
-"message" -> e.getMessage))
+"path" -> path.toString))
   }
 
   def failToSetOriginalACLBackError(
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index 5e733ad9e5b..b47f0e376ac 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -530,8 +530,8 @@ case class TruncateTableCommand(
 fs.setPermission(path, permission)
   } catch {
 case NonFatal(e) =>
-  throw 
QueryExecutionErrors.failToSetOriginalPermissionBackError(
-permission, path, e)
+  throw 
QueryExecutionErrors.cannotRestorePermissionsForPathError(
+permission, path)
   }
 }
 optAcls.foreach { acls =>
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
index e692a1e2ffa..70311a2f7b8 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
+++ 

[spark] branch master updated: [SPARK-41975][SQL] Improve error message for `INDEX_ALREADY_EXISTS`

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 27b9538a3bf [SPARK-41975][SQL] Improve error message for 
`INDEX_ALREADY_EXISTS`
27b9538a3bf is described below

commit 27b9538a3bf44973b7b7d266fbcff88631753ffd
Author: itholic 
AuthorDate: Wed Jan 18 17:15:40 2023 +0800

[SPARK-41975][SQL] Improve error message for `INDEX_ALREADY_EXISTS`

### What changes were proposed in this pull request?

This PR proposes to improve error message for `INDEX_ALREADY_EXISTS`.

### Why are the changes needed?

Make the error message more clear and proper.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Fix UT and `./build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39497 from itholic/INDEX_ALREADY_EXISTS.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 .../test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 11 +++
 core/src/main/resources/error/error-classes.json |  2 +-
 .../spark/sql/catalyst/analysis/AlreadyExistException.scala  | 12 +---
 .../src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala |  7 ++-
 .../main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala  |  7 ++-
 .../scala/org/apache/spark/sql/jdbc/PostgresDialect.scala|  8 +++-
 .../test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala   |  3 ++-
 7 files changed, 38 insertions(+), 12 deletions(-)

diff --git 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
index 3528540b425..5bedcbd172e 100644
--- 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
+++ 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala
@@ -253,10 +253,13 @@ private[v2] trait V2JDBCTest extends SharedSparkSession 
with DockerIntegrationFu
 // This should pass without exception
 sql(s"CREATE index IF NOT EXISTS i1 ON $catalogName.new_table (col1)")
 
-m = intercept[IndexAlreadyExistsException] {
-  sql(s"CREATE index i1 ON $catalogName.new_table (col1)")
-}.getMessage
-assert(m.contains("Failed to create index i1 in new_table"))
+checkError(
+  exception = intercept[IndexAlreadyExistsException] {
+sql(s"CREATE index i1 ON $catalogName.new_table (col1)")
+  },
+  errorClass = "INDEX_ALREADY_EXISTS",
+  parameters = Map("indexName" -> "i1", "tableName" -> "new_table")
+)
 
 sql(s"DROP index i1 ON $catalogName.new_table")
 assert(jdbcTable.indexExists("i1") == false)
diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 91655f6e62f..9db0dc74562 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -643,7 +643,7 @@
   },
   "INDEX_ALREADY_EXISTS" : {
 "message" : [
-  "Cannot create the index because it already exists. ."
+  "Cannot create the index  on table  because it 
already exists."
 ],
 "sqlState" : "42710"
   },
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
index 1b5dca840d6..762b6155d5d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
@@ -136,6 +136,12 @@ class FunctionAlreadyExistsException(errorClass: String, 
messageParameters: Map[
   }
 }
 
-class IndexAlreadyExistsException(message: String, cause: Option[Throwable] = 
None)
-  extends AnalysisException(errorClass = "INDEX_ALREADY_EXISTS",
-Map("message" -> message), cause)
+class IndexAlreadyExistsException(
+indexName: String,
+tableName: String,
+cause: Option[Throwable] = None)
+  extends AnalysisException(
+errorClass = "INDEX_ALREADY_EXISTS",
+Map("indexName" -> indexName, "tableName" -> tableName),
+cause
+  )
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
index 0b735dd0e46..eac3dab4f6b 100644
--- a/sql/core/

[spark] branch master updated: [SPARK-41974][SQL] Turn `INCORRECT_END_OFFSET` into `INTERNAL_ERROR`

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new edafe266144 [SPARK-41974][SQL] Turn `INCORRECT_END_OFFSET` into 
`INTERNAL_ERROR`
edafe266144 is described below

commit edafe266144c5c70852491fef9bb6907a001b286
Author: itholic 
AuthorDate: Wed Jan 18 17:14:16 2023 +0800

[SPARK-41974][SQL] Turn `INCORRECT_END_OFFSET` into `INTERNAL_ERROR`

### What changes were proposed in this pull request?

This PR proposes to update `INCORRECT_END_OFFSET` as `INTERNAL_ERROR`.

### Why are the changes needed?

We should turn error class into INTERNAL_ERROR when it's not triggered by 
user space.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

`./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*`

Closes #39496 from itholic/INCORRECT_END_OFFSET.

Authored-by: itholic 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/errors/QueryExecutionErrors.scala  | 10 +++---
 .../streaming/sources/RateStreamProviderSuite.scala | 17 +
 2 files changed, 12 insertions(+), 15 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 9598933d941..a957fffc97c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -2469,13 +2469,9 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
   def incorrectEndOffset(rowsPerSecond: Long,
   maxSeconds: Long,
   endSeconds: Long): Throwable = {
-new SparkRuntimeException(
-  errorClass = "INCORRECT_END_OFFSET",
-  messageParameters = Map(
-"rowsPerSecond" -> rowsPerSecond.toString,
-"maxSeconds" -> maxSeconds.toString,
-"endSeconds" -> endSeconds.toString
-  ))
+SparkException.internalError(
+  s"Max offset with ${rowsPerSecond.toString} rowsPerSecond is 
${maxSeconds.toString}, " +
+s"but it's ${endSeconds.toString} now.")
   }
 
   def failedToReadDeltaFileError(fileToRead: Path, clazz: String, keySize: 
Int): Throwable = {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
index aebeb08775f..730611f8f35 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
@@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
-import org.apache.spark.{SparkRuntimeException}
+import org.apache.spark.{SparkException, SparkRuntimeException}
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream}
@@ -218,7 +218,7 @@ class RateStreamProviderSuite extends StreamTest {
 withTempDir { temp =>
   val maxSeconds = (Long.MaxValue / 100)
   val endSeconds = Long.MaxValue
-  val e = intercept[SparkRuntimeException](
+  val e = intercept[SparkException](
 new RateStreamMicroBatchStream(
   rowsPerSecond = 100,
   rampUpTimeSeconds = 2,
@@ -228,11 +228,12 @@ class RateStreamProviderSuite extends StreamTest {
 
   checkError(
 exception = e,
-errorClass = "INCORRECT_END_OFFSET",
+errorClass = "INTERNAL_ERROR",
 parameters = Map(
-  "rowsPerSecond" -> "100",
-  "maxSeconds" -> maxSeconds.toString,
-  "endSeconds" -> endSeconds.toString))
+  ("message" ->
+("Max offset with 100 rowsPerSecond is 92233720368547758, " +
+"but it's 9223372036854775807 now.")
+)))
 }
   }
 
@@ -310,8 +311,8 @@ class RateStreamProviderSuite extends StreamTest {
   .distinct()
 testStream(input)(
   AdvanceRateManualClock(2),
-  ExpectFailure[SparkRuntimeException](t => {
-Seq("INCORRECT_END_OFFSET", "rowsPerSecond").foreach { msg =>
+  ExpectFailure[SparkException](t => {
+Seq("INTERNAL_ERROR", "rowsPerSecond").foreach { msg =>
   assert(t.getMessage.contains(msg))
 }
   })


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (0f1acedad4e -> 38052f9f5ae)

2023-01-18 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 0f1acedad4e [SPARK-42100][SQL][UI] Protect NPE in 
`SQLExecutionUIDataSerializer#serialize`
 add 38052f9f5ae [SPARK-41973][SQL] Assign name to _LEGACY_ERROR_TEMP_1311

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/error-classes.json   | 10 +-
 .../org/apache/spark/sql/errors/QueryCompilationErrors.scala   |  4 ++--
 .../src/main/scala/org/apache/spark/sql/DataFrameWriter.scala  |  2 +-
 .../org/apache/spark/sql/sources/BucketedWriteSuite.scala  |  2 +-
 4 files changed, 9 insertions(+), 9 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-42061][SQL] mark expression InvokeLike and ExternalMapToCatalyst stateful

2023-01-17 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 1f929c3095f [SPARK-42061][SQL] mark expression InvokeLike and 
ExternalMapToCatalyst stateful
1f929c3095f is described below

commit 1f929c3095f7b7918ed02a0d666c0ee469b1ccfc
Author: Fred Liu 
AuthorDate: Wed Jan 18 09:54:14 2023 +0800

[SPARK-42061][SQL] mark expression InvokeLike and ExternalMapToCatalyst 
stateful

### What changes were proposed in this pull request?

Those two expressions involves Array/Buffer that are not thread-safe. Need 
to mark those stateful so existing Spark infra can copy those properly.

### Why are the changes needed?

Avoid correctness issues.

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

This was found in our internal workload which runs queries concurrently. 
The problem is gone after the fix.

Closes #39630 from lzlfred/fix_stateful_expression.

Authored-by: Fred Liu 
Signed-off-by: Wenchen Fan 
---
 .../org/apache/spark/sql/catalyst/expressions/objects/objects.scala | 6 ++
 1 file changed, 6 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index 56facda2af6..299a928f267 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -50,6 +50,9 @@ trait InvokeLike extends Expression with NonSQLExpression 
with ImplicitCastInput
 
   def propagateNull: Boolean
 
+  // InvokeLike is stateful because of the evaluatedArgs Array
+  override def stateful: Boolean = true
+
   override def foldable: Boolean =
 children.forall(_.foldable) && deterministic && 
trustedSerializable(dataType)
   protected lazy val needNullCheck: Boolean = 
needNullCheckForIndex.contains(true)
@@ -1400,6 +1403,9 @@ case class ExternalMapToCatalyst private(
 
   override def nullable: Boolean = inputData.nullable
 
+  // ExternalMapToCatalyst is stateful because of the rowBuffer in 
mapCatalystConverter
+  override def stateful: Boolean = true
+
   override def children: Seq[Expression] = Seq(
 keyLoopVar, keyConverter, valueLoopVar, valueConverter, inputData)
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-40599][SQL] Add multiTransform methods to TreeNode to generate alternatives

2023-01-17 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d8d604bc07b [SPARK-40599][SQL] Add multiTransform methods to TreeNode 
to generate alternatives
d8d604bc07b is described below

commit d8d604bc07bc3b8c98f73c4b10f93cb4eb7113be
Author: Peter Toth 
AuthorDate: Tue Jan 17 20:58:37 2023 +0800

[SPARK-40599][SQL] Add multiTransform methods to TreeNode to generate 
alternatives

### What changes were proposed in this pull request?
This PR introduce `TreeNode.multiTransform()` methods to be able to 
recursively transform a `TreeNode` (and so a tree) into multiple alternatives. 
These functions are particularly useful if we want to transform an expression 
with a projection in which subexpressions can be aliased with multiple 
different attributes.

E.g. if we have a partitioning expression `HashPartitioning(a + b)` and we 
have a `Project` node that aliases `a` as `a1` and `a2` and `b` as `b1` and 
`b2` we can easily generate a stream of alternative transformations of the 
original partitioning:
```
// This is a simplified test, some arguments are missing to make it conciese
val partitioning = HashPartitioning(Add(a, b))
val aliases: Map[Expression, Seq[Attribute]] = ... // collect the alias map 
from project
val s = partitioning.multiTransform {
  case e: Expression if aliases.contains(e.canonicalized) => 
aliases(e.canonicalized)
}
s // Stream(HashPartitioning(Add(a1, b1)), HashPartitioning(Add(a1, b2)), 
HashPartitioning(Add(a2, b2)), HashPartitioning(Add(a2, b2)))
```

The result of `multiTransform` is a lazy stream to be able to limit the 
number of alternatives generated at the caller side as needed.

### Why are the changes needed?
`TreeNode.multiTransform()` is a useful helper method.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
New UTs are added.

Closes #38034 from peter-toth/SPARK-40599-multitransform.

Authored-by: Peter Toth 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/trees/TreeNode.scala | 128 +
 .../spark/sql/catalyst/trees/TreeNodeSuite.scala   | 104 +
 2 files changed, 232 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 9510aa4d9e7..dc64e5e2560 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -618,6 +618,134 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] 
extends Product with Tre
 }
   }
 
+  /**
+   * Returns alternative copies of this node where `rule` has been recursively 
applied to it and all
+   * of its children (pre-order).
+   *
+   * @param rule a function used to generate alternatives for a node
+   * @return the stream of alternatives
+   */
+  def multiTransformDown(
+  rule: PartialFunction[BaseType, Stream[BaseType]]): Stream[BaseType] = {
+multiTransformDownWithPruning(AlwaysProcess.fn, UnknownRuleId)(rule)
+  }
+
+  /**
+   * Returns alternative copies of this node where `rule` has been recursively 
applied to it and all
+   * of its children (pre-order).
+   *
+   * As it is very easy to generate enormous number of alternatives when the 
input tree is huge or
+   * when the rule returns many alternatives for many nodes, this function 
returns the alternatives
+   * as a lazy `Stream` to be able to limit the number of alternatives 
generated at the caller side
+   * as needed.
+   *
+   * The rule should not apply or can return a one element stream of original 
node to indicate that
+   * the original node without any transformation is a valid alternative.
+   *
+   * The rule can return `Stream.empty` to indicate that the original node 
should be pruned. In this
+   * case `multiTransform()` returns an empty `Stream`.
+   *
+   * Please consider the following examples of 
`input.multiTransformDown(rule)`:
+   *
+   * We have an input expression:
+   *`Add(a, b)`
+   *
+   * 1.
+   * We have a simple rule:
+   *   `a` => `Stream(1, 2)`
+   *   `b` => `Stream(10, 20)`
+   *   `Add(a, b)` => `Stream(11, 12, 21, 22)`
+   *
+   * The output is:
+   *   `Stream(11, 12, 21, 22)`
+   *
+   * 2.
+   * In the previous example if we want to generate alternatives of `a` and 
`b` too then we need to
+   * explicitly add the original `Add(a, b)` expression to the rule:
+   *   `a` => `Stream(1, 2)`
+   *   `b` => `Stream(10, 20)`
+   *   `Add(a, b)` => `Stream(11, 12, 21, 22, Add(a, b))`
+   *
+   * The output is:
+   *   `Stream(11, 12, 21, 2

[spark] branch master updated (e6e454299ca -> 97a6955278c)

2023-01-16 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from e6e454299ca [SPARK-41866][CONNECT][TESTS] Enable 
test_create_dataframe_from_array_of_long in dataframe parity test
 add 97a6955278c [SPARK-41982][SQL] Partitions of type string should not be 
treated as numeric types

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/parser/AstBuilder.scala | 20 +---
 .../org/apache/spark/sql/internal/SQLConf.scala| 10 ++
 .../spark/sql/execution/SparkSqlParser.scala   |  2 +-
 .../org/apache/spark/sql/SQLInsertTestSuite.scala  | 29 +
 .../command/AlterTableAddPartitionSuiteBase.scala  | 22 +
 .../command/AlterTableDropPartitionSuiteBase.scala | 30 ++
 .../AlterTableRenamePartitionSuiteBase.scala   | 36 ++
 7 files changed, 143 insertions(+), 6 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (f1f17f408ab -> 69d5532502a)

2023-01-16 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from f1f17f408ab [SPARK-42067][CONNECT][BUILD] Upgrade buf from 1.11.0 to 
1.12.0
 add 69d5532502a [SPARK-42058] SQLSTATE 2/2

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/README.md|  57 ++--
 core/src/main/resources/error/error-classes.json   |  85 +++--
 python/pyspark/sql/tests/test_utils.py |   2 +-
 .../resources/sql-tests/results/ansi/array.sql.out |   4 +
 .../resources/sql-tests/results/ansi/cast.sql.out  |   8 +
 .../resources/sql-tests/results/ansi/date.sql.out  |  13 +
 .../results/ansi/datetime-parsing-invalid.sql.out  |   8 +
 .../double-quoted-identifiers-disabled.sql.out |  16 +-
 .../ansi/double-quoted-identifiers-enabled.sql.out |  36 +--
 .../sql-tests/results/ansi/interval.sql.out|  43 ++-
 .../sql-tests/results/ansi/literals.sql.out|  10 +
 .../resources/sql-tests/results/ansi/map.sql.out   |   2 +
 .../results/ansi/string-functions.sql.out  |   7 +
 .../sql-tests/results/ansi/timestamp.sql.out   |  10 +
 .../sql-tests/results/ansi/try_arithmetic.sql.out  |   1 +
 .../results/ansi/try_datetime_functions.sql.out|   1 +
 .../test/resources/sql-tests/results/array.sql.out |   4 +
 .../resources/sql-tests/results/bitwise.sql.out|   2 +
 .../results/ceil-floor-with-scale-param.sql.out|   2 +
 .../results/columnresolution-negative.sql.out  |   8 +-
 .../resources/sql-tests/results/comments.sql.out   |   6 +-
 .../test/resources/sql-tests/results/count.sql.out |   1 +
 .../sql-tests/results/csv-functions.sql.out|   3 +
 .../resources/sql-tests/results/cte-legacy.sql.out |  10 +-
 .../resources/sql-tests/results/cte-nested.sql.out |   6 +-
 .../sql-tests/results/cte-nonlegacy.sql.out|   6 +-
 .../test/resources/sql-tests/results/cte.sql.out   |   8 +-
 .../test/resources/sql-tests/results/date.sql.out  |  17 +
 .../results/datetime-formatting-invalid.sql.out|  22 ++
 .../sql-tests/results/datetime-legacy.sql.out  |  22 ++
 .../results/datetime-parsing-invalid.sql.out   |   8 +
 .../resources/sql-tests/results/describe.sql.out   |   2 +-
 .../results/double-quoted-identifiers.sql.out  |  16 +-
 .../sql-tests/results/group-by-all-duckdb.sql.out  |   4 +-
 .../sql-tests/results/group-by-all-mosha.sql.out   |   6 +-
 .../sql-tests/results/group-by-all.sql.out |   8 +-
 .../sql-tests/results/group-by-ordinal.sql.out |   3 +-
 .../resources/sql-tests/results/group-by.sql.out   |  10 +-
 .../resources/sql-tests/results/having.sql.out |   1 +
 .../sql-tests/results/inline-table.sql.out |   2 +-
 .../resources/sql-tests/results/interval.sql.out   |  43 ++-
 .../sql-tests/results/join-lateral.sql.out |  11 +-
 .../sql-tests/results/json-functions.sql.out   |  10 +
 .../resources/sql-tests/results/literals.sql.out   |  10 +
 .../test/resources/sql-tests/results/map.sql.out   |   2 +
 .../sql-tests/results/mask-functions.sql.out   |  17 +
 .../sql-tests/results/natural-join.sql.out |   2 +-
 .../sql-tests/results/order-by-all.sql.out |   2 +-
 .../test/resources/sql-tests/results/pivot.sql.out |   4 +-
 .../results/postgreSQL/aggregates_part1.sql.out|   3 +-
 .../results/postgreSQL/create_view.sql.out |   8 +-
 .../sql-tests/results/postgreSQL/join.sql.out  |  16 +-
 .../results/postgreSQL/select_having.sql.out   |   2 +-
 .../results/postgreSQL/select_implicit.sql.out |   4 +-
 .../sql-tests/results/postgreSQL/union.sql.out |   2 +-
 .../results/postgreSQL/window_part3.sql.out|   5 +-
 .../sql-tests/results/postgreSQL/with.sql.out  |   4 +-
 .../sql-tests/results/query_regex_column.sql.out   |  16 +-
 .../resources/sql-tests/results/random.sql.out |   2 +
 .../sql-tests/results/regexp-functions.sql.out |   2 +
 .../sql-tests/results/show-tables.sql.out  |   4 +-
 .../resources/sql-tests/results/show-views.sql.out |   2 +-
 .../sql-tests/results/show_columns.sql.out |   8 +-
 .../results/sql-compatibility-functions.sql.out|   1 +
 .../sql-tests/results/string-functions.sql.out |   7 +
 .../results/subquery/in-subquery/in-basic.sql.out  |   1 +
 .../negative-cases/invalid-correlation.sql.out |   5 +-
 .../negative-cases/subq-input-typecheck.sql.out|   3 +
 .../scalar-subquery/scalar-subquery-select.sql.out |   1 +
 .../sql-tests/results/table-aliases.sql.out|   2 +-
 .../results/table-valued-functions.sql.out |   8 +
 .../sql-tests/results/timestamp-ntz.sql.out|   1 +
 .../resources/sql-tests/results/timestamp.sql.out  |  14 +
 .../results/timestampNTZ/timestamp-ansi.sql.out|   5 +
 .../results/timestampNTZ/timestamp.sql.out |   9 +
 .../sql-tests/results/try_arithmetic.sql.out   |   1 +
 .../results

[spark] branch master updated (cedc9d2d351 -> b414b895ffd)

2023-01-16 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from cedc9d2d351 [SPARK-41708][SQL][FOLLOWUP] Add a new `replaceAll` to 
`SQLQueryTestHelper#replaceNotIncludedMsg` to remove `@hashCode`
 add b414b895ffd [SPARK-41994] Assign SQLSTATE's (1/2)

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/README.md| 1343 +---
 core/src/main/resources/error/error-classes.json   |  203 +--
 .../org/apache/spark/SparkThrowableSuite.scala |6 +-
 .../catalyst/parser/ExpressionParserSuite.scala|   12 +-
 .../resources/sql-tests/results/ansi/array.sql.out |8 +
 .../resources/sql-tests/results/ansi/cast.sql.out  |   82 +-
 .../resources/sql-tests/results/ansi/date.sql.out  |   14 +-
 .../results/ansi/datetime-parsing-invalid.sql.out  |   38 +-
 .../ansi/decimalArithmeticOperations.sql.out   |   20 +-
 .../double-quoted-identifiers-disabled.sql.out |   26 +-
 .../ansi/double-quoted-identifiers-enabled.sql.out |4 +-
 .../sql-tests/results/ansi/interval.sql.out|   62 +-
 .../sql-tests/results/ansi/literals.sql.out|8 +-
 .../results/ansi/string-functions.sql.out  |   22 +-
 .../sql-tests/results/ansi/timestamp.sql.out   |   18 +-
 .../sql-tests/results/ansi/try_element_at.sql.out  |3 +-
 .../test/resources/sql-tests/results/array.sql.out |3 +-
 .../test/resources/sql-tests/results/cast.sql.out  |   12 +-
 .../results/columnresolution-negative.sql.out  |9 +
 .../sql-tests/results/csv-functions.sql.out|8 +-
 .../test/resources/sql-tests/results/cte.sql.out   |2 +-
 .../test/resources/sql-tests/results/date.sql.out  |6 +-
 .../sql-tests/results/datetime-legacy.sql.out  |   12 +-
 .../sql-tests/results/describe-query.sql.out   |6 +-
 .../resources/sql-tests/results/describe.sql.out   |2 +
 .../results/double-quoted-identifiers.sql.out  |   26 +-
 .../resources/sql-tests/results/except-all.sql.out |1 +
 .../resources/sql-tests/results/extract.sql.out|9 +
 .../sql-tests/results/group-by-filter.sql.out  |5 +-
 .../sql-tests/results/group-by-ordinal.sql.out |   18 +-
 .../resources/sql-tests/results/group-by.sql.out   |   13 +-
 .../sql-tests/results/grouping_set.sql.out |6 +-
 .../sql-tests/results/intersect-all.sql.out|1 +
 .../resources/sql-tests/results/interval.sql.out   |   40 +-
 .../sql-tests/results/json-functions.sql.out   |7 +-
 .../resources/sql-tests/results/literals.sql.out   |8 +-
 .../sql-tests/results/order-by-ordinal.sql.out |6 +-
 .../test/resources/sql-tests/results/pivot.sql.out |9 +-
 .../results/postgreSQL/aggregates_part3.sql.out|1 +
 .../sql-tests/results/postgreSQL/boolean.sql.out   |   30 +-
 .../results/postgreSQL/create_view.sql.out |2 +-
 .../sql-tests/results/postgreSQL/date.sql.out  |   24 +-
 .../sql-tests/results/postgreSQL/float4.sql.out|   14 +-
 .../sql-tests/results/postgreSQL/float8.sql.out|   10 +-
 .../sql-tests/results/postgreSQL/int8.sql.out  |9 +-
 .../sql-tests/results/postgreSQL/join.sql.out  |2 +
 .../sql-tests/results/postgreSQL/numeric.sql.out   |2 +
 .../results/postgreSQL/select_having.sql.out   |1 +
 .../results/postgreSQL/select_implicit.sql.out |5 +-
 .../sql-tests/results/postgreSQL/strings.sql.out   |8 +
 .../sql-tests/results/postgreSQL/text.sql.out  |4 +-
 .../sql-tests/results/postgreSQL/union.sql.out |   18 +-
 .../results/postgreSQL/window_part2.sql.out|2 +-
 .../results/postgreSQL/window_part3.sql.out|4 +-
 .../sql-tests/results/show-tables.sql.out  |4 +-
 .../sql-tests/results/string-functions.sql.out |   14 +-
 .../negative-cases/invalid-correlation.sql.out |1 +
 .../negative-cases/subq-input-typecheck.sql.out|2 +
 .../results/table-valued-functions.sql.out |1 +
 .../resources/sql-tests/results/timestamp.sql.out  |6 +-
 .../results/timestampNTZ/timestamp-ansi.sql.out|   20 +-
 .../results/timestampNTZ/timestamp.sql.out |6 +-
 .../resources/sql-tests/results/transform.sql.out  |6 +-
 .../sql-tests/results/try_element_at.sql.out   |3 +-
 .../results/udaf/udaf-group-by-ordinal.sql.out |   11 +-
 .../sql-tests/results/udaf/udaf-group-by.sql.out   |   12 +-
 .../results/udaf/udaf-grouping-set.sql.out |4 +-
 .../udf/postgreSQL/udf-aggregates_part3.sql.out|1 +
 .../results/udf/postgreSQL/udf-join.sql.out|2 +
 .../udf/postgreSQL/udf-select_having.sql.out   |1 +
 .../udf/postgreSQL/udf-select_implicit.sql.out |5 +-
 .../sql-tests/results/udf/udf-except-all.sql.out   |1 +
 .../sql-tests/results/udf/udf-group-by.sql.out |   12 +-
 .../results/udf/udf-intersect

[spark] branch master updated (9b647e80b9e -> 2d4be52b71b)

2023-01-16 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 9b647e80b9e [SPARK-42087][SQL][TESTS] Use `--no-same-owner` when 
`HiveExternalCatalogVersionsSuite` untars
 add 2d4be52b71b [SPARK-41993][SQL] Move RowEncoder to AgnosticEncoders

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/JavaTypeInference.scala |   4 +-
 .../spark/sql/catalyst/ScalaReflection.scala   | 315 --
 .../spark/sql/catalyst/SerializerBuildHelper.scala |  25 +-
 .../sql/catalyst/encoders/AgnosticEncoder.scala| 128 ++--
 .../sql/catalyst/encoders/ExpressionEncoder.scala  |   5 +-
 .../spark/sql/catalyst/encoders/RowEncoder.scala   | 354 -
 .../sql/catalyst/expressions/objects/objects.scala |  76 ++---
 .../spark/sql/catalyst/ScalaReflectionSuite.scala  |   9 +-
 .../catalyst/encoders/ExpressionEncoderSuite.scala |   2 +
 .../sql/catalyst/encoders/RowEncoderSuite.scala|  14 +
 .../catalyst/expressions/CodeGenerationSuite.scala |   2 +-
 .../expressions/ObjectExpressionsSuite.scala   |   9 +-
 12 files changed, 444 insertions(+), 499 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41990][SQL][FOLLOWUP] Add comments to explain why `FieldReference.column` is used when `ParseException` happens

2023-01-16 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new b13dc699f7d [SPARK-41990][SQL][FOLLOWUP] Add comments to explain why 
`FieldReference.column` is used when `ParseException` happens
b13dc699f7d is described below

commit b13dc699f7df589663b1239e5e551815d9043f38
Author: huaxingao 
AuthorDate: Mon Jan 16 16:52:46 2023 +0800

[SPARK-41990][SQL][FOLLOWUP] Add comments to explain why 
`FieldReference.column` is used when `ParseException` happens

### What changes were proposed in this pull request?

Add comments to explain why `FieldReference.column` is used when 
`ParseException` happens

### Why are the changes needed?
To address this 
[comment](https://github.com/apache/spark/pull/39564#discussion_r1070788648)

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Only comments are added. No need to test.

Closes #39597 from huaxingao/followup.

Authored-by: huaxingao 
Signed-off-by: Wenchen Fan 
---
 .../src/main/scala/org/apache/spark/sql/sources/filters.scala   | 6 ++
 1 file changed, 6 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
index 080d17b47fb..af5e4f5ef5a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
@@ -80,6 +80,12 @@ sealed abstract class Filter {
 try {
   FieldReference(attribute)
 } catch {
+  // The column name in V1 Filter is generated by PushableColumn. When 
nested
+  // predicate push down is enabled, PushableColumn quotes the column name.
+  // However, when nested predicate push down is disabled, PushableColumn
+  // does not quote the column. If the column name is not a valid SQL 
identifier,
+  // FieldReference.apply throws ParseException. We catch the 
ParseException here
+  // and use FieldReference.column instead.
   case _: ParseException =>
 FieldReference.column(attribute)
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (edb96a068fa -> f0a901058aa)

2023-01-13 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from edb96a068fa [SPARK-42001][CONNECT][PYTHON][TESTS] Update the related 
JIRA tickets of two DataFrameReader tests
 add f0a901058aa [SPARK-41896][SQL] Filtering by row index returns empty 
results

No new revisions were added by this update.

Summary of changes:
 .../execution/datasources/FileSourceStrategy.scala | 46 ++-
 .../datasources/PartitioningAwareFileIndex.scala   | 28 ---
 .../datasources/FileMetadataStructSuite.scala  | 96 ++
 3 files changed, 156 insertions(+), 14 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41961][SQL] Support table-valued functions with LATERAL

2023-01-13 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 50780af2d82 [SPARK-41961][SQL] Support table-valued functions with 
LATERAL
50780af2d82 is described below

commit 50780af2d82689f7501a82d0ff9d5ace99f0703d
Author: allisonwang-db 
AuthorDate: Sat Jan 14 11:09:58 2023 +0800

[SPARK-41961][SQL] Support table-valued functions with LATERAL

### What changes were proposed in this pull request?
This PR allows table-valued functions to reference columns and aliases in 
the preceding FROM items using the LATERAL keyword.

### Why are the changes needed?
To improve the usability of table-valued functions.

### Does this PR introduce _any_ user-facing change?
Yes. Before this PR, users cannot invoke a table-valued function in the 
FROM clause with LATERAL:
```
SELECT * FROM t, LATERAL EXPLODE(ARRAY(t.c1, t.c2));
[INVALID_SQL_SYNTAX] Invalid SQL syntax: LATERAL can only be used with 
subquery.
```
After this PR, this query can run successfully.

### How was this patch tested?

New SQL query tests.

Closes #39479 from allisonwang-db/spark-41961-lateral-tvf.

Authored-by: allisonwang-db 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |   2 +-
 .../spark/sql/catalyst/optimizer/subquery.scala|   6 +
 .../spark/sql/catalyst/parser/AstBuilder.scala |  19 ++-
 .../spark/sql/errors/QueryParsingErrors.scala  |   3 +-
 .../OptimizeOneRowRelationSubquerySuite.scala  |  28 -
 .../resources/sql-tests/inputs/join-lateral.sql|  19 +++
 .../sql-tests/results/join-lateral.sql.out | 135 +
 .../spark/sql/errors/QueryParsingErrorsSuite.scala |   6 +-
 .../execution/datasources/SchemaPruningSuite.scala |  20 +++
 9 files changed, 226 insertions(+), 12 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 3b3a011db97..d6b68a45e77 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -73,7 +73,7 @@ object SimpleAnalyzer extends Analyzer(
 new SessionCatalog(
   new InMemoryCatalog,
   FunctionRegistry.builtin,
-  EmptyTableFunctionRegistry) {
+  TableFunctionRegistry.builtin) {
   override def createDatabase(dbDefinition: CatalogDatabase, 
ignoreIfExists: Boolean): Unit = {}
 })) {
   override def resolver: Resolver = caseSensitiveResolution
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 4d2c64c7c32..faafeecc316 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -761,6 +761,7 @@ object OptimizeOneRowRelationSubquery extends 
Rule[LogicalPlan] {
   CollapseProject(EliminateSubqueryAliases(plan), alwaysInline = 
alwaysInline) match {
 case p @ Project(_, _: OneRowRelation) => Some(p)
 case g @ Generate(_, _, _, _, _, _: OneRowRelation) => Some(g)
+case p @ Project(_, Generate(_, _, _, _, _, _: OneRowRelation)) => 
Some(p)
 case _ => None
   }
 }
@@ -787,6 +788,11 @@ object OptimizeOneRowRelationSubquery extends 
Rule[LogicalPlan] {
   val newGenerator = stripOuterReference(generator)
   g.copy(generator = newGenerator, child = left)
 
+case Project(projectList, g @ Generate(generator, _, _, _, _, _: 
OneRowRelation)) =>
+  val newPList = stripOuterReferences(projectList)
+  val newGenerator = stripOuterReference(generator)
+  Project(left.output ++ newPList, g.copy(generator = newGenerator, 
child = left))
+
 case o =>
   throw SparkException.internalError(
 s"Unexpected plan when optimizing one row relation subquery: $o")
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index eb85aee25ce..e74fe5ce003 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -914,15 +914,19 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] 
with SQLConfHelper wit
*/
   override def visitFromClause(ctx: FromClauseContext): LogicalPlan = 
withOrigin(ctx) {
 val from = ctx.re

[spark] branch master updated: [SPARK-42003][SQL] Reduce duplicate code in ResolveGroupByAll

2023-01-12 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c05236237ed [SPARK-42003][SQL] Reduce duplicate code in 
ResolveGroupByAll
c05236237ed is described below

commit c05236237ed7c0ad7dfbe2a185bd96acf51a2c4f
Author: Gengliang Wang 
AuthorDate: Thu Jan 12 17:03:45 2023 +0800

[SPARK-42003][SQL] Reduce duplicate code in ResolveGroupByAll

### What changes were proposed in this pull request?

Reduce duplicate code in ResolveGroupByAll by moving the group by 
expression inference into a new method.

### Why are the changes needed?

Code clean up

### Does this PR introduce _any_ user-facing change?

No
### How was this patch tested?

Existing UT

Closes #39523 from gengliangwang/refactorAll.

Authored-by: Gengliang Wang 
Signed-off-by: Wenchen Fan 
---
 .../sql/catalyst/analysis/ResolveGroupByAll.scala  | 36 +++---
 1 file changed, 25 insertions(+), 11 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupByAll.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupByAll.scala
index d45ea412031..8c6ba20cd1a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupByAll.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupByAll.scala
@@ -47,6 +47,24 @@ object ResolveGroupByAll extends Rule[LogicalPlan] {
 }
   }
 
+  /**
+   * Returns all the grouping expressions inferred from a GROUP BY ALL 
aggregate.
+   * The result is optional. If Spark fails to infer the grouping columns, it 
is None.
+   * Otherwise, it contains all the non-aggregate expressions from the project 
list of the input
+   * Aggregate.
+   */
+  private def getGroupingExpressions(a: Aggregate): Option[Seq[Expression]] = {
+val groupingExprs = 
a.aggregateExpressions.filter(!_.exists(AggregateExpression.isAggregate))
+// If the grouping exprs are empty, this could either be (1) a valid 
global aggregate, or
+// (2) we simply fail to infer the grouping columns. As an example, in "i 
+ sum(j)", we will
+// not automatically infer the grouping column to be "i".
+if (groupingExprs.isEmpty && 
a.aggregateExpressions.exists(containsAttribute)) {
+  None
+} else {
+  Some(groupingExprs)
+}
+  }
+
   override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUpWithPruning(
 _.containsAllPatterns(UNRESOLVED_ATTRIBUTE, AGGREGATE), ruleId) {
 case a: Aggregate
@@ -54,18 +72,15 @@ object ResolveGroupByAll extends Rule[LogicalPlan] {
   // Only makes sense to do the rewrite once all the aggregate expressions 
have been resolved.
   // Otherwise, we might incorrectly pull an actual aggregate expression 
over to the grouping
   // expression list (because we don't know they would be aggregate 
expressions until resolved).
-  val groupingExprs = 
a.aggregateExpressions.filter(!_.exists(AggregateExpression.isAggregate))
+  val groupingExprs = getGroupingExpressions(a)
 
-  // If the grouping exprs are empty, this could either be (1) a valid 
global aggregate, or
-  // (2) we simply fail to infer the grouping columns. As an example, in 
"i + sum(j)", we will
-  // not automatically infer the grouping column to be "i".
-  if (groupingExprs.isEmpty && 
a.aggregateExpressions.exists(containsAttribute)) {
-// Case (2): don't replace the ALL. We will eventually tell the user 
in checkAnalysis
-// that we cannot resolve the all in group by.
+  if (groupingExprs.isEmpty) {
+// Don't replace the ALL when we fail to infer the grouping columns. 
We will eventually
+// tell the user in checkAnalysis that we cannot resolve the all in 
group by.
 a
   } else {
-// Case (1): this is a valid global aggregate.
-a.copy(groupingExpressions = groupingExprs)
+// This is a valid GROUP BY ALL aggregate.
+a.copy(groupingExpressions = groupingExprs.get)
   }
   }
 
@@ -94,8 +109,7 @@ object ResolveGroupByAll extends Rule[LogicalPlan] {
*/
   def checkAnalysis(operator: LogicalPlan): Unit = operator match {
 case a: Aggregate if a.aggregateExpressions.forall(_.resolved) && 
matchToken(a) =>
-  val noAgg = 
a.aggregateExpressions.filter(!_.exists(AggregateExpression.isAggregate))
-  if (noAgg.isEmpty && a.aggregateExpressions.exists(containsAttribute)) {
+  if (getGroupingExpressions(a).isEmpty) {
 operator.failAnalysis(
   errorClass = "UNRESOLVED_ALL_IN_GROUP_BY",
   messageParameters = Map.empty)


-

[spark] branch master updated (47afefb6679 -> 63479a235a1)

2023-01-11 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 47afefb6679 [SPARK-41980][CONNECT][TESTS] Enable 
test_functions_broadcast in functions parity test
 add 63479a235a1 [SPARK-41635][SQL] Fix group by all error reporting

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/analysis/ResolveGroupByAll.scala  |  5 +--
 .../resources/sql-tests/inputs/group-by-all.sql| 11 +++---
 .../sql-tests/results/group-by-all.sql.out | 40 +-
 3 files changed, 42 insertions(+), 14 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (aaee89a12fd -> 21e9691cb44)

2023-01-10 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from aaee89a12fd [SPARK-41575][SQL] Assign name to _LEGACY_ERROR_TEMP_2054
 add 21e9691cb44 [SPARK-41708][SQL][FOLLOWUP] WriteFiles should replace 
exprId using new query

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/execution/datasources/V1Writes.scala |  6 ++-
 .../sql/execution/datasources/WriteFiles.scala |  1 +
 .../test/resources/sql-tests/inputs/explain.sql|  5 +++
 .../sql-tests/results/explain-aqe.sql.out  | 48 ++
 .../resources/sql-tests/results/explain.sql.out| 48 ++
 .../org/apache/spark/sql/SQLQueryTestHelper.scala  |  1 +
 .../datasources/V1WriteCommandSuite.scala  |  1 +
 .../sql/hive/execution/InsertIntoHiveTable.scala   |  3 +-
 8 files changed, 110 insertions(+), 3 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41752][SQL][UI] Group nested executions under the root execution

2023-01-10 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c124037b975 [SPARK-41752][SQL][UI] Group nested executions under the 
root execution
c124037b975 is described below

commit c124037b97538b2656d29ce547b2a42209a41703
Author: Linhong Liu 
AuthorDate: Tue Jan 10 20:52:15 2023 +0800

[SPARK-41752][SQL][UI] Group nested executions under the root execution

### What changes were proposed in this pull request?
This PR proposes to group all sub-executions together in SQL UI if they 
belong to the same root execution.

This feature is controlled by conf `spark.ui.sql.groupSubExecutionEnabled` 
and the default value is set to `true`

We can have some follow-up improvements after this PR:
1. Add links to SQL page and Job page to indicate the root execution ID.
2. Better handling for the root execution missing case (e.g. eviction due 
to retaining limit). In this PR, the sub-executions will be displayed ungrouped.

### Why are the changes needed?
better user experience.

In PR #39220, the CTAS query will trigger a sub-execution to perform the 
data insertion. But the current UI will display the two executions separately 
which may confuse the users.
In addition, this change should also help the structured streaming cases

### Does this PR introduce _any_ user-facing change?
Yes, the screenshot of the UI change is shown below
SQL Query:
```
CREATE TABLE t USING PARQUET AS SELECT 'a' as a, 1 as b
```
UI before this PR
https://user-images.githubusercontent.com/67896261/209889679-83909bc9-0e15-4ff1-9aeb-3118e4bab524.png;>

UI after this PR with sub executions collapsed
https://user-images.githubusercontent.com/67896261/209889688-973a4ec9-a5dc-4a8b-8618-c0800733fffa.png;>

UI after this PR with sub execution expanded
https://user-images.githubusercontent.com/67896261/209889718-0e24be12-23d6-4f81-a508-15eac62ec231.png;>

### How was this patch tested?
UT

Closes #39268 from linhongliu-db/SPARK-41752.

Authored-by: Linhong Liu 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/status/protobuf/store_types.proto |   1 +
 .../resources/org/apache/spark/ui/static/webui.css |  12 +
 .../org/apache/spark/internal/config/UI.scala  |   7 +
 .../apache/spark/sql/execution/SQLExecution.scala  |  14 ++
 .../spark/sql/execution/ui/AllExecutionsPage.scala | 275 +++--
 .../sql/execution/ui/SQLAppStatusListener.scala|   6 +-
 .../spark/sql/execution/ui/SQLAppStatusStore.scala |   1 +
 .../spark/sql/execution/ui/SQLListener.scala   |   2 +
 .../org/apache/spark/sql/execution/ui/SQLTab.scala |   3 +
 .../sql/SQLExecutionUIDataSerializer.scala |   1 +
 .../spark/sql/execution/SQLJsonProtocolSuite.scala |   2 +-
 .../history/SQLEventFilterBuilderSuite.scala   |   2 +-
 .../history/SQLLiveEntitiesEventFilterSuite.scala  |   4 +-
 .../sql/execution/ui/AllExecutionsPageSuite.scala  |  50 
 .../execution/ui/MetricsAggregationBenchmark.scala |   1 +
 .../execution/ui/SQLAppStatusListenerSuite.scala   |  12 +-
 .../spark/status/api/v1/sql/SqlResourceSuite.scala |   1 +
 .../sql/KVStoreProtobufSerializerSuite.scala   |   2 +
 18 files changed, 319 insertions(+), 77 deletions(-)

diff --git 
a/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto 
b/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto
index 04c857f7c3c..e9aaad261f9 100644
--- a/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto
+++ b/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto
@@ -417,6 +417,7 @@ message SQLExecutionUIData {
   repeated int64 stages = 11;
   bool metric_values_is_null = 12;
   map metric_values = 13;
+  optional int64 root_execution_id = 14;
 }
 
 message SparkPlanGraphNode {
diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css 
b/core/src/main/resources/org/apache/spark/ui/static/webui.css
index 0252bc80047..f952f86503e 100755
--- a/core/src/main/resources/org/apache/spark/ui/static/webui.css
+++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css
@@ -187,6 +187,18 @@ pre {
   display: none;
 }
 
+.sub-execution-list {
+  font-size: 0.9rem;
+}
+
+.sub-execution-list.collapsed {
+  display: none;
+}
+
+.table-striped .sub-execution-list table tr {
+  background-color: inherit;
+}
+
 .description-input {
   overflow: hidden;
   text-overflow: ellipsis;
diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala 
b/core/src/main/scala/org/apache/spark/internal/config/UI.scala
index d09620b8e34..a32e60de2a4 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/UI.scala
+++ b/core/src/main/scala/org/apache/spark

[spark] branch master updated: [SPARK-41595][SQL] Support generator function explode/explode_outer in the FROM clause

2023-01-09 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new eca21c7ad58 [SPARK-41595][SQL] Support generator function 
explode/explode_outer in the FROM clause
eca21c7ad58 is described below

commit eca21c7ad582c9c374108518842ad816843e1224
Author: allisonwang-db 
AuthorDate: Tue Jan 10 14:09:06 2023 +0800

[SPARK-41595][SQL] Support generator function explode/explode_outer in the 
FROM clause

### What changes were proposed in this pull request?
This PR supports using table-valued generator functions in the FROM clause 
of a query. A generator function can be registered in the table function 
registry and resolved as a table function during analysis.

Note this PR only adds support for two built-in generator functions: 
`explode` and `explode_outer` with literal input values. We will support more 
generator functions and LATERAL references in separate PRs.

### Why are the changes needed?
To make table-valued generator functions more user-friendly and consistent 
with Spark's built-in table function Range.

### Does this PR introduce _any_ user-facing change?
Yes. Before this PR, the built-in generator function explode/explode_outer 
cannot be used in the FROM clause:
```
select * from explode(array(1, 2))

AnalysisException: could not resolve `explode` to a table-valued function;
```
After this PR, we can support this usage:
```
select * from explode(array(1, 2))

+---+
|col|
+---+
|  1|
|  2|
+---+
```

### How was this patch tested?

New SQL query tests.

Closes #39133 from allisonwang-db/spark-41595-explode-in-from.

Authored-by: allisonwang-db 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  52 +++--
 .../sql/catalyst/analysis/FunctionRegistry.scala   |  23 +-
 .../spark/sql/catalyst/analysis/unresolved.scala   |  65 --
 .../sql/catalyst/catalog/SessionCatalog.scala  |   4 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala |   8 +-
 .../spark/sql/catalyst/trees/TreePatterns.scala|   1 +
 .../sql/catalyst/analysis/AnalysisSuite.scala  |   6 +-
 .../sql/catalyst/parser/PlanParserSuite.scala  |  12 +-
 .../sql-tests/inputs/table-valued-functions.sql|  33 +++
 .../results/table-valued-functions.sql.out | 257 +
 10 files changed, 410 insertions(+), 51 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 00a24357226..3b3a011db97 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -2431,7 +2431,7 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 
 def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUpWithPruning(
   _.containsAnyPattern(UNRESOLVED_FUNC, UNRESOLVED_FUNCTION, GENERATOR,
-UNRESOLVED_TABLE_VALUED_FUNCTION), ruleId) {
+UNRESOLVED_TABLE_VALUED_FUNCTION, UNRESOLVED_TVF_ALIASES), ruleId) {
   // Resolve functions with concrete relations from v2 catalog.
   case u @ UnresolvedFunctionName(nameParts, cmd, requirePersistentFunc, 
mismatchHint, _) =>
 lookupBuiltinOrTempFunction(nameParts)
@@ -2453,7 +2453,7 @@ class Analyzer(override val catalogManager: 
CatalogManager)
   // Resolve table-valued function references.
   case u: UnresolvedTableValuedFunction if 
u.functionArgs.forall(_.resolved) =>
 withPosition(u) {
-  val resolvedFunc = try {
+  try {
 resolveBuiltinOrTempTableFunction(u.name, 
u.functionArgs).getOrElse {
   val CatalogAndIdentifier(catalog, ident) = 
expandIdentifier(u.name)
   if (CatalogV2Util.isSessionCatalog(catalog)) {
@@ -2470,27 +2470,26 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 errorClass = "_LEGACY_ERROR_TEMP_2308",
 messageParameters = Map("name" -> u.name.quoted))
   }
-  // If alias names assigned, add `Project` with the aliases
-  if (u.outputNames.nonEmpty) {
-val outputAttrs = resolvedFunc.output
-// Checks if the number of the aliases is equal to expected one
-if (u.outputNames.size != outputAttrs.size) {
-  u.failAnalysis(
-errorClass = "_LEGACY_ERROR_TEMP_2307",
-messageParameters = Map(
-  "funcName" -> u.name.quoted,
-  "aliasesNum" -> u.outputNames.size.toStr

[spark] branch master updated: [SPARK-41914][SQL] FileFormatWriter materializes AQE plan before accessing outputOrdering

2023-01-09 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c0911981887 [SPARK-41914][SQL] FileFormatWriter materializes AQE plan 
before accessing outputOrdering
c0911981887 is described below

commit c091198188789afb1282bc76419cf6e1397b0bc9
Author: Enrico Minack 
AuthorDate: Tue Jan 10 13:10:07 2023 +0800

[SPARK-41914][SQL] FileFormatWriter materializes AQE plan before accessing 
outputOrdering

### What changes were proposed in this pull request?
The `FileFormatWriter` materializes an `AdaptiveQueryPlan` before accessing 
the plan's `outputOrdering`. This is required when planned writing is disabled 
(`spark.sql.optimizer.plannedWrite.enabled` is `true` by default). With planned 
writing enabled `FileFormatWriter` gets the final plan already.

### Why are the changes needed?
`FileFormatWriter` enforces an ordering if the written plan does not 
provide that ordering. An `AdaptiveQueryPlan` does not know its final ordering, 
in which case `FileFormatWriter` enforces the ordering (e.g. by column `"a"`) 
even if the plan provides a compatible ordering (e.g. by columns `"a", "b"`). 
In case of spilling, that order (e.g. by columns `"a", "b"`) gets broken (see 
SPARK-40588).

### Does this PR introduce _any_ user-facing change?
This fixes SPARK-40588 for 3.4, which was introduced in 3.0. This restores 
behaviour from Spark 2.4.

### How was this patch tested?
The final plan that is written to files is now stored in 
`FileFormatWriter.executedPlan` (similar to existing 
`FileFormatWriter.outputOrderingMatched`). Unit tests assert the outermost sort 
order written to files.

The actual plan written into the files changed from (taken from 
`"SPARK-41914: v1 write with AQE and in-partition sorted - non-string partition 
column"`):

```
Sort [input[2, int, false] ASC NULLS FIRST], false, 0
+- *(3) Sort [key#13 ASC NULLS FIRST, value#14 ASC NULLS FIRST], false, 0
   +- *(3) Project [b#24, value#14, key#13]
  +- *(3) BroadcastHashJoin [key#13], [a#23], Inner, BuildLeft, false
 :- BroadcastQueryStage 2
 :  +- BroadcastExchange 
HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), 
[plan_id=376]
 : +- AQEShuffleRead local
 :+- ShuffleQueryStage 0
 :   +- Exchange hashpartitioning(key#13, 5), 
ENSURE_REQUIREMENTS, [plan_id=328]
 :  +- *(1) SerializeFromObject 
[knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, 
staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, 
fromString, knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false, 
true) AS value#14]
 : +- Scan[obj#12]
 +- AQEShuffleRead local
+- ShuffleQueryStage 1
   +- Exchange hashpartitioning(a#23, 5), ENSURE_REQUIREMENTS, 
[plan_id=345]
  +- *(2) SerializeFromObject 
[knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, 
knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24]
 +- Scan[obj#22]
```

where `FileFormatWriter` enforces order with `Sort [input[2, int, false] 
ASC NULLS FIRST], false, 0`, to

```
*(3) Sort [key#13 ASC NULLS FIRST, value#14 ASC NULLS FIRST], false, 0
+- *(3) Project [b#24, value#14, key#13]
   +- *(3) BroadcastHashJoin [key#13], [a#23], Inner, BuildLeft, false
  :- BroadcastQueryStage 2
  :  +- BroadcastExchange 
HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), 
[plan_id=375]
  : +- AQEShuffleRead local
  :+- ShuffleQueryStage 0
  :   +- Exchange hashpartitioning(key#13, 5), 
ENSURE_REQUIREMENTS, [plan_id=327]
  :  +- *(1) SerializeFromObject 
[knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, 
staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, 
fromString, knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false, 
true) AS value#14]
  : +- Scan[obj#12]
  +- AQEShuffleRead local
 +- ShuffleQueryStage 1
+- Exchange hashpartitioning(a#23, 5), ENSURE_REQUIREMENTS, 
[plan_id=344]
   +- *(2) SerializeFromObject 
[knownnotnull(assertnotnull(input[0, 
org.apache.spark.sql.test.SQLTestDa

[spark] branch master updated: [SPARK-41943][CORE] Use java api to create files and grant permissions

2023-01-09 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new b22946ed8b5 [SPARK-41943][CORE] Use java api to create files and grant 
permissions
b22946ed8b5 is described below

commit b22946ed8b5f41648a32a4e0c4c40226141a06a0
Author: smallzhongfeng 
AuthorDate: Tue Jan 10 10:40:44 2023 +0800

[SPARK-41943][CORE] Use java api to create files and grant permissions

### What changes were proposed in this pull request?

For method `createDirWithPermission770`, using java api to create files and 
grant permissions instead of calling shell commands.

### Why are the changes needed?

Safer and more efficient.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Origin uts.

Closes #39448 from smallzhongfeng/java-api-mkdir.

Authored-by: smallzhongfeng 
Signed-off-by: Wenchen Fan 
---
 .../org/apache/spark/storage/DiskBlockManager.scala  | 16 +---
 1 file changed, 5 insertions(+), 11 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala 
b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
index e29f3fc1b80..a7ed9226c57 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -19,7 +19,7 @@ package org.apache.spark.storage
 
 import java.io.{File, IOException}
 import java.nio.file.Files
-import java.nio.file.attribute.PosixFilePermission
+import java.nio.file.attribute.{PosixFilePermission, PosixFilePermissions}
 import java.util.UUID
 
 import scala.collection.mutable.HashMap
@@ -301,9 +301,6 @@ private[spark] class DiskBlockManager(
* Create a directory that is writable by the group.
* Grant the permission 770 "rwxrwx---" to the directory so the shuffle 
server can
* create subdirs/files within the merge folder.
-   * TODO: Find out why can't we create a dir using java api with permission 
770
-   *  Files.createDirectories(mergeDir.toPath, 
PosixFilePermissions.asFileAttribute(
-   *  PosixFilePermissions.fromString("rwxrwx---")))
*/
   def createDirWithPermission770(dirToCreate: File): Unit = {
 var attempts = 0
@@ -315,16 +312,13 @@ private[spark] class DiskBlockManager(
 throw 
SparkCoreErrors.failToCreateDirectoryError(dirToCreate.getAbsolutePath, 
maxAttempts)
   }
   try {
-val builder = new ProcessBuilder().command(
-  "mkdir", "-p", "-m770", dirToCreate.getAbsolutePath)
-val proc = builder.start()
-val exitCode = proc.waitFor()
+dirToCreate.mkdirs()
+Files.setPosixFilePermissions(
+  dirToCreate.toPath, PosixFilePermissions.fromString("rwxrwx---"))
 if (dirToCreate.exists()) {
   created = dirToCreate
 }
-logDebug(
-  s"Created directory at ${dirToCreate.getAbsolutePath} with 
permission " +
-s"770 and exitCode $exitCode")
+logDebug(s"Created directory at ${dirToCreate.getAbsolutePath} with 
permission 770")
   } catch {
 case e: SecurityException =>
   logWarning(s"Failed to create directory 
${dirToCreate.getAbsolutePath} " +


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-40711][SQL] Add spill size metrics for window

2023-01-08 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c6788e5c2fc [SPARK-40711][SQL] Add spill size metrics for window
c6788e5c2fc is described below

commit c6788e5c2fc3ea6e65f9d49f98dec1cd5f2b820d
Author: ulysses-you 
AuthorDate: Mon Jan 9 12:44:23 2023 +0800

[SPARK-40711][SQL] Add spill size metrics for window

### What changes were proposed in this pull request?

Window may spill if one partition size is large that can not hold in 
memory. This pr makes window support report spill size metrics.

### Why are the changes needed?

Help user get window spill information, to track how much size would spill.

### Does this PR introduce _any_ user-facing change?

yes, a new metrics. people can see it in UI

### How was this patch tested?

add test for window and manual test for WindowInPandasExec:

https://user-images.githubusercontent.com/12025282/194706054-91c75f5f-e513-40fb-a148-6493d97f8c51.png;>

Closes #38163 from ulysses-you/window-metrics.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/execution/python/WindowInPandasExec.scala  |  6 ++
 .../apache/spark/sql/execution/window/WindowExec.scala   |  6 ++
 .../spark/sql/execution/metric/SQLMetricsSuite.scala | 16 
 3 files changed, 28 insertions(+)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
index dcaffed89cc..5e903aa991d 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, 
SparkPlan}
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
 import org.apache.spark.sql.execution.window._
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.ArrowUtils
@@ -85,6 +86,9 @@ case class WindowInPandasExec(
 orderSpec: Seq[SortOrder],
 child: SparkPlan)
   extends WindowExecBase with PythonSQLMetrics {
+  override lazy val metrics: Map[String, SQLMetric] = pythonMetrics ++ Map(
+"spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")
+  )
 
   /**
* Helper functions and data structures for window bounds
@@ -245,6 +249,7 @@ case class WindowInPandasExec(
 
 val allInputs = windowBoundsInput ++ dataInputs
 val allInputTypes = allInputs.map(_.dataType)
+val spillSize = longMetric("spillSize")
 
 // Start processing.
 child.execute().mapPartitions { iter =>
@@ -337,6 +342,7 @@ case class WindowInPandasExec(
   if (!found) {
 // clear final partition
 buffer.clear()
+spillSize += buffer.spillSize
   }
   found
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala
index dc85585b13d..dda5da6c9e9 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala
@@ -21,6 +21,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, 
SparkPlan}
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
 
 /**
  * This class calculates and outputs (windowed) aggregates over the rows in a 
single (sorted)
@@ -89,6 +90,9 @@ case class WindowExec(
 orderSpec: Seq[SortOrder],
 child: SparkPlan)
   extends WindowExecBase {
+  override lazy val metrics: Map[String, SQLMetric] = Map(
+"spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")
+  )
 
   protected override def doExecute(): RDD[InternalRow] = {
 // Unwrap the window expressions and window frame factories from the map.
@@ -96,6 +100,7 @@ case class WindowExec(
 val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray
 val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold
 val spillThreshold = conf.windowExecBufferSpillThreshold
+val spillSize = longMetric("spillSize")
 
 // Start processing.
 child.execute().mapPartitions { stream =>
@@ -163,6 +168,7 @@ 

[spark] branch master updated: [SPARK-41708][SQL] Pull v1write information to `WriteFiles`

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 27e20fe9eb1 [SPARK-41708][SQL] Pull v1write information to `WriteFiles`
27e20fe9eb1 is described below

commit 27e20fe9eb1b1ef1b3d32e180de55931f31fc345
Author: ulysses-you 
AuthorDate: Fri Jan 6 12:13:30 2023 +0800

[SPARK-41708][SQL] Pull v1write information to `WriteFiles`

### What changes were proposed in this pull request?

This pr aims to pull out the v1write information from `V1WriteCommand` to 
`WriteFiles`:
```scala
case class WriteFiles(child: LogicalPlan)

=>

case class WriteFiles(
child: LogicalPlan,
fileFormat: FileFormat,
partitionColumns: Seq[Attribute],
bucketSpec: Option[BucketSpec],
options: Map[String, String],
staticPartitions: TablePartitionSpec)
```

Also, this pr do a cleanup for `WriteSpec` which is unnecessary.

### Why are the changes needed?

After this pr, `WriteFiles` will hold write information that can help 
developers

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

Pass CI

Closes #39277 from ulysses-you/SPARK-41708.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../org/apache/spark/sql/internal/WriteSpec.java   |  33 ---
 .../org/apache/spark/sql/execution/SparkPlan.scala |   9 +-
 .../spark/sql/execution/SparkStrategies.scala  |   5 +-
 .../spark/sql/execution/datasources/V1Writes.scala |  24 ++-
 .../sql/execution/datasources/WriteFiles.scala |  26 ++-
 .../org/apache/spark/sql/hive/HiveStrategies.scala |   3 +-
 .../{SaveAsHiveFile.scala => HiveTempPath.scala}   | 204 ++-
 .../hive/execution/InsertIntoHiveDirCommand.scala  |  13 +-
 .../sql/hive/execution/InsertIntoHiveTable.scala   |  88 +---
 .../spark/sql/hive/execution/SaveAsHiveFile.scala  | 221 +
 .../sql/hive/execution/V1WritesHiveUtils.scala |  33 ++-
 .../org/apache/spark/sql/hive/InsertSuite.scala|  15 +-
 12 files changed, 224 insertions(+), 450 deletions(-)

diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/internal/WriteSpec.java 
b/sql/core/src/main/java/org/apache/spark/sql/internal/WriteSpec.java
deleted file mode 100644
index c51a3ed7dc6..000
--- a/sql/core/src/main/java/org/apache/spark/sql/internal/WriteSpec.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.internal;
-
-import java.io.Serializable;
-
-/**
- * Write spec is a input parameter of
- * {@link org.apache.spark.sql.execution.SparkPlan#executeWrite}.
- *
- * 
- * This is an empty interface, the concrete class which implements
- * {@link org.apache.spark.sql.execution.SparkPlan#doExecuteWrite}
- * should define its own class and use it.
- *
- * @since 3.4.0
- */
-public interface WriteSpec extends Serializable {}
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index 401302e5bde..5ca36a8a216 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -36,8 +36,9 @@ import org.apache.spark.sql.catalyst.plans.physical._
 import org.apache.spark.sql.catalyst.trees.{BinaryLike, LeafLike, TreeNodeTag, 
UnaryLike}
 import org.apache.spark.sql.connector.write.WriterCommitMessage
 import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.datasources.WriteFilesSpec
 import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.internal.{SQLConf, WriteSpec}
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.vectorized.ColumnarBatch
 import org.apache.spark.util.NextIterator
 import org.apache.spark.util.io.{ChunkedByteBuffer, 
ChunkedByteBufferOutputStream}
@@ -230,11

[spark] branch branch-3.2 updated: [SPARK-41162][SQL][3.3] Fix anti- and semi-join for self-join with aggregations

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 7eca60d4f30 [SPARK-41162][SQL][3.3] Fix anti- and semi-join for 
self-join with aggregations
7eca60d4f30 is described below

commit 7eca60d4f304d4a1a66add9fd04166d8eed1dd4f
Author: Enrico Minack 
AuthorDate: Fri Jan 6 11:32:45 2023 +0800

[SPARK-41162][SQL][3.3] Fix anti- and semi-join for self-join with 
aggregations

### What changes were proposed in this pull request?
Backport #39131 to branch-3.3.

Rule `PushDownLeftSemiAntiJoin` should not push an anti-join below an 
`Aggregate` when the join condition references an attribute that exists in its 
right plan and its left plan's child. This usually happens when the anti-join / 
semi-join is a self-join while `DeduplicateRelations` cannot deduplicate those 
attributes (in this example due to the projection of `value` to `id`).

This behaviour already exists for `Project` and `Union`, but `Aggregate` 
lacks this safety guard.

### Why are the changes needed?
Without this change, the optimizer creates an incorrect plan.

This example fails with `distinct()` (an aggregation), and succeeds without 
`distinct()`, but both queries are identical:
```scala
val ids = Seq(1, 2, 3).toDF("id").distinct()
val result = ids.withColumn("id", $"id" + 1).join(ids, Seq("id"), 
"left_anti").collect()
assert(result.length == 1)
```
With `distinct()`, rule `PushDownLeftSemiAntiJoin` creates a join condition 
`(value#907 + 1) = value#907`, which can never be true. This effectively 
removes the anti-join.

**Before this PR:**
The anti-join is fully removed from the plan.
```
== Physical Plan ==
AdaptiveSparkPlan (16)
+- == Final Plan ==
   LocalTableScan (1)

(16) AdaptiveSparkPlan
Output [1]: [id#900]
Arguments: isFinalPlan=true
```

This is caused by `PushDownLeftSemiAntiJoin` adding join condition 
`(value#907 + 1) = value#907`, which is wrong as because `id#910` in `(id#910 + 
1) AS id#912` exists in the right child of the join as well as in the left 
grandchild:
```
=== Applying Rule 
org.apache.spark.sql.catalyst.optimizer.PushDownLeftSemiAntiJoin ===
!Join LeftAnti, (id#912 = id#910)  Aggregate [id#910], 
[(id#910 + 1) AS id#912]
!:- Aggregate [id#910], [(id#910 + 1) AS id#912]   +- Project [value#907 AS 
id#910]
!:  +- Project [value#907 AS id#910]  +- Join LeftAnti, 
((value#907 + 1) = value#907)
!: +- LocalRelation [value#907]  :- LocalRelation 
[value#907]
!+- Aggregate [id#910], [id#910] +- Aggregate 
[id#910], [id#910]
!   +- Project [value#914 AS id#910]+- Project 
[value#914 AS id#910]
!  +- LocalRelation [value#914]+- 
LocalRelation [value#914]
```

The right child of the join and in the left grandchild would become the 
children of the pushed-down join, which creates an invalid join condition.

**After this PR:**
Join condition `(id#910 + 1) AS id#912` is understood to become ambiguous 
as both sides of the prospect join contain `id#910`. Hence, the join is not 
pushed down. The rule is then not applied any more.

The final plan contains the anti-join:
```
== Physical Plan ==
AdaptiveSparkPlan (24)
+- == Final Plan ==
   * BroadcastHashJoin LeftSemi BuildRight (14)
   :- * HashAggregate (7)
   :  +- AQEShuffleRead (6)
   : +- ShuffleQueryStage (5), Statistics(sizeInBytes=48.0 B, 
rowCount=3)
   :+- Exchange (4)
   :   +- * HashAggregate (3)
   :  +- * Project (2)
   : +- * LocalTableScan (1)
   +- BroadcastQueryStage (13), Statistics(sizeInBytes=1024.0 KiB, 
rowCount=3)
  +- BroadcastExchange (12)
 +- * HashAggregate (11)
+- AQEShuffleRead (10)
   +- ShuffleQueryStage (9), Statistics(sizeInBytes=48.0 B, 
rowCount=3)
  +- ReusedExchange (8)

(8) ReusedExchange [Reuses operator id: 4]
Output [1]: [id#898]

(24) AdaptiveSparkPlan
Output [1]: [id#900]
Arguments: isFinalPlan=true
```

### Does this PR introduce _any_ user-facing change?
It fixes correctness.

### How was this patch tested?
Unit tests in `DataFrameJoinSuite` and `LeftSemiAntiJoinPushDownSuite`.

Closes #39409 from EnricoMi/branch-antijoin-selfjoin-fix-3.3.

    Authored-by: Enrico Minack 
Signed-off-by: Wenchen Fan 
(cherry picked from commit b97f79da04acc9bde1cb4def7dc33c22cfc11372)
Signed-off-by: Wenchen Fan 

[spark] branch branch-3.3 updated: [SPARK-41162][SQL][3.3] Fix anti- and semi-join for self-join with aggregations

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new b97f79da04a [SPARK-41162][SQL][3.3] Fix anti- and semi-join for 
self-join with aggregations
b97f79da04a is described below

commit b97f79da04acc9bde1cb4def7dc33c22cfc11372
Author: Enrico Minack 
AuthorDate: Fri Jan 6 11:32:45 2023 +0800

[SPARK-41162][SQL][3.3] Fix anti- and semi-join for self-join with 
aggregations

### What changes were proposed in this pull request?
Backport #39131 to branch-3.3.

Rule `PushDownLeftSemiAntiJoin` should not push an anti-join below an 
`Aggregate` when the join condition references an attribute that exists in its 
right plan and its left plan's child. This usually happens when the anti-join / 
semi-join is a self-join while `DeduplicateRelations` cannot deduplicate those 
attributes (in this example due to the projection of `value` to `id`).

This behaviour already exists for `Project` and `Union`, but `Aggregate` 
lacks this safety guard.

### Why are the changes needed?
Without this change, the optimizer creates an incorrect plan.

This example fails with `distinct()` (an aggregation), and succeeds without 
`distinct()`, but both queries are identical:
```scala
val ids = Seq(1, 2, 3).toDF("id").distinct()
val result = ids.withColumn("id", $"id" + 1).join(ids, Seq("id"), 
"left_anti").collect()
assert(result.length == 1)
```
With `distinct()`, rule `PushDownLeftSemiAntiJoin` creates a join condition 
`(value#907 + 1) = value#907`, which can never be true. This effectively 
removes the anti-join.

**Before this PR:**
The anti-join is fully removed from the plan.
```
== Physical Plan ==
AdaptiveSparkPlan (16)
+- == Final Plan ==
   LocalTableScan (1)

(16) AdaptiveSparkPlan
Output [1]: [id#900]
Arguments: isFinalPlan=true
```

This is caused by `PushDownLeftSemiAntiJoin` adding join condition 
`(value#907 + 1) = value#907`, which is wrong as because `id#910` in `(id#910 + 
1) AS id#912` exists in the right child of the join as well as in the left 
grandchild:
```
=== Applying Rule 
org.apache.spark.sql.catalyst.optimizer.PushDownLeftSemiAntiJoin ===
!Join LeftAnti, (id#912 = id#910)  Aggregate [id#910], 
[(id#910 + 1) AS id#912]
!:- Aggregate [id#910], [(id#910 + 1) AS id#912]   +- Project [value#907 AS 
id#910]
!:  +- Project [value#907 AS id#910]  +- Join LeftAnti, 
((value#907 + 1) = value#907)
!: +- LocalRelation [value#907]  :- LocalRelation 
[value#907]
!+- Aggregate [id#910], [id#910] +- Aggregate 
[id#910], [id#910]
!   +- Project [value#914 AS id#910]+- Project 
[value#914 AS id#910]
!  +- LocalRelation [value#914]+- 
LocalRelation [value#914]
```

The right child of the join and in the left grandchild would become the 
children of the pushed-down join, which creates an invalid join condition.

**After this PR:**
Join condition `(id#910 + 1) AS id#912` is understood to become ambiguous 
as both sides of the prospect join contain `id#910`. Hence, the join is not 
pushed down. The rule is then not applied any more.

The final plan contains the anti-join:
```
== Physical Plan ==
AdaptiveSparkPlan (24)
+- == Final Plan ==
   * BroadcastHashJoin LeftSemi BuildRight (14)
   :- * HashAggregate (7)
   :  +- AQEShuffleRead (6)
   : +- ShuffleQueryStage (5), Statistics(sizeInBytes=48.0 B, 
rowCount=3)
   :+- Exchange (4)
   :   +- * HashAggregate (3)
   :  +- * Project (2)
   : +- * LocalTableScan (1)
   +- BroadcastQueryStage (13), Statistics(sizeInBytes=1024.0 KiB, 
rowCount=3)
  +- BroadcastExchange (12)
 +- * HashAggregate (11)
+- AQEShuffleRead (10)
   +- ShuffleQueryStage (9), Statistics(sizeInBytes=48.0 B, 
rowCount=3)
  +- ReusedExchange (8)

(8) ReusedExchange [Reuses operator id: 4]
Output [1]: [id#898]

(24) AdaptiveSparkPlan
Output [1]: [id#900]
Arguments: isFinalPlan=true
```

### Does this PR introduce _any_ user-facing change?
It fixes correctness.

### How was this patch tested?
Unit tests in `DataFrameJoinSuite` and `LeftSemiAntiJoinPushDownSuite`.

Closes #39409 from EnricoMi/branch-antijoin-selfjoin-fix-3.3.

    Authored-by: Enrico Minack 
Signed-off-by: Wenchen Fan 
---
 .../optimizer/PushDownLeftSemiAntiJoin.scala   | 13 ++---
 .../optimizer/LeftSemiAntiJoinPushDow

[spark] branch master updated: [SPARK-41912][SQL] Subquery should not validate CTE

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 89666d44a39 [SPARK-41912][SQL] Subquery should not validate CTE
89666d44a39 is described below

commit 89666d44a39c48df841a0102ff6f54eaeb4c6140
Author: Rui Wang 
AuthorDate: Fri Jan 6 11:30:48 2023 +0800

[SPARK-41912][SQL] Subquery should not validate CTE

### What changes were proposed in this pull request?

The commit https://github.com/apache/spark/pull/38029 actually intended to 
do the right thing: it checks CTE more aggressively even if a CTE is not used, 
which is ok. However, it triggers an existing issue where a subquery checks 
itself but in the CTE case if the subquery contains a CTE which is defined 
outside of the subquery, the check will fail as CTE not found (e.g. key not 
found).

So it is:

the commit checks more thus in the repro examples, every CTE is checked now 
(in the past only used CTE is checked).

One of the CTE that is checked after the commit in the example contains 
subquery.

The subquery contains another CTE which is defined outside of the subquery.

The subquery checks itself thus fail due to CTE not found.

This PR fixes the issue by removing the subquery self-validation on CTE 
case.

### Why are the changes needed?

This fixed a regression that
```
val df = sql("""
   |WITH
   |cte1 as (SELECT 1 col1),
   |cte2 as (SELECT (SELECT MAX(col1) FROM cte1))
   |SELECT * FROM cte1
   |""".stripMargin
)
checkAnswer(df, Row(1) :: Nil)
```

cannot pass analyzer anymore.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

UT

Closes #39414 from amaliujia/fix_subquery_validate.

Authored-by: Rui Wang 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala|  2 +-
 .../src/test/scala/org/apache/spark/sql/SubquerySuite.scala   | 11 +++
 2 files changed, 12 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index 8309186d566..4dc0bf98a54 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -923,7 +923,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog with QueryErrorsB
 }
 
 // Validate the subquery plan.
-checkAnalysis(expr.plan)
+checkAnalysis0(expr.plan)
 
 // Check if there is outer attribute that cannot be found from the plan.
 checkOuterReference(plan, expr)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
index 3d4a629f7a9..86a0c4d1799 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -1019,6 +1019,17 @@ class SubquerySuite extends QueryTest
 }
   }
 
+  test("SPARK-41912: Subquery does not validate CTE") {
+val df = sql("""
+   |WITH
+   |cte1 as (SELECT 1 col1),
+   |cte2 as (SELECT (SELECT MAX(col1) FROM cte1))
+   |SELECT * FROM cte1
+   |""".stripMargin
+)
+checkAnswer(df, Row(1) :: Nil)
+  }
+
   test("SPARK-21835: Join in correlated subquery should be duplicateResolved: 
case 1") {
 withTable("t1") {
   withTempPath { path =>


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (eba31a8de3f -> 729c4bff167)

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from eba31a8de3f [SPARK-41806][SQL] Use AppendData.byName for SQL INSERT 
INTO by name for DSV2
 add 729c4bff167 [SPARK-41861][SQL] Make v2 ScanBuilders' build() return 
typed scan

No new revisions were added by this update.

Summary of changes:
 .../src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala | 3 +--
 .../spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala   | 3 +--
 .../spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala | 4 ++--
 .../spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala | 3 +--
 .../spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala   | 4 ++--
 .../sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala | 4 ++--
 .../spark/sql/execution/datasources/v2/text/TextScanBuilder.scala | 3 +--
 7 files changed, 10 insertions(+), 14 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41806][SQL] Use AppendData.byName for SQL INSERT INTO by name for DSV2

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new eba31a8de3f [SPARK-41806][SQL] Use AppendData.byName for SQL INSERT 
INTO by name for DSV2
eba31a8de3f is described below

commit eba31a8de3fb79f96255a0feb58db19842c9d16d
Author: Allison Portis 
AuthorDate: Fri Jan 6 10:42:16 2023 +0800

[SPARK-41806][SQL] Use AppendData.byName for SQL INSERT INTO by name for 
DSV2

### What changes were proposed in this pull request?

Use DSv2 AppendData.byName for INSERT INTO by name instead of reordering 
and converting to AppendData.byOrdinal

### Why are the changes needed?

Currently for INSERT INTO by name we reorder the value list and convert it 
to INSERT INTO by ordinal. Since DSv2 logical nodes have the `isByName` flag we 
don't need to do this. The current approach is limiting in that

- Users must provide the full list of table columns (this limits the 
functionality for features like generated columns see 
[SPARK-41290](https://issues.apache.org/jira/browse/SPARK-41290))
- It allows ambiguous queries such as `INSERT OVERWRITE t PARTITION (c='1') 
(c) VALUES ('2')` where the user provides both the static partition column 'c' 
and the column 'c' in the column list. We should check that the static 
partition column is not in the column list. See the added test for more 
detailed example.

### Does this PR introduce _any_ user-facing change?

For versions 3.3 and below:
```sql
CREATE TABLE t(i STRING, c string) USING PARQUET PARTITIONED BY (c);
INSERT OVERWRITE t PARTITION (c='1') (c) VALUES ('2')
SELECT * FROM t
```
```
+---+---+
|  i|  c|
+---+---+
|  2|  1|
+---+---+
```
For versions 3.4 and above:
```sql
CREATE TABLE t(i STRING, c string) USING PARQUET PARTITIONED BY (c);
INSERT OVERWRITE t PARTITION (c='1') (c) VALUES ('2')
```
```
AnalysisException: [STATIC_PARTITION_COLUMN_IN_COLUMN_LIST] Static 
partition column c is also specified in the column list.
```

### How was this patch tested?

Unit tests are added.

Closes #39334 from allisonport-db/insert-into-by-name.

Authored-by: Allison Portis 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   |  5 ++
 .../spark/sql/catalyst/analysis/Analyzer.scala | 99 +++---
 .../spark/sql/errors/QueryCompilationErrors.scala  |  6 ++
 .../org/apache/spark/sql/SQLInsertTestSuite.scala  | 16 +++-
 .../spark/sql/connector/DataSourceV2SQLSuite.scala | 96 +
 .../execution/command/PlanResolutionSuite.scala| 30 ++-
 6 files changed, 239 insertions(+), 13 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 29cafdcc1b6..1d1952dce1b 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1145,6 +1145,11 @@
   "Star (*) is not allowed in a select list when GROUP BY an ordinal 
position is used."
 ]
   },
+  "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST" : {
+"message" : [
+  "Static partition column  is also specified in the column 
list."
+]
+  },
   "STREAM_FAILED" : {
 "message" : [
   "Query [id = , runId = ] terminated with exception: "
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 1ebbfb9a39a..8fff0d41add 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -1291,28 +1291,92 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 }
   }
 
+  /** Handle INSERT INTO for DSv2 */
   object ResolveInsertInto extends Rule[LogicalPlan] {
+
+/** Add a project to use the table column names for INSERT INTO BY NAME */
+private def createProjectForByNameQuery(i: InsertIntoStatement): 
LogicalPlan = {
+  SchemaUtils.checkColumnNameDuplication(i.userSpecifiedCols, resolver)
+
+  if (i.userSpecifiedCols.size != i.query.output.size) {
+throw QueryCompilationErrors.writeTableWithMismatchedColumnsError(
+  i.userSpecifiedCols.size, i.query.output.size, i.query)
+  }
+  val projectByName = i.userSpecifiedCols.zip(i.query.output)
+.map { case (userSpecifiedCol, queryOutputCol) =>
+  val resolvedCol = i.table.resolve(Seq(userSpecifiedCol), resolver)
+.getOrElse(
+  throw QueryCompilationErrors.unresolvedAttributeError(
+

[spark] branch master updated (737eecded4d -> 3dc881afcfc)

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 737eecded4d [SPARK-41162][SQL] Fix anti- and semi-join for self-join 
with aggregations
 add 3dc881afcfc [SPARK-41791] Add new file source metadata column types

No new revisions were added by this update.

Summary of changes:
 .../catalyst/expressions/namedExpressions.scala| 95 +++---
 .../org/apache/spark/sql/types/StructField.scala   |  4 +
 .../org/apache/spark/sql/types/StructType.scala|  3 +-
 .../spark/sql/execution/DataSourceScanExec.scala   | 34 
 .../sql/execution/datasources/FileFormat.scala |  9 --
 .../execution/datasources/FileSourceStrategy.scala | 68 +---
 .../datasources/PartitioningAwareFileIndex.scala   |  3 +-
 .../datasources/FileMetadataStructSuite.scala  | 10 +--
 8 files changed, 145 insertions(+), 81 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41162][SQL] Fix anti- and semi-join for self-join with aggregations

2023-01-05 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 737eecded4d [SPARK-41162][SQL] Fix anti- and semi-join for self-join 
with aggregations
737eecded4d is described below

commit 737eecded4dc2a828c978147a396f8808b09566f
Author: Enrico Minack 
AuthorDate: Thu Jan 5 18:55:11 2023 +0800

[SPARK-41162][SQL] Fix anti- and semi-join for self-join with aggregations

### What changes were proposed in this pull request?
Rule `PushDownLeftSemiAntiJoin` should not push an anti-join below an 
`Aggregate` when the join condition references an attribute that exists in its 
right plan and its left plan's child. This usually happens when the anti-join / 
semi-join is a self-join while `DeduplicateRelations` cannot deduplicate those 
attributes (in this example due to the projection of `value` to `id`).

This behaviour already exists for `Project` and `Union`, but `Aggregate` 
lacks this safety guard.

### Why are the changes needed?
Without this change, the optimizer creates an incorrect plan.

This example fails with `distinct()` (an aggregation), and succeeds without 
`distinct()`, but both queries are identical:
```scala
val ids = Seq(1, 2, 3).toDF("id").distinct()
val result = ids.withColumn("id", $"id" + 1).join(ids, Seq("id"), 
"left_anti").collect()
assert(result.length == 1)
```
With `distinct()`, rule `PushDownLeftSemiAntiJoin` creates a join condition 
`(value#907 + 1) = value#907`, which can never be true. This effectively 
removes the anti-join.

**Before this PR:**
The anti-join is fully removed from the plan.
```
== Physical Plan ==
AdaptiveSparkPlan (16)
+- == Final Plan ==
   LocalTableScan (1)

(16) AdaptiveSparkPlan
Output [1]: [id#900]
Arguments: isFinalPlan=true
```

This is caused by `PushDownLeftSemiAntiJoin` adding join condition 
`(value#907 + 1) = value#907`, which is wrong as because `id#910` in `(id#910 + 
1) AS id#912` exists in the right child of the join as well as in the left 
grandchild:
```
=== Applying Rule 
org.apache.spark.sql.catalyst.optimizer.PushDownLeftSemiAntiJoin ===
!Join LeftAnti, (id#912 = id#910)  Aggregate [id#910], 
[(id#910 + 1) AS id#912]
!:- Aggregate [id#910], [(id#910 + 1) AS id#912]   +- Project [value#907 AS 
id#910]
!:  +- Project [value#907 AS id#910]  +- Join LeftAnti, 
((value#907 + 1) = value#907)
!: +- LocalRelation [value#907]  :- LocalRelation 
[value#907]
!+- Aggregate [id#910], [id#910] +- Aggregate 
[id#910], [id#910]
!   +- Project [value#914 AS id#910]+- Project 
[value#914 AS id#910]
!  +- LocalRelation [value#914]+- 
LocalRelation [value#914]
```

The right child of the join and in the left grandchild would become the 
children of the pushed-down join, which creates an invalid join condition.

**After this PR:**
Join condition `(id#910 + 1) AS id#912` is understood to become ambiguous 
as both sides of the prospect join contain `id#910`. Hence, the join is not 
pushed down. The rule is then not applied any more.

The final plan contains the anti-join:
```
== Physical Plan ==
AdaptiveSparkPlan (24)
+- == Final Plan ==
   * BroadcastHashJoin LeftSemi BuildRight (14)
   :- * HashAggregate (7)
   :  +- AQEShuffleRead (6)
   : +- ShuffleQueryStage (5), Statistics(sizeInBytes=48.0 B, 
rowCount=3)
   :+- Exchange (4)
   :   +- * HashAggregate (3)
   :  +- * Project (2)
   : +- * LocalTableScan (1)
   +- BroadcastQueryStage (13), Statistics(sizeInBytes=1024.0 KiB, 
rowCount=3)
  +- BroadcastExchange (12)
 +- * HashAggregate (11)
+- AQEShuffleRead (10)
   +- ShuffleQueryStage (9), Statistics(sizeInBytes=48.0 B, 
rowCount=3)
  +- ReusedExchange (8)

(8) ReusedExchange [Reuses operator id: 4]
Output [1]: [id#898]

(24) AdaptiveSparkPlan
Output [1]: [id#900]
Arguments: isFinalPlan=true
```

### Does this PR introduce _any_ user-facing change?
It fixes correctness.

### How was this patch tested?
Unit tests in `DataFrameJoinSuite` and `LeftSemiAntiJoinPushDownSuite`.

Closes #39131 from EnricoMi/branch-antijoin-selfjoin-fix.

    Authored-by: Enrico Minack 
Signed-off-by: Wenchen Fan 
---
 .../optimizer/PushDownLeftSemiAntiJoin.scala   | 13 ++---
 .../optimizer/LeftSemiAntiJoinPushDownSuite.scala  | 57 ++
 .../org/apache/spark/s

[spark] branch master updated (0b786901633 -> 3130ca9748b)

2023-01-03 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 0b786901633 [SPARK-41850][CONNECT][PYTHON][TESTS] Enable doctest for 
`isnan`
 add 3130ca9748b [SPARK-41859][SQL] CreateHiveTableAsSelectCommand should 
set the overwrite flag correctly

No new revisions were added by this update.

Summary of changes:
 .../sql/hive/execution/CreateHiveTableAsSelectCommand.scala  | 12 +---
 1 file changed, 5 insertions(+), 7 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (f0d9692c5d2 -> 3c40be2dddc)

2023-01-03 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from f0d9692c5d2 
[SPARK-41855][SPARK-41814][SPARK-41851][SPARK-41852][CONNECT][PYTHON] Make 
`createDataFrame` handle None/NaN properly
 add 3c40be2dddc [SPARK-41405][SQL] Centralize the column resolution logic

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/Analyzer.scala | 797 ++---
 .../ResolveLateralColumnAliasReference.scala   |  24 +-
 .../spark/sql/catalyst/analysis/unresolved.scala   |  19 +-
 .../catalyst/expressions/namedExpressions.scala|  23 +-
 .../spark/sql/catalyst/expressions/subquery.scala  |   9 +-
 .../sql/catalyst/rules/RuleIdCollection.scala  |   1 -
 .../spark/sql/catalyst/trees/TreePatterns.scala|   1 +
 .../apache/spark/sql/LateralColumnAliasSuite.scala |   3 +-
 8 files changed, 424 insertions(+), 453 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41049][SQL] Revisit stateful expression handling

2022-12-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 4321604fe0b [SPARK-41049][SQL] Revisit stateful expression handling
4321604fe0b is described below

commit 4321604fe0b8f54b7cc5d1372f9b89a91fbad8b2
Author: Wenchen Fan 
AuthorDate: Sat Dec 31 10:05:41 2022 +0800

[SPARK-41049][SQL] Revisit stateful expression handling

### What changes were proposed in this pull request?

Spark has a `Stateful` trait for stateful expressions. The basic idea is to 
have fresh copies of stateful expressions before evaluating them. This is to 
avoid issues caused by the flexible DataFrame APIs:
1. A single expression instance may appear more than once in the expression 
tree. We have to replace it with fresh copies to avoid sharing states.
2. An expression tree can be evaluated by multiple callers at the same 
time. We have to use fresh copies before expression evaluation to avoid sharing 
states.

However, the handling of stateful expression has several problems. This PR 
fixes all of them:
1. We should use fresh copies with codegen as well. If the root expression 
extends `CodegenFallback`, then the expression tree will be evaluated using the 
interpreted mode, even with the codegen code path.
2. The fresh copies will be dropped if the stateful expression is deeply 
nested (3 layers).
3. `InterpretedSafeProjection` never implemented initialize() for 
initializing Nondeterministic expressions.
4. `ConvertToLocalRelation` called a `InterpretedMutableProjection` 
constructor which did not implement the existing Stateful-copying logic. I 
fixed this by moving that logic out of a factory method and into class's main 
constructor, guaranteeing that it will always run.
5. Stateful expression is not always nondeterministic, e.g. `ScalaUDF`. I 
removed the `Stateful` trait and added a `def stateful: Boolean` function in 
`Expression`.

### Why are the changes needed?

Fix stateful expression handling

### Does this PR introduce _any_ user-facing change?

Yes, now we never share states for stateful expressions, which may produce 
wrong result.

### How was this patch tested?

new tests

Closes #39248 from cloud-fan/expr.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 .../sql/catalyst/expressions/Expression.scala  | 79 ++
 .../expressions/ExpressionsEvaluator.scala | 45 
 .../expressions/InterpretedMutableProjection.scala | 16 +
 .../expressions/InterpretedSafeProjection.scala| 21 +++---
 .../expressions/InterpretedUnsafeProjection.scala  | 14 +---
 .../expressions/MonotonicallyIncreasingID.scala| 10 ++-
 .../sql/catalyst/expressions/Projection.scala  | 12 ++--
 .../spark/sql/catalyst/expressions/ScalaUDF.scala  |  5 ++
 .../expressions/codegen/CodeGenerator.scala|  7 +-
 .../expressions/collectionOperations.scala |  8 +--
 .../spark/sql/catalyst/expressions/misc.scala  |  6 +-
 .../spark/sql/catalyst/expressions/package.scala   | 10 +--
 .../sql/catalyst/expressions/predicates.scala  | 17 +
 .../catalyst/expressions/randomExpressions.scala   |  8 +--
 .../apache/spark/sql/catalyst/trees/TreeNode.scala |  3 +-
 .../expressions/CollectionExpressionsSuite.scala   |  6 --
 .../expressions/MiscExpressionsSuite.scala |  6 --
 .../spark/sql/catalyst/trees/TreeNodeSuite.scala   | 44 
 .../org/apache/spark/sql/DataFrameSuite.scala  | 11 +++
 19 files changed, 203 insertions(+), 125 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 0ddf1a7df19..de0e90285f5 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -26,7 +26,7 @@ import 
org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
 import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.expressions.codegen.Block._
-import org.apache.spark.sql.catalyst.trees.{BinaryLike, LeafLike, 
QuaternaryLike, SQLQueryContext, TernaryLike, TreeNode, UnaryLike}
+import org.apache.spark.sql.catalyst.trees.{BinaryLike, CurrentOrigin, 
LeafLike, QuaternaryLike, SQLQueryContext, TernaryLike, TreeNode, UnaryLike}
 import org.apache.spark.sql.catalyst.trees.TreePattern.{RUNTIME_REPLACEABLE, 
TreePattern}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.errors.{QueryErrorsBase

[spark] branch master updated: [SPARK-41440][CONNECT][PYTHON] Avoid the cache operator for general Sample

2022-12-30 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7143cbd0725 [SPARK-41440][CONNECT][PYTHON] Avoid the cache operator 
for general Sample
7143cbd0725 is described below

commit 7143cbd072557c8ea231b378572e8a7554d8a3f5
Author: Jiaan Geng 
AuthorDate: Fri Dec 30 20:22:21 2022 +0800

[SPARK-41440][CONNECT][PYTHON] Avoid the cache operator for general Sample

### What changes were proposed in this pull request?
https://github.com/apache/spark/pull/39017 supported 
`DataFrame.randomSplit`. But cache the Sample plan incorrectly.

### Why are the changes needed?
This PR avoid the cache operator for general `Sample`.
This PR also give a more suitable name `deterministic_order` to replace 
`force_stable_sort`.

### Does this PR introduce _any_ user-facing change?
'No'.
New feature.

### How was this patch tested?
Tests updated.

Closes #39240 from beliefer/SPARK-41440_followup2.

Authored-by: Jiaan Geng 
Signed-off-by: Wenchen Fan 
---
 .../main/protobuf/spark/connect/relations.proto|  7 +-
 .../org/apache/spark/sql/connect/dsl/package.scala |  2 +-
 .../sql/connect/planner/SparkConnectPlanner.scala  |  9 ++-
 python/pyspark/sql/connect/dataframe.py|  2 +-
 python/pyspark/sql/connect/plan.py |  6 +-
 python/pyspark/sql/connect/proto/relations_pb2.py  | 88 +++---
 python/pyspark/sql/connect/proto/relations_pb2.pyi | 25 ++
 .../sql/tests/connect/test_connect_plan_only.py| 10 +--
 8 files changed, 71 insertions(+), 78 deletions(-)

diff --git 
a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto 
b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto
index afff04f8f0d..3bb0b362b27 100644
--- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto
+++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto
@@ -358,9 +358,10 @@ message Sample {
   // (Optional) The random seed.
   optional int64 seed = 5;
 
-  // (Optional) Explicitly sort the underlying plan to make the ordering 
deterministic.
-  // This flag is only used to randomly splits DataFrame with the provided 
weights.
-  optional bool force_stable_sort = 6;
+  // (Required) Explicitly sort the underlying plan to make the ordering 
deterministic or cache it.
+  // This flag is true when invoking `dataframe.randomSplit` to randomly 
splits DataFrame with the
+  // provided weights. Otherwise, it is false.
+  bool deterministic_order = 6;
 }
 
 // Relation of type [[Range]] that generates a sequence of integers.
diff --git 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
index 3bd713a9710..c4a5eac46c0 100644
--- 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
+++ 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala
@@ -964,7 +964,7 @@ package object dsl {
   .setUpperBound(x(1))
   .setWithReplacement(false)
   .setSeed(seed)
-  .setForceStableSort(true)
+  .setDeterministicOrder(true)
   .build())
   .build()
   }
diff --git 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
index 7d6fdc2883e..a11ebd8b7d1 100644
--- 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
+++ 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
@@ -211,8 +211,9 @@ class SparkConnectPlanner(session: SparkSession) {
* wrap such fields into proto messages.
*/
   private def transformSample(rel: proto.Sample): LogicalPlan = {
-val input = Dataset.ofRows(session, transformRelation(rel.getInput))
-val plan = if (rel.getForceStableSort) {
+val plan = if (rel.getDeterministicOrder) {
+  val input = Dataset.ofRows(session, transformRelation(rel.getInput))
+
   // It is possible that the underlying dataframe doesn't guarantee the 
ordering of rows in its
   // constituent partitions each time a split is materialized which could 
result in
   // overlapping splits. To prevent this, we explicitly sort each input 
partition to make the
@@ -224,11 +225,11 @@ class SparkConnectPlanner(session: SparkSession) {
   if (sortOrder.nonEmpty) {
 Sort(sortOrder, global = false, input.logicalPlan)
   } else {
+input.cache()
 input.logicalPlan

[spark] branch master updated: [SPARK-41554] fix changing of Decimal scale when scale decreased by m…

2022-12-29 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 838954e5080 [SPARK-41554] fix changing of Decimal scale when scale 
decreased by m…
838954e5080 is described below

commit 838954e50807e583ceb8317877710d58acff0a4b
Author: oleksii.diagiliev 
AuthorDate: Fri Dec 30 15:52:05 2022 +0800

[SPARK-41554] fix changing of Decimal scale when scale decreased by m…

…ore than 18

### What changes were proposed in this pull request?
Fix `Decimal` scaling that is stored as compact long internally when scale 
decreased by more than 18. For example,
```
Decimal(1, 38, 19).changePrecision(38, 0)
```
produces an exception
```
java.lang.ArrayIndexOutOfBoundsException: 19
at org.apache.spark.sql.types.Decimal.changePrecision(Decimal.scala:377)
at org.apache.spark.sql.types.Decimal.changePrecision(Decimal.scala:328)
```
Another way to reproduce it with SQL query
```
sql("select cast(cast(cast(cast(id as decimal(38,15)) as decimal(38,30)) as 
decimal(38,37)) as decimal(38,17)) from range(3)").show
```

The bug exists for Decimal that is stored using compact long only, it works 
fine with Decimal that uses `scala.math.BigDecimal` internally.

### Why are the changes needed?
Not able to execute the SQL query mentioned above. Please note, for my use 
case the SQL query is generated programatically, so I cannot optimize it 
manually.

### Does this PR introduce _any_ user-facing change?

Yes, it will allow scale Decimal properly that is not currently possible 
due to the exception.

### How was this patch tested?
Tests were added. The fix affects the scale decrease only, but I decided to 
also include tests for scale increase as I didn't find them.

Closes #39099 from fe2s/SPARK-41554-fix-decimal-scaling.

Authored-by: oleksii.diagiliev 
Signed-off-by: Wenchen Fan 
---
 .../scala/org/apache/spark/sql/types/Decimal.scala | 60 +-
 .../org/apache/spark/sql/types/DecimalSuite.scala  | 52 ++-
 2 files changed, 87 insertions(+), 25 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
index 44c00df379f..2c0b6677541 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
@@ -374,30 +374,42 @@ final class Decimal extends Ordered[Decimal] with 
Serializable {
   if (scale < _scale) {
 // Easier case: we just need to divide our scale down
 val diff = _scale - scale
-val pow10diff = POW_10(diff)
-// % and / always round to 0
-val droppedDigits = lv % pow10diff
-lv /= pow10diff
-roundMode match {
-  case ROUND_FLOOR =>
-if (droppedDigits < 0) {
-  lv += -1L
-}
-  case ROUND_CEILING =>
-if (droppedDigits > 0) {
-  lv += 1L
-}
-  case ROUND_HALF_UP =>
-if (math.abs(droppedDigits) * 2 >= pow10diff) {
-  lv += (if (droppedDigits < 0) -1L else 1L)
-}
-  case ROUND_HALF_EVEN =>
-val doubled = math.abs(droppedDigits) * 2
-if (doubled > pow10diff || doubled == pow10diff && lv % 2 != 0) {
-  lv += (if (droppedDigits < 0) -1L else 1L)
-}
-  case _ =>
-throw QueryExecutionErrors.unsupportedRoundingMode(roundMode)
+// If diff is greater than max number of digits we store in Long, then
+// value becomes 0. Otherwise we calculate new value dividing by power 
of 10.
+// In both cases we apply rounding after that.
+if (diff > MAX_LONG_DIGITS) {
+  lv = roundMode match {
+case ROUND_FLOOR => if (lv < 0) -1L else 0L
+case ROUND_CEILING => if (lv > 0) 1L else 0L
+case ROUND_HALF_UP | ROUND_HALF_EVEN => 0L
+case _ => throw 
QueryExecutionErrors.unsupportedRoundingMode(roundMode)
+  }
+} else {
+  val pow10diff = POW_10(diff)
+  // % and / always round to 0
+  val droppedDigits = lv % pow10diff
+  lv /= pow10diff
+  roundMode match {
+case ROUND_FLOOR =>
+  if (droppedDigits < 0) {
+lv += -1L
+  }
+case ROUND_CEILING =>
+  if (droppedDigits > 0) {
+lv += 1L
+  }
+case ROUND_HALF_UP =>
+  if (math.abs(droppedDigits) * 2 >= pow10diff) {
+

[spark] branch master updated: [SPARK-41760][BUILD][CONNECT] Enforce scalafmt for Connect Client module

2022-12-29 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new af8442ee903 [SPARK-41760][BUILD][CONNECT] Enforce scalafmt for Connect 
Client module
af8442ee903 is described below

commit af8442ee9036bcf5b864d863d7a918b8fe9dcafd
Author: dengziming 
AuthorDate: Fri Dec 30 12:08:03 2022 +0800

[SPARK-41760][BUILD][CONNECT] Enforce scalafmt for Connect Client module

### What changes were proposed in this pull request?
1. This changes enables enforcing `scalafmt` for the Connect client module 
since it's a new module.
2. This change applies `scalafmt` on the existing code-base.

### Why are the changes needed?
Faster, focussed code reviews.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Faster, focussed code reviews.

Closes #39274 from dengziming/SPARK-41760.

Authored-by: dengziming 
Signed-off-by: Wenchen Fan 
---
 .../scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala | 3 ++-
 dev/lint-scala | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
index beaae6412be..e188ef0d409 100644
--- 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala
@@ -23,7 +23,8 @@ class SparkConnectClient(private val userContext: 
proto.UserContext) {
 
   /**
* Placeholder method.
-   * @return User ID.
+   * @return
+   *   User ID.
*/
   def userId: String = userContext.getUserId()
 }
diff --git a/dev/lint-scala b/dev/lint-scala
index 48ecf57ef47..2549f775e49 100755
--- a/dev/lint-scala
+++ b/dev/lint-scala
@@ -30,13 +30,14 @@ ERRORS=$(./build/mvn \
 -Dscalafmt.validateOnly=true \
 -Dscalafmt.changedOnly=false \
 -pl connector/connect/server \
+-pl connector/connect/client/jvm \
 2>&1 | grep -e "^Requires formatting" \
 )
 
 if test ! -z "$ERRORS"; then
   echo -e "The scalafmt check failed on connector/connect at following 
occurrences:\n\n$ERRORS\n"
   echo "Before submitting your change, please make sure to format your code 
using the following command:"
-  echo "./build/mvn -Pscala-2.12 scalafmt:format -Dscalafmt.skip=false 
-Dscalafmt.validateOnly=false -Dscalafmt.changedOnly=false -pl 
connector/connect/server"
+  echo "./build/mvn -Pscala-2.12 scalafmt:format -Dscalafmt.skip=false 
-Dscalafmt.validateOnly=false -Dscalafmt.changedOnly=false -pl 
connector/connect/server -pl connector/connect/client/jvm"
   exit 1
 else
   echo -e "Scalafmt checks passed."


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41631][FOLLOWUP][SQL] Fix two issues in implicit lateral column alias resolution on Aggregate

2022-12-28 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 281378468cc [SPARK-41631][FOLLOWUP][SQL] Fix two issues in implicit 
lateral column alias resolution on Aggregate
281378468cc is described below

commit 281378468cc9f4e3ab25c87ad10094ad347fc20a
Author: Xinyi Yu 
AuthorDate: Thu Dec 29 15:42:42 2022 +0800

[SPARK-41631][FOLLOWUP][SQL] Fix two issues in implicit lateral column 
alias resolution on Aggregate

### What changes were proposed in this pull request?

Fix two issues in implicit lateral column alias resolution of Aggregate; 
added related test cases.
* The check condition whether to lift up in Aggregate is incorrect. Leaf 
expressions, e.g. `now()` will fail the check and won't be lift up. Changed to 
follow the code pattern in checkAnalysis.
* Another condition that requires the new Aggregate expressions to be 
non-empty to lift up in Aggregate is unnecessary. Think about the Aggregate on 
expressions without any grouping expressions or aggregate expressions, e.g. 
`select 1 as a, a + 1 .. group by ..`.

### Why are the changes needed?

Fix the bug mentioned above.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #39269 from anchovYu/SPARK-41631-bug-fix.

Authored-by: Xinyi Yu 
Signed-off-by: Wenchen Fan 
---
 .../ResolveLateralColumnAliasReference.scala   | 23 +++---
 .../apache/spark/sql/LateralColumnAliasSuite.scala | 28 +-
 2 files changed, 36 insertions(+), 15 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
index ec8bdb97fbc..2fad1faec3f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, 
Expression, LateralColumnAliasReference, LeafExpression, Literal, 
NamedExpression, ScalarSubquery}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap, Expression, LateralColumnAliasReference, NamedExpression, 
ScalarSubquery}
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, 
Project}
 import org.apache.spark.sql.catalyst.rules.Rule
@@ -168,8 +168,8 @@ object ResolveLateralColumnAliasReference extends 
Rule[LogicalPlan] {
 && 
aggregateExpressions.exists(_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)) 
=>
 
   // Check if current Aggregate is eligible to lift up with Project: 
the aggregate
-  // expression only contains: 1) aggregate functions, 2) grouping 
expressions, 3) lateral
-  // column alias reference or 4) literals.
+  // expression only contains: 1) aggregate functions, 2) grouping 
expressions, 3) leaf
+  // expressions excluding attributes not in grouping expressions
   // This check is to prevent unnecessary transformation on invalid 
plan, to guarantee it
   // throws the same exception. For example, cases like non-aggregate 
expressions not
   // in group by, once transformed, will throw a different exception: 
missing input.
@@ -177,10 +177,9 @@ object ResolveLateralColumnAliasReference extends 
Rule[LogicalPlan] {
 exp match {
   case e if AggregateExpression.isAggregate(e) => true
   case e if groupingExpressions.exists(_.semanticEquals(e)) => true
-  case _: Literal | _: LateralColumnAliasReference => true
+  case a: Attribute => false
   case s: ScalarSubquery if s.children.nonEmpty
-  && !groupingExpressions.exists(_.semanticEquals(s)) => false
-  case _: LeafExpression => false
+&& !groupingExpressions.exists(_.semanticEquals(s)) => false
   case e => e.children.forall(eligibleToLiftUp)
 }
   }
@@ -210,14 +209,10 @@ object ResolveLateralColumnAliasReference extends 
Rule[LogicalPlan] {
 ne.toAttribute
 }.asInstanceOf[NamedExpression]
   }
-  if (newAggExprs.isEmpty) {
-agg
-  } else {
-Project(
-  projectList = projectExprs,
-  child 

[spark] branch master updated (93fc677d282 -> 9d5357df13b)

2022-12-28 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 93fc677d282 
[SPARK-41747][SPARK-41744][SPARK-41748][SPARK-41749][CONNECT][TESTS] Reeanble 
tests for multiple arguments in max, min, sum and avg in groupby
 add 9d5357df13b [SPARK-41753][SQL][TEST] Add tests for ArrayZip to check 
the result size and nullability

No new revisions were added by this update.

Summary of changes:
 .../src/test/resources/sql-tests/inputs/array.sql  | 10 +
 .../resources/sql-tests/results/ansi/array.sql.out | 48 ++
 .../test/resources/sql-tests/results/array.sql.out | 48 ++
 3 files changed, 106 insertions(+)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41713][SQL] Make CTAS hold a nested execution for data writing

2022-12-28 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 4b40920e331 [SPARK-41713][SQL] Make CTAS hold a nested execution for 
data writing
4b40920e331 is described below

commit 4b40920e33176fc8b18380703e4dcf4d16824094
Author: ulysses-you 
AuthorDate: Wed Dec 28 17:11:59 2022 +0800

[SPARK-41713][SQL] Make CTAS hold a nested execution for data writing

### What changes were proposed in this pull request?

This pr aims to make ctas use a nested execution instead of running data 
writing cmmand.

So, we can clean up ctas itself to remove the unnecessary v1write 
information. Now, the v1writes only have two implementation: 
`InsertIntoHadoopFsRelationCommand` and `InsertIntoHiveTable`

### Why are the changes needed?

Make v1writes code clear.

```sql
EXPLAIN FORMATTED CREATE TABLE t2 USING PARQUET AS SELECT * FROM t;

== Physical Plan ==
Execute CreateDataSourceTableAsSelectCommand (1)
   +- CreateDataSourceTableAsSelectCommand (2)
 +- Project (5)
+- SubqueryAlias (4)
   +- LogicalRelation (3)

(1) Execute CreateDataSourceTableAsSelectCommand
Output: []

(2) CreateDataSourceTableAsSelectCommand
Arguments: `spark_catalog`.`default`.`t2`, ErrorIfExists, [c1, c2]

(3) LogicalRelation
Arguments: parquet, [c1#11, c2#12], `spark_catalog`.`default`.`t`, 
org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe, false

(4) SubqueryAlias
Arguments: spark_catalog.default.t

(5) Project
Arguments: [c1#11, c2#12]
```

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

improve existed test

Closes #39220 from ulysses-you/SPARK-41713.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../execution/command/createDataSourceTables.scala | 40 +++
 .../sql/execution/datasources/DataSource.scala | 48 ++
 .../spark/sql/execution/datasources/V1Writes.scala |  8 +--
 .../scala/org/apache/spark/sql/ExplainSuite.scala  |  7 ++-
 .../adaptive/AdaptiveQueryExecSuite.scala  | 58 ++
 .../datasources/V1WriteCommandSuite.scala  | 17 +++
 .../sql/execution/metric/SQLMetricsSuite.scala | 41 ++-
 .../spark/sql/util/DataFrameCallbackSuite.scala| 12 +++--
 .../execution/CreateHiveTableAsSelectCommand.scala | 46 +
 .../sql/hive/execution/HiveExplainSuite.scala  | 16 ++
 .../spark/sql/hive/execution/SQLMetricsSuite.scala | 49 +++---
 11 files changed, 159 insertions(+), 183 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
index 9bf9f43829e..bf14ef14cf4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
@@ -20,13 +20,11 @@ package org.apache.spark.sql.execution.command
 import java.net.URI
 
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.util.CharVarcharUtils
 import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.execution.{CommandExecutionMode, SparkPlan}
+import org.apache.spark.sql.execution.CommandExecutionMode
 import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.sources.BaseRelation
 import org.apache.spark.sql.types.StructType
@@ -143,29 +141,11 @@ case class CreateDataSourceTableAsSelectCommand(
 mode: SaveMode,
 query: LogicalPlan,
 outputColumnNames: Seq[String])
-  extends V1WriteCommand {
-
-  override def fileFormatProvider: Boolean = {
-table.provider.forall { provider =>
-  classOf[FileFormat].isAssignableFrom(DataSource.providingClass(provider, 
conf))
-}
-  }
-
-  override lazy val partitionColumns: Seq[Attribute] = {
-val unresolvedPartitionColumns = 
table.partitionColumnNames.map(UnresolvedAttribute.quoted)
-DataSource.resolvePartitionColumns(
-  unresolvedPartitionColumns,
-  outputColumns,
-  query,
-  SparkSession.active.sessionState.conf.resolver)
-  }
-
-  override def requiredOrdering: Seq[SortOrder] = {
-val options = table.storage.properties
-V1WritesUtils.getSortOrder(outputColumns, partitionColumns, 
table.bucketS

[spark] branch branch-3.3 updated: Revert "[MINOR][TEST][SQL] Add a CTE subquery scope test case"

2022-12-28 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 0887a2f09fc Revert "[MINOR][TEST][SQL] Add a CTE subquery scope test 
case"
0887a2f09fc is described below

commit 0887a2f09fcd7dc0ddfb4b28f6fc9a4dcae2d93c
Author: Wenchen Fan 
AuthorDate: Wed Dec 28 16:57:18 2022 +0800

Revert "[MINOR][TEST][SQL] Add a CTE subquery scope test case"

This reverts commit aa39b06462a98f37be59e239d12edd9f09a25b88.
---
 .../test/resources/sql-tests/inputs/cte-nested.sql | 10 
 .../resources/sql-tests/results/cte-legacy.sql.out | 28 --
 .../resources/sql-tests/results/cte-nested.sql.out | 28 --
 .../sql-tests/results/cte-nonlegacy.sql.out| 28 --
 4 files changed, 94 deletions(-)

diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql 
b/sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql
index e5ef2443417..5f12388b9cb 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql
@@ -17,16 +17,6 @@ SELECT (
   SELECT * FROM t
 );
 
--- Make sure CTE in subquery is scoped to that subquery rather than global
--- the 2nd half of the union should fail because the cte is scoped to the 
first half
-SELECT * FROM
-  (
-   WITH cte AS (SELECT * FROM range(10))
-   SELECT * FROM cte WHERE id = 8
-  ) a
-UNION
-SELECT * FROM cte;
-
 -- CTE in CTE definition shadows outer
 WITH
   t AS (SELECT 1),
diff --git a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out 
b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out
index ebdd64c3ac8..264b64ffe96 100644
--- a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out
@@ -36,34 +36,6 @@ struct
 1
 
 
--- !query
-SELECT * FROM
-  (
-   WITH cte AS (SELECT * FROM range(10))
-   SELECT * FROM cte WHERE id = 8
-  ) a
-UNION
-SELECT * FROM cte
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-{
-  "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
-  "sqlState" : "42000",
-  "messageParameters" : {
-"relationName" : "`cte`"
-  },
-  "queryContext" : [ {
-"objectType" : "",
-"objectName" : "",
-"startIndex" : 120,
-"stopIndex" : 122,
-"fragment" : "cte"
-  } ]
-}
-
-
 -- !query
 WITH
   t AS (SELECT 1),
diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out 
b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out
index b6e1793f7d7..2c622de3f36 100644
--- a/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out
@@ -36,34 +36,6 @@ struct
 1
 
 
--- !query
-SELECT * FROM
-  (
-   WITH cte AS (SELECT * FROM range(10))
-   SELECT * FROM cte WHERE id = 8
-  ) a
-UNION
-SELECT * FROM cte
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-{
-  "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
-  "sqlState" : "42000",
-  "messageParameters" : {
-"relationName" : "`cte`"
-  },
-  "queryContext" : [ {
-"objectType" : "",
-"objectName" : "",
-"startIndex" : 120,
-"stopIndex" : 122,
-"fragment" : "cte"
-  } ]
-}
-
-
 -- !query
 WITH
   t AS (SELECT 1),
diff --git 
a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out 
b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out
index 546ab7ecb95..283f5a54a42 100644
--- a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out
@@ -36,34 +36,6 @@ struct
 1
 
 
--- !query
-SELECT * FROM
-  (
-   WITH cte AS (SELECT * FROM range(10))
-   SELECT * FROM cte WHERE id = 8
-  ) a
-UNION
-SELECT * FROM cte
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-{
-  "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
-  "sqlState" : "42000",
-  "messageParameters" : {
-"relationName" : "`cte`"
-  },
-  "queryContext" : [ {
-"objectType" : "",
-"objectName" : "",
-"startIndex" : 120,
-"stopIndex" : 122,
-"fragment" : "cte"
-  } ]
-}
-
-
 -- !query
 WITH
   t AS (SELECT 1),


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (030c1baf2bd -> a1767bdcfaa)

2022-12-26 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 030c1baf2bd [SPARK-41690][SQL][CONNECT] Agnostic Encoders
 add a1767bdcfaa [SPARK-41235][SQL][PYTHON] High-order function: 
array_compact implementation

No new revisions were added by this update.

Summary of changes:
 .../source/reference/pyspark.sql/functions.rst |  1 +
 python/pyspark/sql/functions.py| 26 +
 .../sql/catalyst/analysis/FunctionRegistry.scala   |  1 +
 .../expressions/collectionOperations.scala | 31 +++
 .../scala/org/apache/spark/sql/functions.scala | 10 
 .../sql-functions/sql-expression-schema.md |  1 +
 .../src/test/resources/sql-tests/inputs/array.sql  |  8 +++
 .../resources/sql-tests/results/ansi/array.sql.out | 65 ++
 .../test/resources/sql-tests/results/array.sql.out | 65 ++
 .../apache/spark/sql/DataFrameFunctionsSuite.scala | 44 +++
 10 files changed, 252 insertions(+)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (6cf42fe4e04 -> 030c1baf2bd)

2022-12-26 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 6cf42fe4e04 [SPARK-41717][CONNECT] Deduplicate print and _repr_html_ 
at LogicalPlan
 add 030c1baf2bd [SPARK-41690][SQL][CONNECT] Agnostic Encoders

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/DeserializerBuildHelper.scala |2 +-
 .../spark/sql/catalyst/ScalaReflection.scala   | 1049 
 .../spark/sql/catalyst/SerializerBuildHelper.scala |9 +
 .../sql/catalyst/encoders/AgnosticEncoder.scala|  154 +++
 .../sql/catalyst/encoders/ExpressionEncoder.scala  |   16 +-
 .../spark/sql/catalyst/ScalaReflectionSuite.scala  |   12 +-
 .../catalyst/encoders/EncoderResolutionSuite.scala |8 +-
 .../expressions/ObjectExpressionsSuite.scala   |   28 +-
 .../sql/errors/QueryCompilationErrorsSuite.scala   |4 +-
 9 files changed, 637 insertions(+), 645 deletions(-)
 create mode 100644 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/AgnosticEncoder.scala


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [MINOR][SQL] Document AnalysisOnlyCommand

2022-12-25 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 9bebf4b36cd [MINOR][SQL] Document AnalysisOnlyCommand
9bebf4b36cd is described below

commit 9bebf4b36cdfc85e2ae53c448ea9fc1bf1f0f843
Author: Reynold Xin 
AuthorDate: Mon Dec 26 13:31:12 2022 +0800

[MINOR][SQL] Document AnalysisOnlyCommand

### What changes were proposed in this pull request?
This patch adds more documentation to AnalysisOnlyCommand, since it's not 
obvious why it's needed or how it works.

### Why are the changes needed?
See above.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
N/A

Closes #39199 from rxin/AnalysisOnlyCommand_comment.

Authored-by: Reynold Xin 
Signed-off-by: Wenchen Fan 
---
 .../org/apache/spark/sql/catalyst/plans/logical/Command.scala  | 10 +-
 1 file changed, 9 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
index 8e23c2ea0e2..fc9eb5d03e4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
@@ -44,7 +44,15 @@ trait BinaryCommand extends Command with 
BinaryLike[LogicalPlan]
 
 /**
  * A logical node that can be used for a command that requires its children to 
be only analyzed,
- * but not optimized.
+ * but not optimized. An example would be "create view": we don't need to 
optimize the view subtree
+ * because we will just store the entire view text as is in the catalog.
+ *
+ * The way we do this is by setting the children to empty once the subtree is 
analyzed. This will
+ * prevent the optimizer (or the analyzer from that point on) from traversing 
into the children.
+ *
+ * There's a corresponding rule
+ * [[org.apache.spark.sql.catalyst.analysis.Analyzer.HandleSpecialCommand]] 
that marks these
+ * commands analyzed.
  */
 trait AnalysisOnlyCommand extends Command {
   val isAnalyzed: Boolean


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (c526741ed09 -> 827ca9b8247)

2022-12-23 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from c526741ed09 [MINOR][DOCS] Fix grammatical error in streaming 
programming guide
 add 827ca9b8247 [SPARK-41498] Propagate metadata through Union

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/Analyzer.scala |  20 +-
 .../plans/logical/basicLogicalOperators.scala  |  58 --
 .../sql/connector/catalog/InMemoryBaseTable.scala  |   2 +-
 .../spark/sql/connector/MetadataColumnSuite.scala  | 231 +
 4 files changed, 291 insertions(+), 20 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (a1c727f3867 -> 2ffa8178df1)

2022-12-23 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from a1c727f3867 [SPARK-41666][PYTHON] Support parameterized SQL by `sql()`
 add 2ffa8178df1 [SPARK-41407][SQL] Pull out v1 write to WriteFiles

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/error-classes.json   |   2 +-
 .../spark/sql/errors/QueryExecutionErrors.scala|   2 +-
 .../org/apache/spark/sql/internal/WriteSpec.java}  |  19 ++-
 .../org/apache/spark/sql/execution/SparkPlan.scala |  26 +++-
 .../spark/sql/execution/SparkStrategies.scala  |   3 +
 .../execution/command/createDataSourceTables.scala |   6 +
 .../sql/execution/datasources/DataSource.scala |  29 ++--
 .../execution/datasources/FileFormatWriter.scala   | 154 -
 .../spark/sql/execution/datasources/V1Writes.scala |  21 ++-
 .../sql/execution/datasources/WriteFiles.scala | 104 ++
 .../datasources/V1WriteCommandSuite.scala  |   7 +-
 11 files changed, 310 insertions(+), 63 deletions(-)
 copy 
sql/{catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java
 => core/src/main/java/org/apache/spark/sql/internal/WriteSpec.java} (68%)
 create mode 100644 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteFiles.scala


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41637][SQL] ORDER BY ALL

2022-12-22 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6d77aa54d04 [SPARK-41637][SQL] ORDER BY ALL
6d77aa54d04 is described below

commit 6d77aa54d04a8f8ee7533b146bc24f5fa0518f7d
Author: Reynold Xin 
AuthorDate: Fri Dec 23 12:52:36 2022 +0800

[SPARK-41637][SQL] ORDER BY ALL

### What changes were proposed in this pull request?
This patch adds ORDER BY ALL support to SQL. ORDER BY ALL is a syntactic 
sugar to sort the output by all the fields, from left to right. It also allows 
specifying asc/desc as well as null ordering.

### Why are the changes needed?
It's a good convenience sugar added initially by DuckDB to avoid repeating 
the fields.

### Does this PR introduce _any_ user-facing change?
Yes. See above.

### How was this patch tested?
Added SQL tests.

Closes #39144 from rxin/orderbyall.

Authored-by: Reynold Xin 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |   1 +
 .../sql/catalyst/analysis/ResolveOrderByAll.scala  |  81 +
 .../sql/catalyst/rules/RuleIdCollection.scala  |   1 +
 .../resources/sql-tests/inputs/order-by-all.sql|  44 +
 .../sql-tests/results/order-by-all.sql.out | 202 +
 5 files changed, 329 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ea3b81722af..ccaf07a4d52 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -297,6 +297,7 @@ class Analyzer(override val catalogManager: CatalogManager)
   ResolveGroupingAnalytics ::
   ResolvePivot ::
   ResolveUnpivot ::
+  ResolveOrderByAll ::
   ResolveGroupByAll ::
   ResolveOrdinalInOrderByAndGroupBy ::
   ResolveAggAliasInGroupBy ::
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveOrderByAll.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveOrderByAll.scala
new file mode 100644
index 000..7cf584dadcf
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveOrderByAll.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.SortOrder
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{SORT, 
UNRESOLVED_ATTRIBUTE}
+
+/**
+ * Resolve "order by all" in the following SQL pattern:
+ *  `select col1, col2 from table order by all`.
+ *
+ * It orders the query result by all columns, from left to right. The query 
above becomes:
+ *
+ *  `select col1, col2 from table order by col1, col2`
+ *
+ * This should also support specifying asc/desc, and nulls first/last.
+ */
+object ResolveOrderByAll extends Rule[LogicalPlan] {
+
+  val ALL = "ALL"
+
+  /**
+   * An extractor to pull out the SortOrder field in the ORDER BY ALL clause. 
We pull out that
+   * SortOrder object so we can pass its direction and null ordering.
+   */
+  object OrderByAll {
+def unapply(s: Sort): Option[SortOrder] = {
+  // This only applies to global ordering.
+  if (!s.global) {
+return None
+  }
+  // Don't do this if we have more than one order field. That means it's 
not order by all.
+  if (s.order.size != 1) {
+return None
+  }
+  // Don't do this if there's a child field called ALL. That should take 
precedence.
+  if (s.child.output.exists(_.name.toUpperCase() == ALL)) {
+return None
+  }
+
+  s.order.find { so =>
+so.child match {
+  case a: UnresolvedAttribute => a.name.t

[spark] branch master updated: [SPARK-41513][SQL] Implement an accumulator to collect per mapper row count metrics

2022-12-22 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 887f83122a0 [SPARK-41513][SQL] Implement an accumulator to collect per 
mapper row count metrics
887f83122a0 is described below

commit 887f83122a0b5684b6a5b0fab7ca9768a6c184f0
Author: Rui Wang 
AuthorDate: Thu Dec 22 19:59:43 2022 +0800

[SPARK-41513][SQL] Implement an accumulator to collect per mapper row count 
metrics

### What changes were proposed in this pull request?

In current Spark optimizer, a single partition shuffle might be created for 
a limit if this limit is not the last non-action operation (e.g. a filter 
following the limit and the data size exceeds a threshold). There is a 
possibility that the previous output partitions before go into this limit are 
sorted. The single partition shuffle approach has a correctness bug in this 
case: shuffle read partitions could be out of partition order and the limit 
exec just take the first limit rows wh [...]

So we propose a row count based AQE algorithm that optimizes this problem 
by two folds:

1. Avoid the extra sort on the shuffle read side (or with the limit exec) 
to achieve the correct result.
2. Avoid reading all shuffle data from mappers for this single partition 
shuffle to reduce shuffle cost.

Note that 1. is only applied for the sorted partition case where 2. is 
applied for general single partition shuffle + limit case

The algorithm works as the following:

1. Each mapper will record a row count when writing shuffle data.
2. Since this is single shuffle partition case, there is only one partition 
but N mappers.
3. A accumulatorV2 is implemented to collect a list of tuple which records 
the mapping between mapper id and the number of row written by the mapper (row 
count metrics)
4. AQE framework detects a plan shape of shuffle plus a global limit.
5. AQE framework reads necessary data from mappers based on the limit. For 
example, if mapper 1 writes 200 rows and mapper 2 writes 300 rows, and the 
limit is 500, AQE creates shuffle read node to write from both mapper 1 and 2, 
thus skip the left mappers.
6. This is both correct for limit with the sorted or non-sorted partitions.

This is the first step to implement the idea in 
https://issues.apache.org/jira/browse/SPARK-41512, which is to implement a row 
count accumulator that will be used to collect row count metrics.

### Why are the changes needed?

Optimization algorithm for global limit with single partition shuffle

### Does this PR introduce _any_ user-facing change?

NO
### How was this patch tested?

UT

Closes #39057 from amaliujia/add_row_counter.

Authored-by: Rui Wang 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/util/MapperRowCounter.scala   | 85 ++
 .../spark/sql/util/MapperRowCounterSuite.scala | 54 ++
 2 files changed, 139 insertions(+)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/util/MapperRowCounter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/util/MapperRowCounter.scala
new file mode 100644
index 000..7e1dfacca4a
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/util/MapperRowCounter.scala
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.util
+
+import java.{lang => jl}
+
+import org.apache.spark.util.AccumulatorV2
+
+/**
+ * An AccumulatorV2 counter for collecting a list of (mapper index, row count).
+ *
+ * @since 3.4.0
+ */
+class MapperRowCounter extends AccumulatorV2[jl.Long, 
java.util.List[(jl.Integer, jl.Long)]] {
+
+  private var _agg: java.util.List[(jl.Integer, jl.Long)] = _
+
+  private def getOrCreate = {
+_agg = Option(_agg).getOrElse(new java.util.ArrayList[(jl.Integer, 
jl.Long)]())
+_agg
+  }
+
+  /**
+   * Returns false if this accumulator has had any values added to it or the 
sum is non-zero.
+   */
+  overr

[spark] branch master updated (02562bb362b -> 3adc9b14a3e)

2022-12-22 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 02562bb362b [SPARK-41681][CONNECT] Factor GroupedData out to group.py
 add 3adc9b14a3e [SPARK-41635][SQL] GROUP BY ALL

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/error-classes.json   |   6 +
 .../spark/sql/catalyst/analysis/Analyzer.scala |   1 +
 .../sql/catalyst/analysis/CheckAnalysis.scala  |   4 +
 .../sql/catalyst/analysis/ResolveGroupByAll.scala  | 104 
 .../spark/sql/catalyst/analysis/unresolved.scala   |   8 +
 .../sql/catalyst/rules/RuleIdCollection.scala  |   1 +
 .../sql-tests/inputs/group-by-all-duckdb.sql   |  29 +++
 .../sql-tests/inputs/group-by-all-mosha.sql|  32 +++
 .../resources/sql-tests/inputs/group-by-all.sql|  82 +++
 .../sql-tests/results/group-by-all-duckdb.sql.out  | 119 +
 .../sql-tests/results/group-by-all-mosha.sql.out   | 142 +++
 .../sql-tests/results/group-by-all.sql.out | 266 +
 12 files changed, 794 insertions(+)
 create mode 100644 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupByAll.scala
 create mode 100644 
sql/core/src/test/resources/sql-tests/inputs/group-by-all-duckdb.sql
 create mode 100644 
sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql
 create mode 100644 
sql/core/src/test/resources/sql-tests/inputs/group-by-all.sql
 create mode 100644 
sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out
 create mode 100644 
sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out
 create mode 100644 
sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41639][SQL][PROTOBUF] Remove ScalaReflectionLock from SchemaConverters

2022-12-21 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6e537f30ac2 [SPARK-41639][SQL][PROTOBUF] Remove ScalaReflectionLock 
from SchemaConverters
6e537f30ac2 is described below

commit 6e537f30ac2b19617c6cc11dd4f7f92946988f7b
Author: SandishKumarHN 
AuthorDate: Thu Dec 22 11:36:30 2022 +0800

[SPARK-41639][SQL][PROTOBUF] Remove ScalaReflectionLock from 
SchemaConverters

### What changes were proposed in this pull request?

Following up from PR https://github.com/apache/spark/pull/38922 to remove 
unnecessary ScalaReflectionLock from SchemaConvertors file.

cc: cloud-fan

### Why are the changes needed?

removing unnecessary code

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

existing unit tests

Closes #39147 from SandishKumarHN/SPARK-41639.

Authored-by: SandishKumarHN 
Signed-off-by: Wenchen Fan 
---
 .../scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala   | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
index 8d321c13a56..bb4aa492f5c 100644
--- 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
+++ 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/SchemaConverters.scala
@@ -22,7 +22,6 @@ import com.google.protobuf.Descriptors.{Descriptor, 
FieldDescriptor}
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.protobuf.ScalaReflectionLock
 import org.apache.spark.sql.types._
 
 @DeveloperApi
@@ -48,7 +47,7 @@ object SchemaConverters {
 
   def toSqlTypeHelper(
   descriptor: Descriptor,
-  protobufOptions: ProtobufOptions): SchemaType = 
ScalaReflectionLock.synchronized {
+  protobufOptions: ProtobufOptions): SchemaType = {
 SchemaType(
   StructType(descriptor.getFields.asScala.flatMap(
 structFieldFor(_,


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch branch-3.3 updated: [SPARK-41350][3.3][SQL][FOLLOWUP] Allow simple name access of join hidden columns after alias

2022-12-21 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 9934b56dbb2 [SPARK-41350][3.3][SQL][FOLLOWUP] Allow simple name access 
of join hidden columns after alias
9934b56dbb2 is described below

commit 9934b56dbb23debfdc02340b493263a665189c0c
Author: Wenchen Fan 
AuthorDate: Thu Dec 22 11:26:19 2022 +0800

[SPARK-41350][3.3][SQL][FOLLOWUP] Allow simple name access of join hidden 
columns after alias

backport https://github.com/apache/spark/pull/39077 to 3.3

Closes #39121 from cloud-fan/backport.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 .../catalyst/expressions/namedExpressions.scala|  7 +++-
 .../resources/sql-tests/inputs/natural-join.sql|  2 +
 .../test/resources/sql-tests/inputs/using-join.sql |  8 
 .../sql-tests/results/natural-join.sql.out | 10 +
 .../resources/sql-tests/results/using-join.sql.out | 44 ++
 5 files changed, 69 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index a399121298b..cb2225bcfc7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark
 import org.apache.spark.sql.catalyst.trees.TreePattern
 import org.apache.spark.sql.catalyst.trees.TreePattern._
-import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, 
METADATA_COL_ATTR_KEY}
+import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.types._
 import org.apache.spark.util.collection.BitSet
@@ -191,7 +191,10 @@ case class Alias(child: Expression, name: String)(
 
   override def toAttribute: Attribute = {
 if (resolved) {
-  AttributeReference(name, child.dataType, child.nullable, 
metadata)(exprId, qualifier)
+  val a = AttributeReference(name, child.dataType, child.nullable, 
metadata)(exprId, qualifier)
+  // Alias has its own qualifier. It doesn't make sense to still restrict 
the hidden columns
+  // of natural/using join to be accessed by qualified name only.
+  if (a.qualifiedAccessOnly) a.markAsAllowAnyAccess() else a
 } else {
   UnresolvedAttribute.quoted(name)
 }
diff --git a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql 
b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql
index 9c9ce6c37ba..5815eee2d9e 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql
@@ -52,6 +52,8 @@ SELECT nt1.k, nt2.k FROM nt1 natural join nt2;
 
 SELECT k FROM (SELECT nt2.k FROM nt1 natural join nt2);
 
+SELECT nt2.k AS key FROM nt1 natural join nt2 ORDER BY key;
+
 SELECT nt1.k, nt2.k FROM nt1 natural join nt2 where k = "one";
 
 SELECT * FROM (SELECT * FROM nt1 natural join nt2);
diff --git a/sql/core/src/test/resources/sql-tests/inputs/using-join.sql 
b/sql/core/src/test/resources/sql-tests/inputs/using-join.sql
index 414221e5b71..0c06023b14d 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/using-join.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/using-join.sql
@@ -21,6 +21,8 @@ SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k);
 
 SELECT k FROM (SELECT nt2.k FROM nt1 left outer join nt2 using (k));
 
+SELECT nt2.k AS key FROM nt1 left outer join nt2 using (k) ORDER BY key;
+
 SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k) ORDER BY nt2.k;
 
 SELECT k, nt1.k FROM nt1 left outer join nt2 using (k);
@@ -47,6 +49,8 @@ SELECT nt1.k, nt2.k FROM nt1 right outer join nt2 using (k);
 
 SELECT k FROM (SELECT nt1.k FROM nt1 right outer join nt2 using (k));
 
+SELECT nt1.k AS key FROM nt1 right outer join nt2 using (k) ORDER BY key;
+
 SELECT k, nt1.k FROM nt1 right outer join nt2 using (k);
 
 SELECT k, nt2.k FROM nt1 right outer join nt2 using (k);
@@ -61,6 +65,8 @@ SELECT nt1.k, nt2.k FROM nt1 full outer join nt2 using (k);
 
 SELECT k FROM (SELECT nt2.k FROM nt1 full outer join nt2 using (k));
 
+SELECT nt2.k AS key FROM nt1 full outer join nt2 using (k) ORDER BY key;
+
 SELECT k, nt1.k FROM nt1 full outer join nt2 using (k);
 
 SELECT k, nt2.k FROM nt1 full outer join nt2 using (k);
@@ -75,6 +81,8 @@ SELECT nt1.k, nt2.k FROM nt1 inner join nt2 using (k);
 
 SELECT k FROM (SELECT nt2.k FROM nt1 inner join nt2 using (k));
 
+SELECT nt2.k AS key FROM nt1 inner join nt2 using (k) ORDER BY key;
+
 SELECT k, 

[spark] branch master updated: [SPARK-41669][SQL] Early pruning in canCollapseExpressions

2022-12-21 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 73593d8670c [SPARK-41669][SQL] Early pruning in canCollapseExpressions
73593d8670c is described below

commit 73593d8670ca5dd933327039fc659dd6ef764963
Author: allisonwang-db 
AuthorDate: Thu Dec 22 10:09:03 2022 +0800

[SPARK-41669][SQL] Early pruning in canCollapseExpressions

### What changes were proposed in this pull request?

This PR speeds up the optimizer rule `CollapseProject` when the project 
list contains a large number of expressions by doing early pruning in 
`canCollapseExpressions`.

This PR also adds a new benchmark in WideSchemaBenchmark to compare the 
performance.

Before this change:
```
[info] Apple M1 Max
[info] optimize large select:Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
[info] 

[info] 100 columns   7  
8   1  0.0 6820416.0   1.0X
[info] 1000 columns170
180   5  0.0   170239250.0   0.0X
[info] 1 columns 13655  
13823 237  0.0 13655051334.0   0.0X
```

After this change:
```
[info] Apple M1 Max
[info] optimize large select:Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
[info] 

[info] 100 columns   5  
5   1  0.0 4766000.0   1.0X
[info] 1000 columns 39 
41   1  0.039015250.0   0.1X
[info] 1 columns   450
468  21  0.0   450092958.0   0.0X
```

### Why are the changes needed?

To improve query compilation performance for wide tables.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Existing unit tests.

Closes #39161 from allisonwang-db/spark-41669-collapse-proj.

Authored-by: allisonwang-db 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  4 +++-
 .../sql/execution/benchmark/WideSchemaBenchmark.scala  | 18 ++
 2 files changed, 21 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 1f0fb667753..fe3d19b2d57 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -1039,7 +1039,9 @@ object CollapseProject extends Rule[LogicalPlan] with 
AliasHelper {
 // We can only collapse expressions if all input expressions meet the 
following criteria:
 // - The input is deterministic.
 // - The input is only consumed once OR the underlying input expression is 
cheap.
-consumers.flatMap(collectReferences)
+consumers
+  .filter(_.references.exists(producerMap.contains))
+  .flatMap(collectReferences)
   .groupBy(identity)
   .mapValues(_.size)
   .forall {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
index 58e92297d3b..eb8d41ee69f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
@@ -21,6 +21,7 @@ import java.io.File
 
 import org.apache.spark.benchmark.Benchmark
 import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.functions.lit
 import org.apache.spark.util.Utils
 
 /**
@@ -94,6 +95,19 @@ object WideSchemaBenchmark extends SqlBasedBenchmark {
 benchmark.run()
   }
 
+  def optimizeLargeSelectExpressions(): Unit = {
+val benchmark = new Benchmark("optimize large select", 1, output = output)
+Seq(100, 1000, 1).foreach { width =>
+  val columns = (1 to width).map(i => s"id as c_$i")
+  val df = spark.range(1).selectExpr(columns: _*).cache()
+  df.count()  // force caching
+  benchmark.addCase

[spark] branch master updated: [SPARK-41509][SQL] Only execute `Murmur3Hash` on aggregate expressions for semi-join runtime filter

2022-12-21 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 739aae15540 [SPARK-41509][SQL] Only execute `Murmur3Hash` on aggregate 
expressions for semi-join runtime filter
739aae15540 is described below

commit 739aae15540ff5957037ad88df3139e3934e3712
Author: Jiaan Geng 
AuthorDate: Wed Dec 21 21:10:46 2022 +0800

[SPARK-41509][SQL] Only execute `Murmur3Hash` on aggregate expressions for 
semi-join runtime filter

### What changes were proposed in this pull request?
Currently, Spark runtime filter supports bloom filter and in subquery 
filter.
The in subquery filter always execute `Murmur3Hash` before aggregate the 
join key.

Because the data size before aggregate lager than after, we can only 
execute `Murmur3Hash` for aggregation and it will reduce the number of calls to 
`Murmur3Hash` and improve performance.

### Why are the changes needed?
Improve performance for semi-join runtime filter.

### Does this PR introduce _any_ user-facing change?
'No'.
Just update the inner implementation.

### How was this patch tested?
Manually test TPC-DS.
```
spark.sql.optimizer.runtime.bloomFilter.enabled=false
spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled=true
```

TPC-DS data size: 2TB.
This improvement is valid for below TPC-DS test cases and no regression for 
other test cases.

| TPC-DS Query   | Before(Seconds)  | After(Seconds)  | Speedup(Percent)  |
|    |   |   |   |
| q23a | 542.03 | 539.06 | 0.55% |
| q23b | 776.7 | 769.74 | 0.90% |
| q24a | 442.25 | 436.75 | 1.26% |
| q24b | 436.16 | 432.86 | 0.76% |
| q50 | 200.92 | 193.36 | 3.91% |
| q64 | 426.73 | 421.03 | 1.35% |
| q67 | 987.79 | 956.82 | 3.24% |
| q93 | 397.26 | 393.9 | 0.85% |
| All TPC-DS | 8392.49 | 8302.56 | 1.08% |

Closes #39049 from beliefer/SPARK-41509.

Authored-by: Jiaan Geng 
Signed-off-by: Wenchen Fan 
---
 .../org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
index df04a248a27..161abff8fe3 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
@@ -101,7 +101,8 @@ object InjectRuntimeFilter extends Rule[LogicalPlan] with 
PredicateHelper with J
 require(filterApplicationSideExp.dataType == 
filterCreationSideExp.dataType)
 val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
 val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
-val aggregate = ColumnPruning(Aggregate(Seq(alias), Seq(alias), 
filterCreationSidePlan))
+val aggregate =
+  ColumnPruning(Aggregate(Seq(filterCreationSideExp), Seq(alias), 
filterCreationSidePlan))
 if (!canBroadcastBySize(aggregate, conf)) {
   // Skip the InSubquery filter if the size of `aggregate` is beyond 
broadcast join threshold,
   // i.e., the semi-join will be a shuffled join, which is not worthwhile.


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41660][SQL] Only propagate metadata columns if they are used

2022-12-21 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new bd864a085f2 [SPARK-41660][SQL] Only propagate metadata columns if they 
are used
bd864a085f2 is described below

commit bd864a085f2764b8ccdfe67ffaf7400b6f44f717
Author: Wenchen Fan 
AuthorDate: Wed Dec 21 21:07:34 2022 +0800

[SPARK-41660][SQL] Only propagate metadata columns if they are used

### What changes were proposed in this pull request?

Ideally it's OK to always propagate metadata columns, as column pruning 
will kick in later and prune them aways if they are not used. However, it may 
cause problems in cases like CTE. https://github.com/apache/spark/pull/39081 
fixed such a bug.

This PR only propagates metadata columns if they are used, to keep the 
analyzed plan simple and reliable.

### Why are the changes needed?

avoid potential bugs.

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

new tests

Closes #39152 from cloud-fan/follow.

Authored-by: Wenchen Fan 
Signed-off-by: Wenchen Fan 
---
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala   | 15 +--
 .../spark/sql/connector/MetadataColumnSuite.scala   | 17 +
 2 files changed, 26 insertions(+), 6 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e959e7208a4..c21ff7bd90f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -978,7 +978,7 @@ class Analyzer(override val catalogManager: CatalogManager)
 if (metaCols.isEmpty) {
   node
 } else {
-  val newNode = addMetadataCol(node)
+  val newNode = addMetadataCol(node, attr => metaCols.exists(_.exprId 
== attr.exprId))
   // We should not change the output schema of the plan. We should 
project away the extra
   // metadata columns if necessary.
   if (newNode.sameOutput(node)) {
@@ -1012,15 +1012,18 @@ class Analyzer(override val catalogManager: 
CatalogManager)
   })
 }
 
-private def addMetadataCol(plan: LogicalPlan): LogicalPlan = plan match {
-  case s: ExposesMetadataColumns => s.withMetadataColumns()
-  case p: Project =>
+private def addMetadataCol(
+plan: LogicalPlan,
+isRequired: Attribute => Boolean): LogicalPlan = plan match {
+  case s: ExposesMetadataColumns if s.metadataOutput.exists(isRequired) =>
+s.withMetadataColumns()
+  case p: Project if p.metadataOutput.exists(isRequired) =>
 val newProj = p.copy(
   projectList = p.projectList ++ p.metadataOutput,
-  child = addMetadataCol(p.child))
+  child = addMetadataCol(p.child, isRequired))
 newProj.copyTagsFrom(p)
 newProj
-  case _ => plan.withNewChildren(plan.children.map(addMetadataCol))
+  case _ => plan.withNewChildren(plan.children.map(addMetadataCol(_, 
isRequired)))
 }
   }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala
index 8454b9f85ec..9abf0fd59e6 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/MetadataColumnSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.connector
 
 import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
 import org.apache.spark.sql.functions.struct
 
 class MetadataColumnSuite extends DatasourceV2SQLBase {
@@ -232,4 +233,20 @@ class MetadataColumnSuite extends DatasourceV2SQLBase {
   )
 }
   }
+
+  test("SPARK-41660: only propagate metadata columns if they are used") {
+withTable(tbl) {
+  prepareTable()
+  val df = sql(s"SELECT t2.id FROM $tbl t1 JOIN $tbl t2 USING (id)")
+  val scans = df.logicalPlan.collect {
+case d: DataSourceV2Relation => d
+  }
+  assert(scans.length == 2)
+  scans.foreach { scan =>
+// The query only access join hidden columns, and scan nodes should 
not expose its metadata
+// columns.
+assert(scan.output.map(_.name) == Seq("id", "data"))
+  }
+}
+  }
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (940946515bd -> fd6d226528e)

2022-12-20 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 940946515bd [SPARK-41440][CONNECT][PYTHON] Implement 
`DataFrame.randomSplit`
 add fd6d226528e [SPARK-41631][SQL] Support implicit lateral column alias 
resolution on Aggregate

No new revisions were added by this update.

Summary of changes:
 core/src/main/resources/error/error-classes.json   |   5 +
 .../spark/sql/catalyst/analysis/Analyzer.scala |  32 +-
 .../ResolveLateralColumnAliasReference.scala   | 107 +++-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  15 +-
 .../org/apache/spark/sql/internal/SQLConf.scala|   2 +-
 .../apache/spark/sql/LateralColumnAliasSuite.scala | 613 ++---
 .../scala/org/apache/spark/sql/QueryTest.scala |   2 +-
 7 files changed, 674 insertions(+), 102 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41396][SQL][PROTOBUF] OneOf field support and recursion checks

2022-12-20 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d33a59c940f [SPARK-41396][SQL][PROTOBUF] OneOf field support and 
recursion checks
d33a59c940f is described below

commit d33a59c940f0e8f0b93d91cc9e700c2cb533d54e
Author: SandishKumarHN 
AuthorDate: Wed Dec 21 09:37:15 2022 +0800

[SPARK-41396][SQL][PROTOBUF] OneOf field support and recursion checks

Oneof fields allow a message to contain one and only one of a defined set 
of field types, while recursive fields provide a way to define messages that 
can refer to themselves, allowing for the creation of complex and nested data 
structures.  with this change users will be able to use protobuf OneOf fields 
with spark-protobuf, making it a more complete and useful tool for processing 
protobuf data.

**Support for circularReferenceDepth:**
The `recursive.fields.max.depth` parameter can be specified in the 
from_protobuf options to control the maximum allowed recursion depth for a 
field. Setting `recursive.fields.max.depth` to 0 drops all-recursive fields, 
setting it to 1 allows it to be recursed once, and setting it to 2 allows it to 
be recursed twice. Attempting to set the `recursive.fields.max.depth` to a 
value greater than 10 is not allowed. If the `recursive.fields.max.depth` is 
not specified, it will default to -1;  [...]
SQL Schema for the protobuf message
 ```
message Person {
 string name = 1;
 Person bff = 2
}
```
will vary based on the value of `recursive.fields.max.depth`.
```
0: struct
1: struct>
2: struct>> ...
```

### What changes were proposed in this pull request?

- Add support for protobuf oneof field
- Stop recursion at the first level when a recursive field is encountered. 
(instead of throwing an error)

### Why are the changes needed?

Stop recursion at the first level and handle nulltype in deserilization.

### Does this PR introduce _any_ user-facing change?

NA
### How was this patch tested?

Added Unit tests for OneOf field support and recursion checks.
Tested full support for nested OneOf fields and message types using real 
data from Kafka on a real cluster

cc: rangadi mposdev21

Closes #38922 from SandishKumarHN/SPARK-41396.

Authored-by: SandishKumarHN 
Signed-off-by: Wenchen Fan 
---
 .../sql/protobuf/ProtobufDataToCatalyst.scala  |   2 +-
 .../spark/sql/protobuf/ProtobufDeserializer.scala  |   8 +-
 .../spark/sql/protobuf/utils/ProtobufOptions.scala |   8 +
 .../sql/protobuf/utils/SchemaConverters.scala  |  69 ++-
 .../test/resources/protobuf/functions_suite.desc   | Bin 6678 -> 8739 bytes
 .../test/resources/protobuf/functions_suite.proto  |  85 ++-
 .../sql/protobuf/ProtobufFunctionsSuite.scala  | 576 -
 core/src/main/resources/error/error-classes.json   |   2 +-
 8 files changed, 721 insertions(+), 29 deletions(-)

diff --git 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
index c0997b1bd06..da44f94d5ea 100644
--- 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
+++ 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
@@ -39,7 +39,7 @@ private[protobuf] case class ProtobufDataToCatalyst(
   override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
 
   override lazy val dataType: DataType = {
-val dt = SchemaConverters.toSqlType(messageDescriptor).dataType
+val dt = SchemaConverters.toSqlType(messageDescriptor, 
protobufOptions).dataType
 parseMode match {
   // With PermissiveMode, the output Catalyst row might contain columns of 
null values for
   // corrupt records, even if some of the columns are not nullable in the 
user-provided schema.
diff --git 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDeserializer.scala
 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDeserializer.scala
index 46366ba268b..224e22c0f52 100644
--- 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDeserializer.scala
+++ 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDeserializer.scala
@@ -156,6 +156,9 @@ private[sql] class ProtobufDeserializer(
 (protoType.getJavaType, catalystType) match {
 
   case (null, NullType) => (updater, ordinal, _) => 
updater.setNullAt(ordinal)
+  // It is possible that this will result in data being dropped, This is 
intentional,
+  // to catch recursive fields and drop them as necessary.
+  case 

[spark] branch master updated (1310ce2d894 -> 52082d3906b)

2022-12-19 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 1310ce2d894 [SPARK-41425][UI] Protobuf serializer for 
RDDStorageInfoWrapper
 add 52082d3906b Revert "[SPARK-27561][SQL][FOLLOWUP] Move the two rules 
for Later column alias into one file"

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/Analyzer.scala | 113 ++-
 ...la => ResolveLateralColumnAliasReference.scala} | 125 +
 .../sql/catalyst/rules/RuleIdCollection.scala  |   2 +-
 3 files changed, 118 insertions(+), 122 deletions(-)
 rename 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/{ResolveLateralColumnAlias.scala
 => ResolveLateralColumnAliasReference.scala} (50%)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated (05428c21844 -> bc7b8f54d40)

2022-12-19 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 05428c21844 [SPARK-41334][CONNECT][PYTHON] Move `SortOrder` proto from 
relations to expressions
 add bc7b8f54d40 [SPARK-41441][SQL] Support Generate with no required child 
output to host outer references

No new revisions were added by this update.

Summary of changes:
 .../catalyst/optimizer/DecorrelateInnerQuery.scala | 12 +
 .../spark/sql/catalyst/optimizer/subquery.scala| 29 ---
 .../OptimizeOneRowRelationSubquerySuite.scala  | 16 +-
 .../resources/sql-tests/inputs/join-lateral.sql|  8 +++
 .../sql-tests/results/join-lateral.sql.out | 58 ++
 5 files changed, 115 insertions(+), 8 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



[spark] branch master updated: [SPARK-41511][SQL] LongToUnsafeRowMap support ignoresDuplicatedKey

2022-12-15 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 1f35577a3ea [SPARK-41511][SQL] LongToUnsafeRowMap support 
ignoresDuplicatedKey
1f35577a3ea is described below

commit 1f35577a3ead9c6268b5ba47c2e3aec60484e3cc
Author: ulysses-you 
AuthorDate: Thu Dec 15 21:50:20 2022 +0800

[SPARK-41511][SQL] LongToUnsafeRowMap support ignoresDuplicatedKey

### What changes were proposed in this pull request?

Make `LongToUnsafeRowMap` take a new flag `ignoresDuplicatedKey` so we can 
skip appending duplicated key if unnecessary.

### Why are the changes needed?

For left semi and left anti hash join, the duplicated keys of build side 
have no meaning.

Previous, we supported ingore duplicated keys for `UnsafeHashedRelation`. 
We can also optimize `LongHashedRelation`.

### Does this PR introduce _any_ user-facing change?

no, only improve performance

### How was this patch tested?

add test

Closes #39051 from ulysses-you/appendIfNoExists.

Authored-by: ulysses-you 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/execution/joins/HashedRelation.scala | 34 +++---
 .../sql/execution/joins/HashedRelationSuite.scala  | 25 
 2 files changed, 49 insertions(+), 10 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
index 11d3af4e546..4d3e63282fa 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
@@ -154,7 +154,7 @@ private[execution] object HashedRelation {
   EmptyHashedRelation
 } else if (key.length == 1 && key.head.dataType == LongType && 
!allowsNullKey) {
   // NOTE: LongHashedRelation does not support NULL keys.
-  LongHashedRelation(input, key, sizeEstimate, mm, isNullAware)
+  LongHashedRelation(input, key, sizeEstimate, mm, isNullAware, 
ignoresDuplicatedKey)
 } else {
   UnsafeHashedRelation(input, key, sizeEstimate, mm, isNullAware, 
allowsNullKey,
 ignoresDuplicatedKey)
@@ -532,7 +532,10 @@ private[joins] object UnsafeHashedRelation {
  *
  * see 
http://java-performance.info/implementing-world-fastest-java-int-to-int-hash-map/
  */
-private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, 
capacity: Int)
+private[execution] final class LongToUnsafeRowMap(
+val mm: TaskMemoryManager,
+capacity: Int,
+ignoresDuplicatedKey: Boolean = false)
   extends MemoryConsumer(mm, MemoryMode.ON_HEAP) with Externalizable with 
KryoSerializable {
 
   // Whether the keys are stored in dense mode or not.
@@ -759,6 +762,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: 
TaskMemoryManager, cap
   throw QueryExecutionErrors.rowLargerThan256MUnsupportedError()
 }
 
+val pos = findKeyPosition(key)
+if (ignoresDuplicatedKey && array(pos + 1) != 0) {
+  return
+}
+
 if (key < minKey) {
   minKey = key
 }
@@ -775,18 +783,22 @@ private[execution] final class LongToUnsafeRowMap(val mm: 
TaskMemoryManager, cap
 Platform.putLong(page, cursor, 0)
 cursor += 8
 numValues += 1
-updateIndex(key, toAddress(offset, row.getSizeInBytes))
+updateIndex(key, pos, toAddress(offset, row.getSizeInBytes))
   }
 
-  /**
-   * Update the address in array for given key.
-   */
-  private def updateIndex(key: Long, address: Long): Unit = {
+  private def findKeyPosition(key: Long): Int = {
 var pos = firstSlot(key)
 assert(numKeys < array.length / 2)
 while (array(pos) != key && array(pos + 1) != 0) {
   pos = nextSlot(pos)
 }
+pos
+  }
+
+  /**
+   * Update the address in array for given key.
+   */
+  private def updateIndex(key: Long, pos: Int, address: Long): Unit = {
 if (array(pos + 1) == 0) {
   // this is the first value for this key, put the address in array.
   array(pos) = key
@@ -838,7 +850,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: 
TaskMemoryManager, cap
 var i = 0
 while (i < old_array.length) {
   if (old_array(i + 1) > 0) {
-updateIndex(old_array(i), old_array(i + 1))
+val key = old_array(i)
+updateIndex(key, findKeyPosition(key), old_array(i + 1))
   }
   i += 2
 }
@@ -1056,9 +1069,10 @@ private[joins] object LongHashedRelation {
   key: Seq[Expression],
   sizeEstimate: Int,
   taskMemoryManager: TaskMemoryManager,
-  isNullAware: Boolean = false): HashedRelation = {
+  isNullAware: Boolean = false,
+  ignoresDuplicatedKey: Boolean = fal

[spark] branch master updated: [SPARK-27561][SQL] Support implicit lateral column alias resolution on Project

2022-12-13 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7e9b88bfceb [SPARK-27561][SQL] Support implicit lateral column alias 
resolution on Project
7e9b88bfceb is described below

commit 7e9b88bfceb86d3b32e82a86b672aab3c74def8c
Author: Xinyi Yu 
AuthorDate: Wed Dec 14 00:14:06 2022 +0800

[SPARK-27561][SQL] Support implicit lateral column alias resolution on 
Project

### What changes were proposed in this pull request?
This PR implements a new feature: Implicit lateral column alias  on 
`Project` case, controlled by 
`spark.sql.lateralColumnAlias.enableImplicitResolution` temporarily (default 
false now, but will turn on this conf once the feature is completely merged).

 Lateral column alias
View https://issues.apache.org/jira/browse/SPARK-27561 for more details on 
lateral column alias.
There are two main cases to support: LCA in Project, and LCA in Aggregate.
```sql
-- LCA in Project. The base_salary references an attribute defined by a 
previous alias
SELECT salary AS base_salary, base_salary + bonus AS total_salary
FROM employee

-- LCA in Aggregate. The avg_salary references an attribute defined by a 
previous alias
SELECT dept, average(salary) AS avg_salary, avg_salary + average(bonus)
FROM employee
GROUP BY dept
```
This **implicit** lateral column alias (no explicit keyword, e.g. 
`lateral.base_salary`) should be supported.

 High level design
This PR defines a new Resolution rule, `ResolveLateralColumnAlias` to 
resolve the implicit lateral column alias, covering the `Project` case.
It introduces a new leaf node NamedExpression, 
`LateralColumnAliasReference`, as a placeholder used to hold a referenced that 
has been temporarily resolved as the reference to a lateral column alias.

The whole process is generally divided into two phases:
1) recognize **resolved** lateral alias, wrap the attributes referencing 
them with `LateralColumnAliasReference`.
 2) when the whole operator is resolved, unwrap 
`LateralColumnAliasReference`. For Project, it further resolves the attributes 
and push down the referenced lateral aliases to the new Project.

For example:
```
// Before
Project [age AS a, 'a + 1]
+- Child

// After phase 1
Project [age AS a, lateralalias(a) + 1]
+- Child

// After phase 2
Project [a, a + 1]
+- Project [child output, age AS a]
   +- Child
```

 Resolution order
Given this new rule, the name resolution order will be (higher -> lower):
```
local table column > local metadata attribute > local lateral column alias 
> all others (outer reference of subquery, parameters of SQL UDF, ..)
```

There is a recent refactor that moves the creation of `OuterReference` in 
the Resolution batch: https://github.com/apache/spark/pull/38851.
Because lateral column alias has higher resolution priority than outer 
reference, it will try to resolve an `OuterReference` using lateral column 
alias, similar as an `UnresolvedAttribute`. If success, it strips 
`OuterReference` and also wraps it with `LateralColumnAliasReference`.

### Why are the changes needed?
The lateral column alias is a popular feature wanted for a long time. It is 
supported by lots of other database vendors (Redshift, snowflake, etc) and 
provides a better user experience.

### Does this PR introduce _any_ user-facing change?
Yes, as shown in the above example, it will be able to resolve lateral 
column alias. I will write the migration guide or release note when most PRs of 
this feature are merged.

### How was this patch tested?
Existing tests and newly added tests.

Closes #38776 from anchovYu/SPARK-27561-refactor.

Authored-by: Xinyi Yu 
Signed-off-by: Wenchen Fan 
---
 core/src/main/resources/error/error-classes.json   |   6 +
 .../sql/catalyst/expressions/AttributeMap.scala|   3 +-
 .../sql/catalyst/expressions/AttributeMap.scala|   3 +
 .../spark/sql/catalyst/analysis/Analyzer.scala | 119 +++-
 .../sql/catalyst/analysis/CheckAnalysis.scala  |  25 +-
 .../ResolveLateralColumnAliasReference.scala   | 135 +
 .../catalyst/expressions/namedExpressions.scala|  33 +++
 .../spark/sql/catalyst/expressions/subquery.scala  |   9 +-
 .../sql/catalyst/rules/RuleIdCollection.scala  |   2 +
 .../spark/sql/catalyst/trees/TreePatterns.scala|   1 +
 .../spark/sql/errors/QueryCompilationErrors.scala  |  19 ++
 .../org/apache/spark/sql/internal/SQLConf.scala|  11 +
 .../apache/spark/sql/LateralColumnAliasSuite.scala | 327 +
 13 files changed, 686 insertions(+), 7 deletions(-)

diff --git a/core/src/ma

[spark] branch master updated: [SPARK-41468][SQL][FOLLOWUP] Handle NamedLambdaVariables in EquivalentExpressions

2022-12-13 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 27f4d1ef848 [SPARK-41468][SQL][FOLLOWUP] Handle NamedLambdaVariables 
in EquivalentExpressions
27f4d1ef848 is described below

commit 27f4d1ef848caf357faaf90d7ee4f625e0a3b5d3
Author: Peter Toth 
AuthorDate: Tue Dec 13 17:05:08 2022 +0800

[SPARK-41468][SQL][FOLLOWUP] Handle NamedLambdaVariables in 
EquivalentExpressions

### What changes were proposed in this pull request?
This is a follow-up PR to https://github.com/apache/spark/pull/39010 to 
handle `NamedLambdaVariable`s too.

### Why are the changes needed?
To avoid possible issues with higer-order functions.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Existing UTs.

Closes #39046 from 
peter-toth/SPARK-41468-fix-planexpressions-in-equivalentexpressions-follow-up.

Authored-by: Peter Toth 
Signed-off-by: Wenchen Fan 
---
 .../spark/sql/catalyst/expressions/EquivalentExpressions.scala   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
index 3ffd9f9d887..330d66a21be 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
@@ -144,9 +144,10 @@ class EquivalentExpressions {
 
   private def supportedExpression(e: Expression) = {
 !e.exists {
-  // `LambdaVariable` is usually used as a loop variable, which can't be 
evaluated ahead of the
-  // loop. So we can't evaluate sub-expressions containing 
`LambdaVariable` at the beginning.
+  // `LambdaVariable` is usually used as a loop variable and 
`NamedLambdaVariable` is used in
+  // higher-order functions, which can't be evaluated ahead of the 
execution.
   case _: LambdaVariable => true
+  case _: NamedLambdaVariable => true
 
   // `PlanExpression` wraps query plan. To compare query plans of 
`PlanExpression` on executor,
   // can cause error like NPE.


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



<    4   5   6   7   8   9   10   11   12   13   >