(spark) branch master updated: [SPARK-46490][SQL] Require error classes in `SparkThrowable` sub-classes

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

maxgekk 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 b05c61266d83 [SPARK-46490][SQL] Require error classes in 
`SparkThrowable` sub-classes
b05c61266d83 is described below

commit b05c61266d83590dcec642ecae929d6529b0ad1d
Author: Max Gekk 
AuthorDate: Sat Dec 30 12:28:23 2023 +0300

[SPARK-46490][SQL] Require error classes in `SparkThrowable` sub-classes

### What changes were proposed in this pull request?
In the PR, I propose to create `SparkThrowable` sub-classes only with an 
error class by making the constructor with `message` private.

### Why are the changes needed?
To improve user experience with Spark SQL by unifying error exceptions: the 
final goal is all Spark exception should contain an error class.

### Does this PR introduce _any_ user-facing change?
No since user's code shouldn't throw `SparkThrowable` sub-classes but it 
can if it depends on error message formats.

### How was this patch tested?
By existing test test suites like:
```
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44464 from MaxGekk/ban-messages-SparkThrowable-subclass.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json|  30 ++
 .../scala/org/apache/spark/SparkException.scala| 112 ++---
 .../connect/client/SparkConnectClientSuite.scala   |  47 +
 .../connect/client/GrpcExceptionConverter.scala|  88 
 4 files changed, 135 insertions(+), 142 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 9f68d4c5a53e..4f34ca29ea65 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -7073,6 +7073,36 @@
   "Namespace '' is non empty. "
 ]
   },
+  "_LEGACY_ERROR_TEMP_3104" : {
+"message" : [
+  ""
+]
+  },
+  "_LEGACY_ERROR_TEMP_3105" : {
+"message" : [
+  ""
+]
+  },
+  "_LEGACY_ERROR_TEMP_3106" : {
+"message" : [
+  ""
+]
+  },
+  "_LEGACY_ERROR_TEMP_3107" : {
+"message" : [
+  ""
+]
+  },
+  "_LEGACY_ERROR_TEMP_3108" : {
+"message" : [
+  ""
+]
+  },
+  "_LEGACY_ERROR_TEMP_3109" : {
+"message" : [
+  ""
+]
+  },
   "_LEGACY_ERROR_USER_RAISED_EXCEPTION" : {
 "message" : [
   ""
diff --git a/common/utils/src/main/scala/org/apache/spark/SparkException.scala 
b/common/utils/src/main/scala/org/apache/spark/SparkException.scala
index 3bcdd0a7c29b..d2a1c6727730 100644
--- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala
+++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala
@@ -133,11 +133,11 @@ private[spark] case class ExecutorDeadException(message: 
String)
 /**
  * Exception thrown when Spark returns different result after upgrading to a 
new version.
  */
-private[spark] class SparkUpgradeException(
-  message: String,
-  cause: Option[Throwable],
-  errorClass: Option[String],
-  messageParameters: Map[String, String])
+private[spark] class SparkUpgradeException private(
+message: String,
+cause: Option[Throwable],
+errorClass: Option[String],
+messageParameters: Map[String, String])
   extends RuntimeException(message, cause.orNull) with SparkThrowable {
 
   def this(
@@ -152,15 +152,6 @@ private[spark] class SparkUpgradeException(
 )
   }
 
-  def this(message: String, cause: Option[Throwable]) = {
-this(
-  message,
-  cause = cause,
-  errorClass = None,
-  messageParameters = Map.empty
-)
-  }
-
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
   override def getErrorClass: String = errorClass.orNull
@@ -169,7 +160,7 @@ private[spark] class SparkUpgradeException(
 /**
  * Arithmetic exception thrown from Spark with an error class.
  */
-private[spark] class SparkArithmeticException(
+private[spark] class SparkArithmeticException private(
 message: String,
 errorClass: Option[String],
 messageParameters: Map[String, String],
@@ -189,14 +180,10 @@ private[spark] class SparkArithmeticException(
 )
   }
 
-  def this(message: String) = {
-this(
-  message,
-  errorClass = None,
-  messageParameters = Map.empty,
-  context = Array.empty
-)
-  }
+  def t

(spark) branch branch-3.5 updated: [SPARK-46535][SQL] Fix NPE when describe extended a column without col stats

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 6838f0db692 [SPARK-46535][SQL] Fix NPE when describe extended a column 
without col stats
6838f0db692 is described below

commit 6838f0db692892fe5ffdd86e4a59a8e9733d5d1b
Author: zouxxyy 
AuthorDate: Thu Dec 28 19:57:01 2023 +0300

[SPARK-46535][SQL] Fix NPE when describe extended a column without col stats

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

### Why are the changes needed?

Currently executing DESCRIBE TABLE EXTENDED a column without col stats with 
v2 table will throw a null pointer exception.

```text
Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
java.lang.NullPointerException: Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
at 
org.apache.spark.sql.execution.datasources.v2.DescribeColumnExec.run(DescribeColumnExec.scala:63)
at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:118)
at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:150)
at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:241)
at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:116)
at 
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:918)
```

This RP will fix it

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

### How was this patch tested?

Add a new test `describe extended (formatted) a column without col stats`

### Was this patch authored or co-authored using generative AI tooling?

Closes #44524 from Zouxxyy/dev/fix-stats.

Lead-authored-by: zouxxyy 
Co-authored-by: Kent Yao 
Signed-off-by: Max Gekk 
(cherry picked from commit af8228ce9aee99eae9d08dbdefaaad32cf5438eb)
Signed-off-by: Max Gekk 
---
 .../datasources/v2/DescribeColumnExec.scala |  2 +-
 .../execution/command/v2/DescribeTableSuite.scala   | 21 +
 2 files changed, 22 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
index 61ccda3fc95..2683d8d547f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
@@ -53,7 +53,7 @@ case class DescribeColumnExec(
   read.newScanBuilder(CaseInsensitiveStringMap.empty()).build() match {
 case s: SupportsReportStatistics =>
   val stats = s.estimateStatistics()
-  Some(stats.columnStats().get(FieldReference.column(column.name)))
+  
Option(stats.columnStats().get(FieldReference.column(column.name)))
 case _ => None
   }
 case _ => None
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
index e2f2aee5611..a21baebe24d 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
@@ -175,4 +175,25 @@ class DescribeTableSuite extends 
command.DescribeTableSuiteBase
   Row("max_col_len", "NULL")))
 }
   }
+
+  test("SPARK-46535: describe extended (formatted) a column without col 
stats") {
+withNamespaceAndTable("ns", "tbl") { tbl =>
+  sql(
+s"""
+   |CREATE TABLE $tbl
+   |(key INT COMMENT 'column_comment', col STRING)
+   |$defaultUsing""".stripMargin)
+
+  val descriptionDf = sql(s"DESCRIBE TABLE EXTENDED $tbl key")
+  assert(descriptionDf.schema.map(field => (field.name, field.dataType)) 
=== 

(spark) branch master updated: [SPARK-46535][SQL] Fix NPE when describe extended a column without col stats

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

maxgekk 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 af8228ce9ae [SPARK-46535][SQL] Fix NPE when describe extended a column 
without col stats
af8228ce9ae is described below

commit af8228ce9aee99eae9d08dbdefaaad32cf5438eb
Author: zouxxyy 
AuthorDate: Thu Dec 28 19:57:01 2023 +0300

[SPARK-46535][SQL] Fix NPE when describe extended a column without col stats

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

### Why are the changes needed?

Currently executing DESCRIBE TABLE EXTENDED a column without col stats with 
v2 table will throw a null pointer exception.

```text
Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
java.lang.NullPointerException: Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
at 
org.apache.spark.sql.execution.datasources.v2.DescribeColumnExec.run(DescribeColumnExec.scala:63)
at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:118)
at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:150)
at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:241)
at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:116)
at 
org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:918)
```

This RP will fix it

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

### How was this patch tested?

Add a new test `describe extended (formatted) a column without col stats`

### Was this patch authored or co-authored using generative AI tooling?

Closes #44524 from Zouxxyy/dev/fix-stats.

Lead-authored-by: zouxxyy 
Co-authored-by: Kent Yao 
Signed-off-by: Max Gekk 
---
 .../datasources/v2/DescribeColumnExec.scala |  2 +-
 .../execution/command/v2/DescribeTableSuite.scala   | 21 +
 2 files changed, 22 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
index 61ccda3fc95..2683d8d547f 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
@@ -53,7 +53,7 @@ case class DescribeColumnExec(
   read.newScanBuilder(CaseInsensitiveStringMap.empty()).build() match {
 case s: SupportsReportStatistics =>
   val stats = s.estimateStatistics()
-  Some(stats.columnStats().get(FieldReference.column(column.name)))
+  
Option(stats.columnStats().get(FieldReference.column(column.name)))
 case _ => None
   }
 case _ => None
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
index e2f2aee5611..a21baebe24d 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala
@@ -175,4 +175,25 @@ class DescribeTableSuite extends 
command.DescribeTableSuiteBase
   Row("max_col_len", "NULL")))
 }
   }
+
+  test("SPARK-46535: describe extended (formatted) a column without col 
stats") {
+withNamespaceAndTable("ns", "tbl") { tbl =>
+  sql(
+s"""
+   |CREATE TABLE $tbl
+   |(key INT COMMENT 'column_comment', col STRING)
+   |$defaultUsing""".stripMargin)
+
+  val descriptionDf = sql(s"DESCRIBE TABLE EXTENDED $tbl key")
+  assert(descriptionDf.schema.map(field => (field.name, field.dataType)) 
=== Seq(
+("info_name", StringType),
+("info_value", StringType)))
+  QueryTes

(spark) branch master updated: [SPARK-46537][SQL] Convert NPE and asserts from commands to internal errors

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

maxgekk 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 5db68247044 [SPARK-46537][SQL] Convert NPE and asserts from commands 
to internal errors
5db68247044 is described below

commit 5db68247044160342cd7a80c8903d3c3aa669f28
Author: Max Gekk 
AuthorDate: Thu Dec 28 17:38:51 2023 +0300

[SPARK-46537][SQL] Convert NPE and asserts from commands to internal errors

### What changes were proposed in this pull request?
In the PR, I propose to handle NPE and asserts from eagerly executed 
commands, and convert them to internal errors.

### Why are the changes needed?
To unify the approach for errors raised by Spark SQL.

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

Before the changes:
```
Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
java.lang.NullPointerException: Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
at 
org.apache.spark.sql.execution.datasources.v2.DescribeColumnExec.run(DescribeColumnExec.scala:63)
```

After:
```
org.apache.spark.SparkException: [INTERNAL_ERROR] Eagerly executed command 
failed. You hit a bug in Spark or the Spark plugins you use. Please, report 
this bug to the corresponding communities or vendors, and provide the full 
stack trace. SQLSTATE: XX000
at 
org.apache.spark.SparkException$.internalError(SparkException.scala:107)
...
Caused by: java.lang.NullPointerException: Cannot invoke 
"org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because 
the return value of "scala.Option.get()" is null
at 
org.apache.spark.sql.execution.datasources.v2.DescribeColumnExec.run(DescribeColumnExec.scala:63)
```

### How was this patch tested?
Manually, by running the test from another PR: 
https://github.com/apache/spark/pull/44524

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44525 from MaxGekk/internal-error-eagerlyExecuteCommands.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../main/scala/org/apache/spark/sql/execution/QueryExecution.scala | 7 +--
 1 file changed, 5 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index eb5b38d4288..a2cfad800e0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -114,8 +114,11 @@ class QueryExecution(
   // for eagerly executed commands we mark this place as beginning of 
execution.
   tracker.setReadyForExecution()
   val qe = sparkSession.sessionState.executePlan(c, 
CommandExecutionMode.NON_ROOT)
-  val result = SQLExecution.withNewExecutionId(qe, 
Some(commandExecutionName(c))) {
-qe.executedPlan.executeCollect()
+  val name = commandExecutionName(c)
+  val result = QueryExecution.withInternalError(s"Eagerly executed $name 
failed.") {
+SQLExecution.withNewExecutionId(qe, Some(name)) {
+  qe.executedPlan.executeCollect()
+}
   }
   CommandResult(
 qe.analyzed.output,


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



(spark) branch master updated: [SPARK-46532][CONNECT] Pass message parameters in metadata of `ErrorInfo`

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

maxgekk 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 6fcc2685cd3 [SPARK-46532][CONNECT] Pass message parameters in metadata 
of `ErrorInfo`
6fcc2685cd3 is described below

commit 6fcc2685cd3f9681dc85e0d53047f2e647d24b0b
Author: Max Gekk 
AuthorDate: Thu Dec 28 11:14:03 2023 +0300

[SPARK-46532][CONNECT] Pass message parameters in metadata of `ErrorInfo`

### What changes were proposed in this pull request?
In the PR, I propose to put message parameters together with an error class 
in the `messageParameter` field in metadata of `ErrorInfo`.

### Why are the changes needed?
To be able to create an error from an error class and message parameters. 
Before the changes, it is not possible to re-construct an error having only an 
error class.

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

### How was this patch tested?
By running the modified test:
```
$ build/sbt "connect-client-jvm/testOnly *ClientE2ETestSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44468 from MaxGekk/messageParameters-in-metadata.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/ClientE2ETestSuite.scala|  8 +++-
 .../spark/sql/connect/client/GrpcExceptionConverter.scala  |  4 
 .../org/apache/spark/sql/connect/config/Connect.scala  |  9 +
 .../org/apache/spark/sql/connect/utils/ErrorUtils.scala| 14 +++---
 python/pyspark/sql/tests/connect/test_connect_basic.py |  2 +-
 5 files changed, 32 insertions(+), 5 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
index c947d948b4c..0740334724e 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
@@ -85,7 +85,13 @@ class ClientE2ETestSuite extends RemoteSparkSession with 
SQLHelper with PrivateM
 |""".stripMargin)
 .collect()
 }
-assert(ex.getErrorClass != null)
+assert(
+  ex.getErrorClass ===
+"INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER")
+assert(
+  ex.getMessageParameters.asScala == Map(
+"datetime" -> "'02-29'",
+"config" -> "\"spark.sql.legacy.timeParserPolicy\""))
 if (enrichErrorEnabled) {
   assert(ex.getCause.isInstanceOf[DateTimeException])
 } else {
diff --git 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
index 075526e7521..cc47924de3b 100644
--- 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
+++ 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
@@ -372,10 +372,14 @@ private[client] object GrpcExceptionConverter {
   .addAllErrorTypeHierarchy(classes.toImmutableArraySeq.asJava)
 
 if (errorClass != null) {
+  val messageParameters = JsonMethods
+.parse(info.getMetadataOrDefault("messageParameters", "{}"))
+.extract[Map[String, String]]
   builder.setSparkThrowable(
 FetchErrorDetailsResponse.SparkThrowable
   .newBuilder()
   .setErrorClass(errorClass)
+  .putAllMessageParameters(messageParameters.asJava)
   .build())
 }
 
diff --git 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
index ab4f06d508a..39bf1a630af 100644
--- 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
+++ 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala
@@ -256,4 +256,13 @@ object Connect {
   .version("4.0.0")
   .booleanConf
   .createWithDefault(true)
+
+  val CONNECT_GRPC_MAX_METADATA_SIZE =
+buildStaticConf("spark.connect.grpc.maxMetadataSize")
+  .doc(
+"Sets the maximum size of metadata fields. For instance, it restricts 
metadata fields " +
+  "in `ErrorInfo`.")
+  .version("4.0.0")
+  .bytesConf(ByteUnit.BYTE)
+  .createWithDefault(1024)
 }

(spark) branch master updated: [SPARK-46519][SQL] Clear unused error classes from `error-classes.json` file

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

maxgekk 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 bb497eb7e8d [SPARK-46519][SQL] Clear unused error classes from 
`error-classes.json` file
bb497eb7e8d is described below

commit bb497eb7e8de4ec0f5acf75fb48e6f96c66e6bfc
Author: panbingkun 
AuthorDate: Thu Dec 28 11:12:44 2023 +0300

[SPARK-46519][SQL] Clear unused error classes from `error-classes.json` file

### What changes were proposed in this pull request?
The pr aims to:
- Clear unused error classes from `error-classes.json`.
- Delete unused methods `dataSourceAlreadyExists` in 
`QueryCompilationErrors.scala`
- Fix an outdated comment.

### Why are the changes needed?
Make code clear.

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

### How was this patch tested?
- Pass GA.
- Manually test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44503 from panbingkun/SPARK-46519.

Authored-by: panbingkun 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 32 --
 ...-error-conditions-invalid-handle-error-class.md |  4 ---
 docs/sql-error-conditions-sqlstates.md |  2 +-
 docs/sql-error-conditions.md   | 12 
 .../apache/spark/sql/errors/DataTypeErrors.scala   |  2 +-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  6 
 .../catalyst/catalog/ExternalCatalogSuite.scala|  7 +++--
 .../spark/sql/execution/QueryExecutionSuite.scala  |  4 ++-
 .../datasources/parquet/ParquetFilterSuite.scala   |  4 +--
 9 files changed, 11 insertions(+), 62 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 700b1ed0751..9f68d4c5a53 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -875,12 +875,6 @@
 ],
 "sqlState" : "42K01"
   },
-  "DATA_SOURCE_ALREADY_EXISTS" : {
-"message" : [
-  "Data source '' already exists in the registry. Please use a 
different name for the new data source."
-],
-"sqlState" : "42710"
-  },
   "DATA_SOURCE_NOT_EXIST" : {
 "message" : [
   "Data source '' not found. Please make sure the data source is 
registered."
@@ -1480,12 +1474,6 @@
 },
 "sqlState" : "42K0B"
   },
-  "INCORRECT_END_OFFSET" : {
-"message" : [
-  "Max offset with  rowsPerSecond is , but it's 
 now."
-],
-"sqlState" : "22003"
-  },
   "INCORRECT_RAMP_UP_RATE" : {
 "message" : [
   "Max offset with  rowsPerSecond is , but 
'rampUpTimeSeconds' is ."
@@ -1906,11 +1894,6 @@
   "Operation not found."
 ]
   },
-  "SESSION_ALREADY_EXISTS" : {
-"message" : [
-  "Session already exists."
-]
-  },
   "SESSION_CLOSED" : {
 "message" : [
   "Session was closed."
@@ -6065,11 +6048,6 @@
   "."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2142" : {
-"message" : [
-  "Attributes for type  is not supported."
-]
-  },
   "_LEGACY_ERROR_TEMP_2144" : {
 "message" : [
   "Unable to find constructor for . This could happen if  is an 
interface, or a trait without companion object constructor."
@@ -6920,11 +6898,6 @@
   ": "
 ]
   },
-  "_LEGACY_ERROR_TEMP_3066" : {
-"message" : [
-  ""
-]
-  },
   "_LEGACY_ERROR_TEMP_3067" : {
 "message" : [
   "Streaming aggregation doesn't support group aggregate pandas UDF"
@@ -6980,11 +6953,6 @@
   "More than one event time columns are available. Please ensure there is 
at most one event time column per stream. event time columns: "
 ]
   },
-  "_LEGACY_ERROR_TEMP_3078" : {
-"message" : [
-  "Can not match ParquetTable in the query."
-]
-  },
   "_LEGACY_ERROR_TEMP_3079" : {
 "message" : [
   "Dynamic partition cannot be the parent of a static partition."
diff --git a/docs/sql-error-conditions-invalid-handle-error-class.md 
b/docs/sql-error-conditions-invalid-handle-error-class.md
index 14526cd5372..8df8e54a8d9 100644
--- a/docs/sql-error-conditions-invalid-handle-error-class.md
+++ b/docs/sql-error-conditions-invalid-handle-error-class.md
@@ -41,10 +41,6 @@ Oper

(spark) branch master updated: [SPARK-46488][SQL] Skipping trimAll call during timestamp parsing

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

maxgekk 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 1a7a2f7a889f [SPARK-46488][SQL] Skipping trimAll call during timestamp 
parsing
1a7a2f7a889f is described below

commit 1a7a2f7a889fe0270318b304cd50c148729dd90b
Author: Stefan Kandic 
AuthorDate: Mon Dec 25 19:41:10 2023 +0300

[SPARK-46488][SQL] Skipping trimAll call during timestamp parsing

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

This PR is a follow up to 
[46173](https://github.com/apache/spark/pull/44110) which added skipping the 
trimAll calls during date parsing.

Now I'm doing the same just for timestamp parsing.

### Why are the changes needed?

These changes should drastically improve edge case where input string in 
cast to date has many whitespace as prefix/sufix.

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

No

### How was this patch tested?

I added the tests to check for cases with prefixes and suffixes of 
whitespaces and control chars.

Also there are benchmark tests in the previous 
[PR](https://github.com/apache/spark/pull/44110)

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #44463 from stefankandic/str2timeStamp-skipTrim.

Authored-by: Stefan Kandic 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/util/SparkDateTimeUtils.scala | 66 --
 .../sql/catalyst/util/DateTimeUtilsSuite.scala | 43 +-
 2 files changed, 79 insertions(+), 30 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
index 35118b449e2f..ed4d68f553f1 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
@@ -315,18 +315,11 @@ trait SparkDateTimeUtils {
 var currentSegmentValue = 0
 var currentSegmentDigits = 0
 val bytes = s.getBytes
-var j = 0
-var strEndTrimmed = bytes.length
+var j = getTrimmedStart(bytes)
+val strEndTrimmed = getTrimmedEnd(j, bytes)
 
-while (j < bytes.length && UTF8String.isWhitespaceOrISOControl(bytes(j))) {
-  j += 1;
-}
-if (j == bytes.length) {
-  return None;
-}
-
-while (strEndTrimmed > j && 
UTF8String.isWhitespaceOrISOControl(bytes(strEndTrimmed - 1))) {
-  strEndTrimmed -= 1;
+if (j == strEndTrimmed) {
+  return None
 }
 
 if (bytes(j) == '-' || bytes(j) == '+') {
@@ -418,7 +411,7 @@ trait SparkDateTimeUtils {
 (segment == 7 && digits <= 2) ||
 (segment != 0 && segment != 6 && segment != 7 && digits > 0 && digits 
<= 2)
 }
-if (s == null || s.trimAll().numBytes() == 0) {
+if (s == null) {
   return (Array.empty, None, false)
 }
 var tz: Option[String] = None
@@ -426,8 +419,14 @@ trait SparkDateTimeUtils {
 var i = 0
 var currentSegmentValue = 0
 var currentSegmentDigits = 0
-val bytes = s.trimAll().getBytes
-var j = 0
+val bytes = s.getBytes
+var j = getTrimmedStart(bytes)
+val strEndTrimmed = getTrimmedEnd(j, bytes)
+
+if (j == strEndTrimmed) {
+  return (Array.empty, None, false)
+}
+
 var digitsMilli = 0
 var justTime = false
 var yearSign: Option[Int] = None
@@ -435,7 +434,7 @@ trait SparkDateTimeUtils {
   yearSign = if (bytes(j) == '-') Some(-1) else Some(1)
   j += 1
 }
-while (j < bytes.length) {
+while (j < strEndTrimmed) {
   val b = bytes(j)
   val parsedValue = b - '0'.toByte
   if (parsedValue < 0 || parsedValue > 9) {
@@ -504,8 +503,8 @@ trait SparkDateTimeUtils {
 currentSegmentValue = 0
 currentSegmentDigits = 0
 i += 1
-tz = Some(new String(bytes, j, bytes.length - j))
-j = bytes.length - 1
+tz = Some(new String(bytes, j, strEndTrimmed - j))
+j = strEndTrimmed - 1
   }
   if (i == 6  && b != '.') {
 i += 1
@@ -619,6 +618,39 @@ trait SparkDateTimeUtils {
   case NonFatal(_) => None
 }
   }
+
+  /**
+   * Returns the index of the first non-whitespace and non-ISO control 
character in the byte array.
+   *
+   * @param bytes The byte array to be processed.
+   * @return The start index after trimming.
+   */
+  @inline private def getTrimmedStart(bytes: Array[Byte]) = {
+var start = 0
+
+while (start < bytes.length && 
UTF8String.isWhitespaceOrISOControl(bytes(start))) {
+  start += 1
+}
+
+start
+ 

(spark) branch master updated: [SPARK-46466][SQL] Vectorized parquet reader should never do rebase for timestamp ntz

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

maxgekk 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 4d21e5547580 [SPARK-46466][SQL] Vectorized parquet reader should never 
do rebase for timestamp ntz
4d21e5547580 is described below

commit 4d21e55475807a089979cffb54076bcb3ae9c02d
Author: Wenchen Fan 
AuthorDate: Thu Dec 21 12:42:19 2023 +0300

[SPARK-46466][SQL] Vectorized parquet reader should never do rebase for 
timestamp ntz

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

This fixes a correctness bug. The TIMESTAMP_NTZ is a new data type in Spark 
and has no legacy files that need to do calendar rebase. However, the 
vectorized parquet reader treat it the same as LTZ and may do rebase if the 
parquet file was written with the legacy rebase mode. This PR fixes it to never 
do rebase for NTZ.

### Why are the changes needed?

bug fix

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

Yes, now we can correctly write and read back NTZ value even if the date is 
before 1582.

### How was this patch tested?

new test

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #44428 from cloud-fan/ntz.

Lead-authored-by: Wenchen Fan 
Co-authored-by: Wenchen Fan 
Signed-off-by: Max Gekk 
---
 .../parquet/ParquetVectorUpdaterFactory.java   | 27 --
 .../datasources/parquet/ParquetQuerySuite.scala| 12 ++
 2 files changed, 27 insertions(+), 12 deletions(-)

diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
index 918f21716f45..31a1957b4fb9 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
@@ -109,24 +109,32 @@ public class ParquetVectorUpdaterFactory {
   // For unsigned int64, it stores as plain signed int64 in Parquet 
when dictionary
   // fallbacks. We read them as decimal values.
   return new UnsignedLongUpdater();
-} else if (isTimestamp(sparkType) &&
+} else if (sparkType == DataTypes.TimestampType &&
   isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
-  validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongUpdater();
   } else {
 boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode);
 return new LongWithRebaseUpdater(failIfRebase, datetimeRebaseTz);
   }
-} else if (isTimestamp(sparkType) &&
+} else if (sparkType == DataTypes.TimestampType &&
   isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
-  validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongAsMicrosUpdater();
   } else {
 final boolean failIfRebase = 
"EXCEPTION".equals(datetimeRebaseMode);
 return new LongAsMicrosRebaseUpdater(failIfRebase, 
datetimeRebaseTz);
   }
+} else if (sparkType == DataTypes.TimestampNTZType &&
+  isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
+  validateTimestampNTZType();
+  // TIMESTAMP_NTZ is a new data type and has no legacy files that 
need to do rebase.
+  return new LongUpdater();
+} else if (sparkType == DataTypes.TimestampNTZType &&
+  isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
+  validateTimestampNTZType();
+  // TIMESTAMP_NTZ is a new data type and has no legacy files that 
need to do rebase.
+  return new LongAsMicrosUpdater();
 } else if (sparkType instanceof DayTimeIntervalType) {
   return new LongUpdater();
 }
@@ -195,12 +203,11 @@ public class ParquetVectorUpdaterFactory {
   annotation.getUnit() == unit;
   }
 
-  void validateTimestampType(DataType sparkType) {
+  private void validateTimestampNTZType() {
 assert(logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation);
-// Throw an exception if the Parquet type is TimestampLTZ and the Catalyst 
type is TimestampNTZ.
+// Throw an exception if the Parquet type is TimestampLTZ as the Catalyst 
type is TimestampNTZ.
 // This is to avoid mistakes in reading the timestamp values.
-if (((TimestampLogicalTypeAnnotation) 
logicalTypeAnnotation).isAdjustedToUTC() &&

(spark) branch master updated: [SPARK-46447][SQL] Remove the legacy datetime rebasing SQL configs

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

maxgekk 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 1005cd5576ef [SPARK-46447][SQL] Remove the legacy datetime rebasing 
SQL configs
1005cd5576ef is described below

commit 1005cd5576ef073afee243848bcad5e5f4a9d309
Author: Max Gekk 
AuthorDate: Wed Dec 20 20:22:09 2023 +0300

[SPARK-46447][SQL] Remove the legacy datetime rebasing SQL configs

### What changes were proposed in this pull request?
In the PR, I propose to remove already deprecated SQL configs (alternatives 
to other configs):
- spark.sql.legacy.parquet.int96RebaseModeInWrite
- spark.sql.legacy.parquet.datetimeRebaseModeInWrite
- spark.sql.legacy.parquet.int96RebaseModeInRead
- spark.sql.legacy.avro.datetimeRebaseModeInWrite
- spark.sql.legacy.avro.datetimeRebaseModeInRead

### Why are the changes needed?
To improve code maintenance.

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

### How was this patch tested?
By existing test suites.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44402 from MaxGekk/remove-legacy-rebase-confs-2.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 docs/sql-migration-guide.md|  6 
 .../org/apache/spark/sql/internal/SQLConf.scala| 36 +++---
 2 files changed, 24 insertions(+), 18 deletions(-)

diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 4e8e2422d7e0..30a37d97042a 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -30,6 +30,12 @@ license: |
 - Since Spark 4.0, `spark.sql.parquet.compression.codec` drops the support of 
codec name `lz4raw`, please use `lz4_raw` instead.
 - Since Spark 4.0, when overflowing during casting timestamp to byte/short/int 
under non-ansi mode, Spark will return null instead a wrapping value.
 - Since Spark 4.0, the `encode()` and `decode()` functions support only the 
following charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 
'UTF-16'. To restore the previous behavior when the function accepts charsets 
of the current JDK used by Spark, set `spark.sql.legacy.javaCharsets` to `true`.
+- Since Spark 4.0, the legacy datetime rebasing SQL configs with the prefix 
`spark.sql.legacy` are removed. To restore the previous behavior, use the 
following configs:
+  - `spark.sql.parquet.int96RebaseModeInWrite` instead of 
`spark.sql.legacy.parquet.int96RebaseModeInWrite`
+  - `spark.sql.parquet.datetimeRebaseModeInWrite` instead of 
`spark.sql.legacy.parquet.datetimeRebaseModeInWrite`
+  - `spark.sql.parquet.int96RebaseModeInRead` instead of 
`spark.sql.legacy.parquet.int96RebaseModeInRead`
+  - `spark.sql.avro.datetimeRebaseModeInWrite` instead of 
`spark.sql.legacy.avro.datetimeRebaseModeInWrite`
+  - `spark.sql.avro.datetimeRebaseModeInRead` instead of 
`spark.sql.legacy.avro.datetimeRebaseModeInRead`
 
 ## Upgrading from Spark SQL 3.4 to 3.5
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 6404779f30ac..d54cb3756638 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -4081,7 +4081,6 @@ object SQLConf {
 "When EXCEPTION, which is the default, Spark will fail the writing if 
it sees ancient " +
 "timestamps that are ambiguous between the two calendars.")
   .version("3.1.0")
-  .withAlternative("spark.sql.legacy.parquet.int96RebaseModeInWrite")
   .stringConf
   .transform(_.toUpperCase(Locale.ROOT))
   .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
@@ -4099,7 +4098,6 @@ object SQLConf {
 "TIMESTAMP_MILLIS, TIMESTAMP_MICROS. The INT96 type has the separate 
config: " +
 s"${PARQUET_INT96_REBASE_MODE_IN_WRITE.key}.")
   .version("3.0.0")
-  .withAlternative("spark.sql.legacy.parquet.datetimeRebaseModeInWrite")
   .stringConf
   .transform(_.toUpperCase(Locale.ROOT))
   .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
@@ -4115,7 +4113,6 @@ object SQLConf {
 "timestamps that are ambiguous between the two calendars. This config 
is only effective " +
 "if the writer info (like Spark, Hive) of the Parquet files is 
unknown.")
   .version("3.1.0")
-  .withAlternative("spark.sql.legacy.parquet.int96RebaseModeInRead")
   .stringConf
   .transform(_.toUpperCase(Locale.ROOT))
   .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
@@ -41

(spark) branch master updated: [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support

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

maxgekk 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 aa1ff3789e49 [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support
aa1ff3789e49 is described below

commit aa1ff3789e492545b07d84ac095fc4c39f7446c6
Author: milastdbx 
AuthorDate: Tue Dec 19 19:04:26 2023 +0300

[SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support

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

Introducing new EXECUTE IMMEDIATE syntax to support parameterized queries 
from within SQL.

This API executes query passed as string with arguments.

Other DBs that support this:
 - 
[Oracle](https://docs.oracle.com/cd/B13789_01/appdev.101/b10807/13_elems017.htm)
 - 
[Snowflake](https://docs.snowflake.com/en/sql-reference/sql/execute-immediate)
 - 
[PgSql](https://www.postgresql.org/docs/current/ecpg-sql-execute-immediate.html#:~:text=Description,statement%2C%20without%20retrieving%20result%20rows.)

### Why are the changes needed?
Often time queries are constructed as a result of other queries. We need a 
way to dynamically construct a query and execute it without leaving SQL API.

### Does this PR introduce _any_ user-facing change?
Yes, it exposes new syntax

### How was this patch tested?
Golden files

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #44093 from milastdbx/dev/executeImmediate.

Lead-authored-by: milastdbx 
Co-authored-by: Milan Stefanovic 
<150366084+milast...@users.noreply.github.com>
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json|  44 ++
 docs/sql-error-conditions.md   |  44 ++
 docs/sql-ref-ansi-compliance.md|   2 +
 docs/sql-ref-syntax-aux-exec-imm.md|  86 +++
 docs/sql-ref-syntax.md |   1 +
 .../spark/sql/catalyst/parser/SqlBaseLexer.g4  |   2 +
 .../spark/sql/catalyst/parser/SqlBaseParser.g4 |  26 +
 .../spark/sql/errors/QueryParsingErrors.scala  |   9 +
 .../spark/sql/catalyst/analysis/Analyzer.scala |   1 +
 .../sql/catalyst/analysis/executeImmediate.scala   | 189 +
 .../spark/sql/catalyst/analysis/parameters.scala   |   5 +-
 .../sql/catalyst/catalog/SessionCatalog.scala  |   2 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala |  59 ++
 .../sql/catalyst/rules/RuleIdCollection.scala  |   1 +
 .../spark/sql/catalyst/trees/TreePatterns.scala|   1 +
 .../spark/sql/errors/QueryCompilationErrors.scala  |  50 ++
 .../sql/catalyst/analysis/AnalysisErrorSuite.scala |  70 ++
 .../sql/catalyst/analysis/AnalysisSuite.scala  |  24 +
 .../spark/sql/catalyst/analysis/AnalysisTest.scala |   3 +
 .../analyzer-results/execute-immediate.sql.out | 782 +
 .../sql-tests/inputs/execute-immediate.sql | 138 
 .../sql-tests/results/ansi/keywords.sql.out|   3 +
 .../sql-tests/results/execute-immediate.sql.out| 738 +++
 .../resources/sql-tests/results/keywords.sql.out   |   2 +
 .../spark/sql/errors/QueryParsingErrorsSuite.scala |  27 +
 .../ThriftServerWithSparkContextSuite.scala|   2 +-
 26 files changed, 2307 insertions(+), 4 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 62e3427fdffd..30aacc07d318 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -5,6 +5,12 @@
 ],
 "sqlState" : "42845"
   },
+  "ALL_PARAMETERS_MUST_BE_NAMED" : {
+"message" : [
+  "Using name parameterized queries requires all parameters to be named. 
Parameters missing names: ."
+],
+"sqlState" : "07001"
+  },
   "ALL_PARTITION_COLUMNS_NOT_ALLOWED" : {
 "message" : [
   "Cannot use all columns for partition columns."
@@ -1024,6 +1030,12 @@
 ],
 "sqlState" : "42702"
   },
+  "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : {
+"message" : [
+  "The USING clause of this EXECUTE IMMEDIATE command contained multiple 
arguments with same alias (), which is invalid; please update the 
command to specify unique aliases and then try it again."
+],
+"sqlState" : "42701"
+  },
   "EXPECT_PERMANENT_VIEW_NOT_TEMP" : {
 "message" : [
   "'' expects a permanent view but  is a temp view."
@@ -2230,6 +2242,12 @@
 ],
 "sqlState" : "42602"
   },
+  "INVALID_QUERY_MIXED_QUERY_PARAMETERS" : {
+"message" : [
+ 

(spark) branch master updated: [SPARK-46454][SQL][DSTREAM] Remove redundant `.headOption`

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

maxgekk 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 ea7d00a8424b [SPARK-46454][SQL][DSTREAM] Remove redundant `.headOption`
ea7d00a8424b is described below

commit ea7d00a8424b9369a5e8807dce29718a3450b28a
Author: yangjie01 
AuthorDate: Tue Dec 19 17:07:42 2023 +0300

[SPARK-46454][SQL][DSTREAM] Remove redundant `.headOption`

### What changes were proposed in this pull request?
This pr just remove redundant `.headOption` due to ` 
Option(xxx).headOption` is a redundant call.

### Why are the changes needed?
Remove redundant `.headOption`

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

### How was this patch tested?
Pass GitHub Actions

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #44411 from LuciferYang/redundant-headOption.

Authored-by: yangjie01 
Signed-off-by: Max Gekk 
---
 .../src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala  | 2 +-
 .../main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala   | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala
index 6e7d8a058ae1..4ac62b987b15 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala
@@ -95,7 +95,7 @@ object StructFilters {
   }
 
   private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = {
-a.zip(b).headOption
+a.zip(b)
   }
 
   private def toLiteral(value: Any): Option[Literal] = {
diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala 
b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
index 5b2b959f8138..57009570b257 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
@@ -53,14 +53,14 @@ case class BatchInfo(
* processing. Essentially, it is `processingEndTime` - 
`processingStartTime`.
*/
   def processingDelay: Option[Long] = 
processingEndTime.zip(processingStartTime)
-.map(x => x._1 - x._2).headOption
+.map(x => x._1 - x._2)
 
   /**
* Time taken for all the jobs of this batch to finish processing from the 
time they
* were submitted.  Essentially, it is `processingDelay` + `schedulingDelay`.
*/
   def totalDelay: Option[Long] = schedulingDelay.zip(processingDelay)
-.map(x => x._1 + x._2).headOption
+.map(x => x._1 + x._2)
 
   /**
* The number of recorders received by the receivers in this batch.


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



(spark) branch branch-3.5 updated: [SPARK-46453][CONNECT] Throw exception from `internalError()` in `SessionHolder`

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new cc4f5787414e [SPARK-46453][CONNECT] Throw exception from 
`internalError()` in `SessionHolder`
cc4f5787414e is described below

commit cc4f5787414e4392499a349dec5b24c8e25e50f3
Author: Max Gekk 
AuthorDate: Tue Dec 19 12:21:20 2023 +0300

[SPARK-46453][CONNECT] Throw exception from `internalError()` in 
`SessionHolder`

### What changes were proposed in this pull request?
In the PR, I propose to throw `SparkException` returned by `internalError` 
in `SessionHolder`.

### Why are the changes needed?
Without the bug fix user won't see the internal error.

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

### How was this patch tested?
N/a

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44400 from MaxGekk/throw-internal-error.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit dc0bfc4c700c347f2f58625facec8c5771bde59a)
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/connect/service/SessionHolder.scala| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
index 1cef02d7e346..218819d114c1 100644
--- 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
+++ 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
@@ -197,7 +197,7 @@ case class SessionHolder(userId: String, sessionId: String, 
session: SparkSessio
*/
   private[connect] def cacheDataFrameById(dfId: String, df: DataFrame): Unit = 
{
 if (dataFrameCache.putIfAbsent(dfId, df) != null) {
-  SparkException.internalError(s"A dataframe is already associated with id 
$dfId")
+  throw SparkException.internalError(s"A dataframe is already associated 
with id $dfId")
 }
   }
 
@@ -221,7 +221,7 @@ case class SessionHolder(userId: String, sessionId: String, 
session: SparkSessio
*/
   private[connect] def cacheListenerById(id: String, listener: 
StreamingQueryListener): Unit = {
 if (listenerCache.putIfAbsent(id, listener) != null) {
-  SparkException.internalError(s"A listener is already associated with id 
$id")
+  throw SparkException.internalError(s"A listener is already associated 
with id $id")
 }
   }
 


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



(spark) branch master updated: [SPARK-46453][CONNECT] Throw exception from `internalError()` in `SessionHolder`

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

maxgekk 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 dc0bfc4c700c [SPARK-46453][CONNECT] Throw exception from 
`internalError()` in `SessionHolder`
dc0bfc4c700c is described below

commit dc0bfc4c700c347f2f58625facec8c5771bde59a
Author: Max Gekk 
AuthorDate: Tue Dec 19 12:21:20 2023 +0300

[SPARK-46453][CONNECT] Throw exception from `internalError()` in 
`SessionHolder`

### What changes were proposed in this pull request?
In the PR, I propose to throw `SparkException` returned by `internalError` 
in `SessionHolder`.

### Why are the changes needed?
Without the bug fix user won't see the internal error.

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

### How was this patch tested?
N/a

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44400 from MaxGekk/throw-internal-error.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/connect/service/SessionHolder.scala| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
index f097f2db5889..427b1a50588c 100644
--- 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
+++ 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala
@@ -307,7 +307,7 @@ case class SessionHolder(userId: String, sessionId: String, 
session: SparkSessio
*/
   private[connect] def cacheDataFrameById(dfId: String, df: DataFrame): Unit = 
{
 if (dataFrameCache.putIfAbsent(dfId, df) != null) {
-  SparkException.internalError(s"A dataframe is already associated with id 
$dfId")
+  throw SparkException.internalError(s"A dataframe is already associated 
with id $dfId")
 }
   }
 
@@ -331,7 +331,7 @@ case class SessionHolder(userId: String, sessionId: String, 
session: SparkSessio
*/
   private[connect] def cacheListenerById(id: String, listener: 
StreamingQueryListener): Unit = {
 if (listenerCache.putIfAbsent(id, listener) != null) {
-  SparkException.internalError(s"A listener is already associated with id 
$id")
+  throw SparkException.internalError(s"A listener is already associated 
with id $id")
 }
   }
 


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



(spark) branch master updated: [SPARK-45725][SQL][FOLLOWUP] Fix arguments of a removed SQL config

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

maxgekk 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 036af4bf2336 [SPARK-45725][SQL][FOLLOWUP] Fix arguments of a removed 
SQL config
036af4bf2336 is described below

commit 036af4bf23361858d4de2429c9312828cd74fdf2
Author: Max Gekk 
AuthorDate: Mon Dec 18 17:38:29 2023 +0300

[SPARK-45725][SQL][FOLLOWUP] Fix arguments of a removed SQL config

### What changes were proposed in this pull request?
In the PR, I propose to fix the order of arguments of the removed SQL 
config `park.sql.optimizer.runtimeFilter.semiJoinReduction.enabled` and check 
the format of Spark version in the constructor of `RemovedConfig`.

### Why are the changes needed?
To don't confuse users and prevent such kind of mistakes.

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

### How was this patch tested?
By running the existing test suite:
```
$ build/sbt "test:testOnly *SQLConfSuite"
```
without the fix the test suite fails with the internal error:
```
Caused by: org.apache.spark.SparkException: [INTERNAL_ERROR] The removed 
SQL config spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled has the 
wrong Spark version: false SQLSTATE: XX000
at 
org.apache.spark.SparkException$.internalError(SparkException.scala:92)
at 
org.apache.spark.SparkException$.internalError(SparkException.scala:96)
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44399 from MaxGekk/followup-semiJoinReduction-conf-2.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../main/scala/org/apache/spark/sql/internal/SQLConf.scala  | 13 +
 1 file changed, 9 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 23217f94d64e..448474ae2faa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -31,7 +31,7 @@ import scala.util.matching.Regex
 
 import org.apache.hadoop.fs.Path
 
-import org.apache.spark.{ErrorMessageFormat, SparkConf, SparkContext, 
TaskContext}
+import org.apache.spark.{ErrorMessageFormat, SparkConf, SparkContext, 
SparkException, TaskContext}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
 import org.apache.spark.io.CompressionCodec
@@ -47,7 +47,7 @@ import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors
 import org.apache.spark.sql.types.{AtomicType, TimestampNTZType, TimestampType}
 import org.apache.spark.storage.{StorageLevel, StorageLevelMapper}
 import org.apache.spark.unsafe.array.ByteArrayMethods
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{Utils, VersionUtils}
 
 

 // This file defines the configuration options for Spark SQL.
@@ -4675,7 +4675,12 @@ object SQLConf {
* users that they set non-default value to an already 
removed config.
* @param comment Additional info regarding to the removed config.
*/
-  case class RemovedConfig(key: String, version: String, defaultValue: String, 
comment: String)
+  case class RemovedConfig(key: String, version: String, defaultValue: String, 
comment: String) {
+if (VersionUtils.majorMinorPatchVersion(version).isEmpty) {
+  throw SparkException.internalError(
+s"The removed SQL config $key has the wrong Spark version: $version")
+}
+  }
 
   /**
* The map contains info about removed SQL configs. Keys are SQL config 
names,
@@ -4713,7 +4718,7 @@ object SQLConf {
   "for more details."),
   RemovedConfig("spark.sql.hive.verifyPartitionPath", "4.0.0", "false",
 s"This config was replaced by '${IGNORE_MISSING_FILES.key}'."),
-  
RemovedConfig("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled", 
"false", "4.0",
+  
RemovedConfig("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled", 
"4.0.0", "false",
 "This optimizer config is useless as runtime filter cannot be an IN 
subquery now.")
 )
 


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



(spark) branch master updated: [SPARK-46440][SQL] Set the rebase configs to the `CORRECTED` mode by default

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

maxgekk 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 bfafad4d47b4 [SPARK-46440][SQL] Set the rebase configs to the 
`CORRECTED` mode by default
bfafad4d47b4 is described below

commit bfafad4d47b4f60e93d17ccc3a8dcc8bae03cf9a
Author: Max Gekk 
AuthorDate: Mon Dec 18 11:41:29 2023 +0300

[SPARK-46440][SQL] Set the rebase configs to the `CORRECTED` mode by default

### What changes were proposed in this pull request?
In the PR, I propose to set all rebase related SQL configs to the 
`CORRECTED` mode by default. Here are the affected configs:
- spark.sql.parquet.int96RebaseModeInWrite
- spark.sql.parquet.datetimeRebaseModeInWrite
- spark.sql.parquet.int96RebaseModeInRead
- spark.sql.parquet.datetimeRebaseModeInRead
- spark.sql.avro.datetimeRebaseModeInWrite
- spark.sql.avro.datetimeRebaseModeInRead

### Why are the changes needed?
The configs were set to the `EXCEPTION` mode to give users a choice to 
select proper mode for compatibility with old Spark versions <= 2.4.5. Those 
versions are not able to detect the rebase mode from meta information in 
parquet and avro files. Since the versions are out of broad usage, Spark 
starting from the version 4.0.0 will write/ read ancient datatime without 
rebasing and any exceptions. This should be more convenient for users.

### Does this PR introduce _any_ user-facing change?
Yes, it can if user's code expects an exception while reading/writing 
ancient datetimes.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "test:testOnly *ParquetRebaseDatetimeV1Suite"
$ build/sbt "test:testOnly *ParquetRebaseDatetimeV2Suite"
$ build/sbt "test:testOnly *RebaseDateTimeSuite"
$ build/sbt "test:testOnly *AvroV1Suite"
$ build/sbt "test:testOnly *AvroV2Suite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44388 from MaxGekk/set-rebase-conf-corrected.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../test/scala/org/apache/spark/sql/avro/AvroSuite.scala | 16 ++--
 .../scala/org/apache/spark/sql/internal/SQLConf.scala| 12 ++--
 .../datasources/parquet/ParquetRebaseDatetimeSuite.scala |  6 +++---
 3 files changed, 19 insertions(+), 15 deletions(-)

diff --git 
a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala 
b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
index e904ecdb3886..8a5634235639 100644
--- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
+++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
@@ -2347,13 +2347,17 @@ abstract class AvroSuite
   |  ]
   |}""".stripMargin
 
-  // By default we should fail to write ancient datetime values.
-  val e = intercept[SparkException] {
-df.write.format("avro").option("avroSchema", 
avroSchema).save(path3_x)
+  withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> 
EXCEPTION.toString) {
+val e = intercept[SparkException] {
+  df.write.format("avro").option("avroSchema", 
avroSchema).save(path3_x)
+}
+assert(e.getCause.getCause.isInstanceOf[SparkUpgradeException])
   }
-  assert(e.getCause.getCause.isInstanceOf[SparkUpgradeException])
   checkDefaultLegacyRead(oldPath)
 
+  // By default we should not fail to write ancient datetime values.
+  df.write.format("avro").option("avroSchema", 
avroSchema).mode("overwrite").save(path3_x)
+
   withSQLConf(SQLConf.AVRO_REBASE_MODE_IN_WRITE.key -> 
CORRECTED.toString) {
 df.write.format("avro").option("avroSchema", 
avroSchema).mode("overwrite").save(path3_x)
   }
@@ -2378,9 +2382,9 @@ abstract class AvroSuite
 }
 def successInRead(path: String): Unit = 
spark.read.format("avro").load(path).collect()
 Seq(
-  // By default we should fail to read ancient datetime values when 
parquet files don't
+  // By default we should not fail to read ancient datetime values when 
parquet files don't
   // contain Spark version.
-  "2_4_5" -> failInRead _,
+  "2_4_5" -> successInRead _,
   "2_4_6" -> successInRead _,
   "3_2_0" -> successInRead _
 ).foreach { case (version, checkDefaultRead) =>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sq

(spark) branch master updated: [SPARK-46410][SQL] Assign error classes/subclasses to JdbcUtils.classifyException

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

maxgekk 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 14a933bbe2eb [SPARK-46410][SQL] Assign error classes/subclasses to 
JdbcUtils.classifyException
14a933bbe2eb is described below

commit 14a933bbe2eb1c71988f475036735f07b2e1fa6a
Author: Max Gekk 
AuthorDate: Sun Dec 17 10:41:07 2023 +0300

[SPARK-46410][SQL] Assign error classes/subclasses to 
JdbcUtils.classifyException

### What changes were proposed in this pull request?
In the PR, I propose to raise exceptions with only error classes from 
`JdbcUtils.classifyException`, and introduce new error class `FAILED_JDBC` with 
sub-classes linked to a particular JDBC operation.

### Why are the changes needed?
To improve user experience with Spark SQL by migrating on new error 
framework when all Spark exceptions from the JDBC datasource have an error 
class.

### Does this PR introduce _any_ user-facing change?
Yes, if user's code depends on exceptions from the JDBC datasource.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "test:testOnly *JDBCV2Suite"
$ build/sbt "test:testOnly *JDBCTableCatalogSuite"
$ build/sbt "test:testOnly *JDBCSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44358 from MaxGekk/error-class-classifyException.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json|  78 -
 .../org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala  |  10 +-
 ...sql-error-conditions-failed-jdbc-error-class.md |  80 +
 docs/sql-error-conditions.md   |   8 +
 project/MimaExcludes.scala |   3 +
 .../sql/catalyst/analysis/NonEmptyException.scala  |   3 -
 .../sql/execution/datasources/jdbc/JdbcUtils.scala |   7 +-
 .../execution/datasources/v2/jdbc/JDBCTable.scala  |  25 ++-
 .../datasources/v2/jdbc/JDBCTableCatalog.scala |  86 --
 .../org/apache/spark/sql/jdbc/DB2Dialect.scala |  15 +-
 .../org/apache/spark/sql/jdbc/H2Dialect.scala  |  33 ++--
 .../org/apache/spark/sql/jdbc/JdbcDialects.scala   |  13 +-
 .../apache/spark/sql/jdbc/MsSqlServerDialect.scala |  15 +-
 .../org/apache/spark/sql/jdbc/MySQLDialect.scala   |  28 ++-
 .../apache/spark/sql/jdbc/PostgresDialect.scala|  57 ---
 .../v2/jdbc/JDBCTableCatalogSuite.scala| 188 +++--
 .../org/apache/spark/sql/jdbc/JDBCV2Suite.scala|   6 +-
 17 files changed, 501 insertions(+), 154 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index b4a3031c06c9..62e3427fdffd 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1096,6 +1096,79 @@
 ],
 "sqlState" : "38000"
   },
+  "FAILED_JDBC" : {
+"message" : [
+  "Failed JDBC  on the operation:"
+],
+"subClass" : {
+  "ALTER_TABLE" : {
+"message" : [
+  "Alter the table ."
+]
+  },
+  "CREATE_INDEX" : {
+"message" : [
+  "Create the index  in the  table."
+]
+  },
+  "CREATE_NAMESPACE" : {
+"message" : [
+  "Create the namespace ."
+]
+  },
+  "CREATE_NAMESPACE_COMMENT" : {
+"message" : [
+  "Create a comment on the namespace: ."
+]
+  },
+  "CREATE_TABLE" : {
+"message" : [
+  "Create the table ."
+]
+  },
+  "DROP_INDEX" : {
+"message" : [
+  "Drop the index  in the  table."
+]
+  },
+  "DROP_NAMESPACE" : {
+"message" : [
+  "Drop the namespace ."
+]
+  },
+  "GET_TABLES" : {
+"message" : [
+  "Get tables from the namespace: ."
+]
+  },
+  "LIST_NAMESPACES" : {
+"message" : [
+  "List namespaces."
+]
+  },
+  "NAMESPACE_EXISTS" : {
+"message" : [
+  "Check that the namespace  exists."
+]
+  },
+  "REMOVE_NAMESPACE_COMMENT" : {
+"message" : [
+  "Remove a comment on the namespace: ."
+]
+  },
+  "RENAME_TABLE" : {
+"message" : [
+

(spark) branch master updated: [SPARK-46393][SQL] Classify exceptions in the JDBC table catalog

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

maxgekk 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 f1e5a136fa79 [SPARK-46393][SQL] Classify exceptions in the JDBC table 
catalog
f1e5a136fa79 is described below

commit f1e5a136fa79449878bb3bd2cc304c9dde020ce8
Author: Max Gekk 
AuthorDate: Thu Dec 14 11:01:34 2023 +0300

[SPARK-46393][SQL] Classify exceptions in the JDBC table catalog

### What changes were proposed in this pull request?
In the PR, I propose to handle exceptions from JDBC drivers in the JDBC 
table catalog, classify them and converted to appropriate Spark exception w/ an 
error class. This PR covers the following functions where such errors haven't 
been classified yet:
- list tables
- namespace exists
- list namespaces

### Why are the changes needed?
To unify Spark exceptions, and migrate onto new error framework.

### Does this PR introduce _any_ user-facing change?
Yes, if user code expects that Spark SQL bypass Java exceptions from JDBC 
drivers.

### How was this patch tested?
By existing test suites:
```
$ build/sbt "test:testOnly *JDBCV2Suite"
$ build/sbt "test:testOnly *JDBCTableCatalogSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

    Closes #44335 from MaxGekk/classifyException-JDBCTableCatalog.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../execution/datasources/v2/jdbc/JDBCTableCatalog.scala   | 14 ++
 1 file changed, 10 insertions(+), 4 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
index 0084abb392ef..6c773d4fb1b0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala
@@ -65,8 +65,10 @@ class JDBCTableCatalog extends TableCatalog
 checkNamespace(namespace)
 JdbcUtils.withConnection(options) { conn =>
   val schemaPattern = if (namespace.length == 1) namespace.head else null
-  val rs = conn.getMetaData
-.getTables(null, schemaPattern, "%", Array("TABLE"))
+  val rs = JdbcUtils.classifyException(
+s"Failed get tables from: ${namespace.mkString(".")}", dialect) {
+conn.getMetaData.getTables(null, schemaPattern, "%", Array("TABLE"))
+  }
   new Iterator[Identifier] {
 def hasNext = rs.next()
 def next() = Identifier.of(namespace, rs.getString("TABLE_NAME"))
@@ -179,14 +181,18 @@ class JDBCTableCatalog extends TableCatalog
   override def namespaceExists(namespace: Array[String]): Boolean = namespace 
match {
 case Array(db) =>
   JdbcUtils.withConnection(options) { conn =>
-JdbcUtils.schemaExists(conn, options, db)
+JdbcUtils.classifyException(s"Failed namespace exists: 
${namespace.mkString}", dialect) {
+  JdbcUtils.schemaExists(conn, options, db)
+}
   }
 case _ => false
   }
 
   override def listNamespaces(): Array[Array[String]] = {
 JdbcUtils.withConnection(options) { conn =>
-  JdbcUtils.listSchemas(conn, options)
+  JdbcUtils.classifyException(s"Failed list namespaces", dialect) {
+JdbcUtils.listSchemas(conn, options)
+  }
 }
   }
 


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



(spark) branch master updated: [SPARK-46381][SQL] Migrate sub-classes of `AnalysisException` to error classes

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

maxgekk 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 b3398e695d92 [SPARK-46381][SQL] Migrate sub-classes of 
`AnalysisException` to error classes
b3398e695d92 is described below

commit b3398e695d929c7f867d408c28fb274509c9854c
Author: Max Gekk 
AuthorDate: Wed Dec 13 12:28:31 2023 +0300

[SPARK-46381][SQL] Migrate sub-classes of `AnalysisException` to error 
classes

### What changes were proposed in this pull request?
In the PR, I propose to migrate the rest two sub-classes of 
`AnalysisException` onto error classes:
- NonEmptyNamespaceException
- ExtendedAnalysisException

and forbid raising of such exception without an error class.

### Why are the changes needed?
This is a part of the migration on the error framework, and unifying errors 
in Spark.

### Does this PR introduce _any_ user-facing change?
Yes, if user's code depends on the format of error messages.

### How was this patch tested?
By existing test suites like:
```
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44314 from MaxGekk/error-class-ExtendedAnalysisException.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../utils/src/main/resources/error/error-classes.json  | 15 +++
 .../sql/catalyst/analysis/NonEmptyException.scala  | 18 --
 .../spark/sql/catalyst/ExtendedAnalysisException.scala |  2 +-
 .../sql/catalyst/analysis/ResolveTimeWindows.scala |  7 +--
 .../analysis/UnsupportedOperationChecker.scala |  5 -
 .../spark/sql/errors/QueryCompilationErrors.scala  |  9 +++--
 .../scala/org/apache/spark/sql/jdbc/DB2Dialect.scala   |  2 +-
 .../org/apache/spark/sql/jdbc/MsSqlServerDialect.scala |  2 +-
 .../org/apache/spark/sql/jdbc/PostgresDialect.scala|  2 +-
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  8 ++--
 10 files changed, 49 insertions(+), 21 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index d52ffc011b72..2aa5420eb22c 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -6958,6 +6958,21 @@
   ""
 ]
   },
+  "_LEGACY_ERROR_TEMP_3101" : {
+"message" : [
+  "The input is not a correct window column: "
+]
+  },
+  "_LEGACY_ERROR_TEMP_3102" : {
+"message" : [
+  ""
+]
+  },
+  "_LEGACY_ERROR_TEMP_3103" : {
+"message" : [
+  "Namespace '' is non empty. "
+]
+  },
   "_LEGACY_ERROR_USER_RAISED_EXCEPTION" : {
 "message" : [
   ""
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
index 2aea9bac12fe..6475ac3093fe 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
@@ -25,12 +25,18 @@ import 
org.apache.spark.sql.catalyst.util.QuotingUtils.quoted
  * Thrown by a catalog when an item already exists. The analyzer will rethrow 
the exception
  * as an [[org.apache.spark.sql.AnalysisException]] with the correct position 
information.
  */
-case class NonEmptyNamespaceException private(
-override val message: String,
+case class NonEmptyNamespaceException(
+namespace: Array[String],
+details: String,
 override val cause: Option[Throwable] = None)
-  extends AnalysisException(message, cause = cause) {
+  extends AnalysisException(
+errorClass = "_LEGACY_ERROR_TEMP_3103",
+messageParameters = Map(
+  "namespace" -> quoted(namespace),
+  "details" -> details)) {
 
-  def this(namespace: Array[String]) = {
-this(s"Namespace '${quoted(namespace)}' is non empty.")
-  }
+  def this(namespace: Array[String]) = this(namespace, "", None)
+
+  def this(details: String, cause: Option[Throwable]) =
+this(Array.empty, details, cause)
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala
index 2eb7054edceb..1565935a8739 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ExtendedAnalysisException.scala
+++ 
b/sql/catalyst/src/main/scala/org/a

(spark) branch master updated: [MINOR][SQL] Convert `UnresolvedException` to an internal error

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

maxgekk 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 071c684dee44 [MINOR][SQL] Convert `UnresolvedException` to an internal 
error
071c684dee44 is described below

commit 071c684dee44665691ddab916021d4920a9ac51b
Author: Max Gekk 
AuthorDate: Tue Dec 12 18:06:48 2023 +0300

[MINOR][SQL] Convert `UnresolvedException` to an internal error

### What changes were proposed in this pull request?
In the PR, I propose to change the parent class of `UnresolvedException` 
from `AnalysisException` to `SparkException` with the error class 
`INTERNAL_ERROR`. If an user observes the error, this is definitely a bug in 
the compiler.

### Why are the changes needed?
To unify all Spark exceptions, and assign an error class. So, this should 
improve user experience with Spark SQL.

### Does this PR introduce _any_ user-facing change?
No, users shouldn't face to the error in regular cases.

### How was this patch tested?
By existing GAs.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44311 from MaxGekk/error-class-UnresolvedException.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala| 5 -
 1 file changed, 4 insertions(+), 1 deletion(-)

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 97912fb5d592..e1dec5955a7f 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
@@ -37,7 +37,10 @@ import org.apache.spark.util.ArrayImplicits._
  * resolved.
  */
 class UnresolvedException(function: String)
-  extends AnalysisException(s"Invalid call to $function on unresolved object")
+  extends SparkException(
+errorClass = "INTERNAL_ERROR",
+messageParameters = Map("message" -> s"Invalid call to $function on 
unresolved object"),
+cause = null)
 
 /** Parent trait for unresolved node types */
 trait UnresolvedNode extends LogicalPlan {


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



(spark) branch master updated: [SPARK-46351][SQL] Require an error class in `AnalysisException`

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

maxgekk 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 3e0808c33f1 [SPARK-46351][SQL] Require an error class in 
`AnalysisException`
3e0808c33f1 is described below

commit 3e0808c33f185c13808ce2d547ce9ba0057d31a6
Author: Max Gekk 
AuthorDate: Tue Dec 12 01:29:26 2023 +0300

[SPARK-46351][SQL] Require an error class in `AnalysisException`

### What changes were proposed in this pull request?
In the PR, I propose to create `AnalysisException` only with an error class 
by making the constructor with `message` protected. So, in this way only 
sub-classes can create `AnalysisException` by passing a `message`, but others 
shall provide an error class.

### Why are the changes needed?
To improve user experience with Spark SQL by unifying error exceptions: the 
final goal is all Spark exception should contain an error class.

### Does this PR introduce _any_ user-facing change?
No since user's code shouldn't throw `AnalysisException` but it can if it 
depends on error message formats.

### How was this patch tested?
By existing test test suites like:
```
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite"
```
and the modified test suites:
```
$ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *HiveDDLSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

    Closes #44277 from MaxGekk/protected-AnalysisException.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 253 +
 .../apache/spark/sql/avro/AvroDataToCatalyst.scala |  19 +-
 .../org/apache/spark/sql/test/SQLHelper.scala  |   4 +-
 .../connect/client/GrpcExceptionConverter.scala|  22 +-
 .../spark/sql/kafka010/KafkaSourceProvider.scala   |   9 +-
 .../apache/spark/sql/kafka010/KafkaWriter.scala|   5 +-
 .../org/apache/spark/sql/AnalysisException.scala   |  14 +-
 .../apache/spark/sql/catalyst/SQLConfHelper.scala  |   4 +-
 .../catalyst/analysis/ColumnResolutionHelper.scala |   8 +-
 .../ResolveRowLevelCommandAssignments.scala|   4 +-
 .../catalyst/analysis/RewriteMergeIntoTable.scala  |  12 +-
 .../catalyst/expressions/V2ExpressionUtils.scala   |  12 +-
 .../spark/sql/catalyst/planning/patterns.scala |   5 +-
 .../sql/catalyst/plans/logical/v2Commands.scala|   8 +-
 .../org/apache/spark/sql/util/SchemaUtils.scala|  25 +-
 .../catalyst/catalog/ExternalCatalogSuite.scala|   4 +-
 .../spark/sql/RelationalGroupedDataset.scala   |   4 +-
 .../spark/sql/execution/SparkStrategies.scala  |   3 +-
 .../spark/sql/execution/aggregate/AggUtils.scala   |   5 +-
 .../execution/datasources/FileSourceStrategy.scala |  13 +-
 .../parquet/ParquetSchemaConverter.scala   |   4 +-
 .../spark/sql/execution/datasources/rules.scala|   5 +-
 .../execution/datasources/v2/MergeRowsExec.scala   |   4 +-
 .../datasources/v2/state/utils/SchemaUtil.scala|   6 +-
 .../RowLevelOperationRuntimeGroupFiltering.scala   |   5 +-
 .../execution/streaming/WatermarkPropagator.scala  |   6 +-
 .../execution/streaming/statefulOperators.scala|   6 +-
 .../org/apache/spark/sql/jdbc/JdbcDialects.scala   |   5 +-
 .../sql/connector/DataSourceV2FunctionSuite.scala  |  10 +-
 .../spark/sql/connector/DataSourceV2SQLSuite.scala |  79 +--
 .../connector/FileDataSourceV2FallBackSuite.scala  |  28 ++-
 .../spark/sql/execution/QueryExecutionSuite.scala  |   2 +-
 .../execution/datasources/orc/OrcFilterSuite.scala |   3 +-
 .../sql/execution/datasources/orc/OrcTest.scala|   3 +-
 .../datasources/parquet/ParquetFilterSuite.scala   |   3 +-
 .../spark/sql/hive/HiveExternalCatalog.scala   |  32 ++-
 .../org/apache/spark/sql/hive/HiveInspectors.scala |  15 +-
 .../spark/sql/hive/HiveMetastoreCatalog.scala  |  17 +-
 .../spark/sql/hive/HiveSessionStateBuilder.scala   |   7 +-
 .../org/apache/spark/sql/hive/HiveStrategies.scala |   8 +-
 .../sql/hive/execution/V1WritesHiveUtils.scala |   4 +-
 .../spark/sql/hive/HiveMetastoreCatalogSuite.scala |  11 +-
 .../spark/sql/hive/execution/HiveDDLSuite.scala|  72 +++---
 .../spark/sql/hive/execution/HiveQuerySuite.scala  |   8 +-
 .../sql/hive/execution/Hive_2_1_DDLSuite.scala |   8 +-
 45 files changed, 611 insertions(+), 173 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 62d10c0d34c..d52ffc011b7 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -6705,6 +6705,259 @@
   "Failed to get block , whi

(spark) branch branch-3.5 updated: [MINOR][DOCS] Fix documentation for `spark.sql.legacy.doLooseUpcast` in SQL migration guide

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 9c83bf501cc [MINOR][DOCS] Fix documentation for 
`spark.sql.legacy.doLooseUpcast` in SQL migration guide
9c83bf501cc is described below

commit 9c83bf501ccefa7c6c0ba071f69e2528f3504854
Author: Amy Tsai 
AuthorDate: Mon Dec 11 18:35:31 2023 +0300

[MINOR][DOCS] Fix documentation for `spark.sql.legacy.doLooseUpcast` in SQL 
migration guide

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

Fixes an error in the SQL migration guide documentation for 
`spark.sql.legacy.doLooseUpcast`. I corrected the config name and moved it to 
the section for migration from Spark 2.4 to 3.0 since it was not made available 
until Spark 3.0.

### Why are the changes needed?

The config was documented as `spark.sql.legacy.looseUpcast` and is 
inaccurately included in the Spark 2.4 to Spark 2.4.1 section.

I changed the docs to match what is implemented in 
https://github.com/apache/spark/blob/20df062d85e80422a55afae80ddbf2060f26516c/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L3873

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

No.

### How was this patch tested?

Docs only change

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44262 from amytsai-stripe/fix-migration-docs-loose-upcast.

Authored-by: Amy Tsai 
Signed-off-by: Max Gekk 
(cherry picked from commit bab884082c0f82e3f9053adac6c7e8a3fcfab11c)
Signed-off-by: Max Gekk 
---
 docs/sql-migration-guide.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 88635ee3d1f..2eba9500e90 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -251,6 +251,8 @@ license: |
 
   - In Spark 3.0, the column metadata will always be propagated in the API 
`Column.name` and `Column.as`. In Spark version 2.4 and earlier, the metadata 
of `NamedExpression` is set as the `explicitMetadata` for the new column at the 
time the API is called, it won't change even if the underlying 
`NamedExpression` changes metadata. To restore the behavior before Spark 3.0, 
you can use the API `as(alias: String, metadata: Metadata)` with explicit 
metadata.
 
+  - When turning a Dataset to another Dataset, Spark will up cast the fields 
in the original Dataset to the type of corresponding fields in the target 
DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. 
`Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and 
throw NPE during execution. In Spark 3.0, the up cast is stricter and turning 
String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` 
will fail during analysis. To res [...]
+
 ### DDL Statements
 
   - In Spark 3.0, when inserting a value into a table column with a different 
data type, the type coercion is performed as per ANSI SQL standard. Certain 
unreasonable type conversions such as converting `string` to `int` and `double` 
to `boolean` are disallowed. A runtime exception is thrown if the value is 
out-of-range for the data type of the column. In Spark version 2.4 and below, 
type conversions during table insertion are allowed as long as they are valid 
`Cast`. When inserting an o [...]
@@ -464,8 +466,6 @@ license: |
 need to specify a value with units like "30s" now, to avoid being 
interpreted as milliseconds; otherwise,
 the extremely short interval that results will likely cause applications 
to fail.
 
-  - When turning a Dataset to another Dataset, Spark will up cast the fields 
in the original Dataset to the type of corresponding fields in the target 
DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. 
`Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and 
throw NPE during execution. In Spark 3.0, the up cast is stricter and turning 
String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` 
will fail during analysis. To res [...]
-
 ## Upgrading from Spark SQL 2.3 to 2.4
 
   - In Spark version 2.3 and earlier, the second parameter to array_contains 
function is implicitly promoted to the element type of first array type 
parameter. This type promotion can be lossy and may cause `array_contains` 
function to return wrong result. This problem has been addressed in 2.4 by 
employing a safer type promotion mechanism. This can cause some change in 
behavior and are illustrated in the table below.


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



(spark) branch master updated: [MINOR][DOCS] Fix documentation for `spark.sql.legacy.doLooseUpcast` in SQL migration guide

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

maxgekk 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 bab884082c0 [MINOR][DOCS] Fix documentation for 
`spark.sql.legacy.doLooseUpcast` in SQL migration guide
bab884082c0 is described below

commit bab884082c0f82e3f9053adac6c7e8a3fcfab11c
Author: Amy Tsai 
AuthorDate: Mon Dec 11 18:35:31 2023 +0300

[MINOR][DOCS] Fix documentation for `spark.sql.legacy.doLooseUpcast` in SQL 
migration guide

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

Fixes an error in the SQL migration guide documentation for 
`spark.sql.legacy.doLooseUpcast`. I corrected the config name and moved it to 
the section for migration from Spark 2.4 to 3.0 since it was not made available 
until Spark 3.0.

### Why are the changes needed?

The config was documented as `spark.sql.legacy.looseUpcast` and is 
inaccurately included in the Spark 2.4 to Spark 2.4.1 section.

I changed the docs to match what is implemented in 
https://github.com/apache/spark/blob/20df062d85e80422a55afae80ddbf2060f26516c/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L3873

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

No.

### How was this patch tested?

Docs only change

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44262 from amytsai-stripe/fix-migration-docs-loose-upcast.

Authored-by: Amy Tsai 
Signed-off-by: Max Gekk 
---
 docs/sql-migration-guide.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 9f9c15521c6..4e8e2422d7e 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -260,6 +260,8 @@ license: |
 
   - In Spark 3.0, the column metadata will always be propagated in the API 
`Column.name` and `Column.as`. In Spark version 2.4 and earlier, the metadata 
of `NamedExpression` is set as the `explicitMetadata` for the new column at the 
time the API is called, it won't change even if the underlying 
`NamedExpression` changes metadata. To restore the behavior before Spark 3.0, 
you can use the API `as(alias: String, metadata: Metadata)` with explicit 
metadata.
 
+  - When turning a Dataset to another Dataset, Spark will up cast the fields 
in the original Dataset to the type of corresponding fields in the target 
DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. 
`Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and 
throw NPE during execution. In Spark 3.0, the up cast is stricter and turning 
String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` 
will fail during analysis. To res [...]
+
 ### DDL Statements
 
   - In Spark 3.0, when inserting a value into a table column with a different 
data type, the type coercion is performed as per ANSI SQL standard. Certain 
unreasonable type conversions such as converting `string` to `int` and `double` 
to `boolean` are disallowed. A runtime exception is thrown if the value is 
out-of-range for the data type of the column. In Spark version 2.4 and below, 
type conversions during table insertion are allowed as long as they are valid 
`Cast`. When inserting an o [...]
@@ -473,8 +475,6 @@ license: |
 need to specify a value with units like "30s" now, to avoid being 
interpreted as milliseconds; otherwise,
 the extremely short interval that results will likely cause applications 
to fail.
 
-  - When turning a Dataset to another Dataset, Spark will up cast the fields 
in the original Dataset to the type of corresponding fields in the target 
DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. 
`Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and 
throw NPE during execution. In Spark 3.0, the up cast is stricter and turning 
String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` 
will fail during analysis. To res [...]
-
 ## Upgrading from Spark SQL 2.3 to 2.4
 
   - In Spark version 2.3 and earlier, the second parameter to array_contains 
function is implicitly promoted to the element type of first array type 
parameter. This type promotion can be lossy and may cause `array_contains` 
function to return wrong result. This problem has been addressed in 2.4 by 
employing a safer type promotion mechanism. This can cause some change in 
behavior and are illustrated in the table below.


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



(spark) branch master updated: [SPARK-46333][SQL] Replace `IllegalStateException` by `SparkException.internalError` in catalyst

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

maxgekk 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 8e95929ac423 [SPARK-46333][SQL] Replace `IllegalStateException` by 
`SparkException.internalError` in catalyst
8e95929ac423 is described below

commit 8e95929ac4238d02dca379837ccf2fbc1cd1926d
Author: Max Gekk 
AuthorDate: Sat Dec 9 12:32:21 2023 +0300

[SPARK-46333][SQL] Replace `IllegalStateException` by 
`SparkException.internalError` in catalyst

### What changes were proposed in this pull request?
In the PR, I propose to replace all `IllegalStateException` exception in 
`catalyst` by `SparkException.internalError`.

### Why are the changes needed?
This is a part of migration onto new error framework and error classes.

### Does this PR introduce _any_ user-facing change?
No, users shouldn't face to `IllegalStateException` in regular cases.

### How was this patch tested?
Using existing GAs.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44263 from MaxGekk/bind-ref-internal-error.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  3 ++-
 .../sql/catalyst/analysis/CheckAnalysis.scala  |  6 ++---
 .../sql/catalyst/analysis/v2ResolutionPlans.scala  | 13 +-
 .../sql/catalyst/catalog/SessionCatalog.scala  |  3 ++-
 .../spark/sql/catalyst/catalog/interface.scala |  3 ++-
 .../sql/catalyst/expressions/BoundAttribute.scala  |  3 ++-
 .../expressions/EquivalentExpressions.scala|  5 ++--
 .../expressions/InterpretedUnsafeProjection.scala  |  4 ++-
 .../expressions/ProjectionOverSchema.scala |  5 ++--
 .../sql/catalyst/expressions/arithmetic.scala  |  4 +--
 .../expressions/codegen/CodeGenerator.scala|  4 +--
 .../catalyst/expressions/codegen/javaCode.scala|  3 ++-
 .../expressions/collectionOperations.scala |  6 ++---
 .../catalyst/expressions/complexTypeCreator.scala  |  5 ++--
 .../sql/catalyst/expressions/csvExpressions.scala  |  3 ++-
 .../sql/catalyst/expressions/jsonExpressions.scala |  5 ++--
 .../catalyst/expressions/namedExpressions.scala|  3 ++-
 .../catalyst/optimizer/DecorrelateInnerQuery.scala |  4 +--
 .../catalyst/optimizer/NestedColumnAliasing.scala  |  3 ++-
 .../optimizer/NormalizeFloatingNumbers.scala   |  5 ++--
 .../spark/sql/catalyst/optimizer/Optimizer.scala   |  5 ++--
 .../optimizer/PushExtraPredicateThroughJoin.scala  |  3 ++-
 .../optimizer/ReplaceExceptWithFilter.scala|  3 ++-
 .../spark/sql/catalyst/optimizer/objects.scala |  7 ++---
 .../spark/sql/catalyst/optimizer/subquery.scala|  2 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala |  6 ++---
 .../sql/catalyst/plans/physical/partitioning.scala |  6 +++--
 .../apache/spark/sql/catalyst/trees/TreeNode.scala |  3 ++-
 .../sql/catalyst/util/ArrayBasedMapBuilder.scala   |  3 ++-
 .../spark/sql/catalyst/util/DateTimeUtils.scala|  8 +++---
 .../sql/catalyst/analysis/AnalysisSuite.scala  | 22 ++--
 .../optimizer/ReassignLambdaVariableIDSuite.scala  |  8 --
 .../sql/catalyst/util/DateTimeUtilsSuite.scala | 30 --
 .../sql/execution/WholeStageCodegenSuite.scala | 14 ++
 34 files changed, 126 insertions(+), 84 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 e5961b46e743..ec91f9b21a76 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
@@ -25,6 +25,7 @@ import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 import scala.util.{Failure, Random, Success, Try}
 
+import org.apache.spark.SparkException
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.catalog._
@@ -3706,7 +3707,7 @@ class Analyzer(override val catalogManager: 
CatalogManager) extends RuleExecutor
 case u @ UpCast(child, _, _) if !child.resolved => u
 
 case UpCast(_, target, _) if target != DecimalType && 
!target.isInstanceOf[DataType] =>
-  throw new IllegalStateException(
+  throw SparkException.internalError(
 s"UpCast only supports DecimalType as AbstractDataType yet, but 
got: $target")
 
 case UpCast(child, target, walkedTypePath) if target == DecimalType
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 ea

(spark) branch master updated: [SPARK-46187][SQL] Align codegen and non-codegen implementation of `StringDecode`

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

maxgekk 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 e93bff6fc0bc [SPARK-46187][SQL] Align codegen and non-codegen 
implementation of `StringDecode`
e93bff6fc0bc is described below

commit e93bff6fc0bc3a549de02958ffc17b1bca3d50b8
Author: Max Gekk 
AuthorDate: Fri Dec 1 10:42:31 2023 +0100

[SPARK-46187][SQL] Align codegen and non-codegen implementation of 
`StringDecode`

### What changes were proposed in this pull request?
In the PR, I propose to change the implementation of interpretation mode of 
`StringDecode` and apparently of the `decode` function. And make it consistent 
to codegen. Both implementation raise the same error with of the error class 
`INVALID_PARAMETER_VALUE.CHARSET`.

### Why are the changes needed?
To make codegen and non-codegen of the `StringDecode` expression 
consistent. So, users will observe the same behaviour in both modes.

### Does this PR introduce _any_ user-facing change?
Yes, if user code depends on error from `decode()`.

### How was this patch tested?
By running the following test suites:
```
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite -- -z string-functions.sql"
$ build/sbt "core/testOnly *SparkThrowableSuite"
$ build/sbt "test:testOnly *.StringFunctionsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #44094 from MaxGekk/align-codegen-stringdecode.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../catalyst/expressions/stringExpressions.scala   | 18 
 .../analyzer-results/ansi/string-functions.sql.out | 15 ++
 .../analyzer-results/string-functions.sql.out  | 15 ++
 .../sql-tests/inputs/string-functions.sql  |  2 ++
 .../results/ansi/string-functions.sql.out  | 34 ++
 .../sql-tests/results/string-functions.sql.out | 34 ++
 6 files changed, 113 insertions(+), 5 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 412422f4da4e..84a5eebd70ec 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
@@ -2648,18 +2648,26 @@ case class StringDecode(bin: Expression, charset: 
Expression)
 
   protected override def nullSafeEval(input1: Any, input2: Any): Any = {
 val fromCharset = input2.asInstanceOf[UTF8String].toString
-UTF8String.fromString(new String(input1.asInstanceOf[Array[Byte]], 
fromCharset))
+try {
+  UTF8String.fromString(new String(input1.asInstanceOf[Array[Byte]], 
fromCharset))
+} catch {
+  case _: UnsupportedEncodingException =>
+throw QueryExecutionErrors.invalidCharsetError(prettyName, fromCharset)
+}
   }
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-nullSafeCodeGen(ctx, ev, (bytes, charset) =>
+nullSafeCodeGen(ctx, ev, (bytes, charset) => {
+  val fromCharset = ctx.freshName("fromCharset")
   s"""
+String $fromCharset = $charset.toString();
 try {
-  ${ev.value} = UTF8String.fromString(new String($bytes, 
$charset.toString()));
+  ${ev.value} = UTF8String.fromString(new String($bytes, 
$fromCharset));
 } catch (java.io.UnsupportedEncodingException e) {
-  org.apache.spark.unsafe.Platform.throwException(e);
+  throw QueryExecutionErrors.invalidCharsetError("$prettyName", 
$fromCharset);
 }
-  """)
+  """
+})
   }
 
   override protected def withNewChildrenInternal(
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out
 
b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out
index 9d8705e3e862..7ace31d5 100644
--- 
a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out
@@ -799,6 +799,21 @@ Project [decode(null, 6, Spark, null, SQL, 4, rocks, null, 
.) AS decode(NULL, 6,
 +- OneRowRelation
 
 
+-- !query
+select decode(X'68656c6c6f', 'Windows-xxx')
+-- !query analysis
+Project [decode(0x68656C6C6F, Windows-xxx) AS decode(X'68656C6C6F', 
Windows-xxx)#x]
++- OneRowRelation
+
+
+-- !query
+select decode(scol, ecol) from values(X'68656c6c6f', 'Windows-xxx') as t(scol, 
ecol)
+-- !query analysis
+P

(spark) branch master updated: [SPARK-45826][SQL] Add a SQL config for stack traces in DataFrame query context

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

maxgekk 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 d30c9a90c6cf [SPARK-45826][SQL] Add a SQL config for stack traces in 
DataFrame query context
d30c9a90c6cf is described below

commit d30c9a90c6cf9033c45f6f418864c8d7013911e5
Author: Max Gekk 
AuthorDate: Sun Nov 26 14:10:27 2023 +0100

[SPARK-45826][SQL] Add a SQL config for stack traces in DataFrame query 
context

### What changes were proposed in this pull request?
In the PR, I propose to add new SQL config 
`spark.sql.stackTracesInDataFrameContext` which defines how many non-Spark 
stack traces should be captured into DataFrame query context. By default, the 
config is set to 1.

### Why are the changes needed?
To improve user experience with Spark SQL. When users troubleshoot an 
issue, they might need more stack traces in the DataFrame context. For example:
```scala
scala> spark.conf.set("spark.sql.ansi.enabled", true)
scala> spark.conf.set("spark.sql.stackTracesInDataFrameContext", 3)
scala> spark.range(1).select(lit(1) / lit(0)).collect()
org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by 
zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If 
necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. 
SQLSTATE: 22012
== DataFrame ==
"div" was called from
(:1)
(:16)
.(:1)
```

### Does this PR introduce _any_ user-facing change?
No, it doesn't change the default behaviour.

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "test:testOnly *QueryContextSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43695 from MaxGekk/df-context-slice-conf-2.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala   | 9 +
 sql/core/src/main/scala/org/apache/spark/sql/package.scala   | 5 -
 .../scala/org/apache/spark/sql/errors/QueryContextSuite.scala| 7 +--
 3 files changed, 18 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 6a8e1f92fc51..5133c40bc6fa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -4577,6 +4577,13 @@ object SQLConf {
   .booleanConf
   .createWithDefault(false)
 
+  val STACK_TRACES_IN_DATAFRAME_CONTEXT = 
buildConf("spark.sql.stackTracesInDataFrameContext")
+.doc("The number of non-Spark stack traces in the captured DataFrame query 
context.")
+.version("4.0.0")
+.intConf
+.checkValue(_ > 0, "The number of stack traces in the DataFrame context 
must be positive.")
+.createWithDefault(1)
+
   /**
* Holds information about keys that have been deprecated.
*
@@ -5465,6 +5472,8 @@ class SQLConf extends Serializable with Logging with 
SqlApiConf {
   def legacyRaiseErrorWithoutErrorClass: Boolean =
 getConf(SQLConf.LEGACY_RAISE_ERROR_WITHOUT_ERROR_CLASS)
 
+  def stackTracesInDataFrameContext: Int = 
getConf(SQLConf.STACK_TRACES_IN_DATAFRAME_CONTEXT)
+
   /** ** SQLConf functionality methods  */
 
   /** Set Spark SQL configuration properties. */
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
index 96bef83af0a8..877d9906a1cf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
@@ -22,6 +22,7 @@ import java.util.regex.Pattern
 import org.apache.spark.annotation.{DeveloperApi, Unstable}
 import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin}
 import org.apache.spark.sql.execution.SparkStrategy
+import org.apache.spark.sql.internal.SQLConf
 
 /**
  * Allows the execution of relational queries, including those expressed in 
SQL using Spark.
@@ -103,7 +104,9 @@ package object sql {
   while (i < st.length && !sparkCode(st(i))) i += 1
   // Stop at the end of the first Spark code traces
   while (i < st.length && sparkCode(st(i))) i += 1
-  val origin = Origin(stackTrace = Some(st.slice(i - 1, i + 1)))
+  val origin = Origin(stackTrace = Some(st.slice(
+from = i - 1,
+until = i + SQLConf.get.stackTracesInDataFrameContext)))
   CurrentOrigin.withOrigin(origin)(f)
 }
   }
diff --git 
a/sql/core/src/test/scala/or

(spark) branch master updated: [SPARK-45887][SQL] Align codegen and non-codegen implementation of `Encode`

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

maxgekk 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 e470e7442f9 [SPARK-45887][SQL] Align codegen and non-codegen 
implementation of `Encode`
e470e7442f9 is described below

commit e470e7442f90f7189346993d76733cafe7469ada
Author: Max Gekk 
AuthorDate: Thu Nov 23 18:33:13 2023 +0300

[SPARK-45887][SQL] Align codegen and non-codegen implementation of `Encode`

### What changes were proposed in this pull request?
In the PR, I propose to change the implementation of interpretation mode, 
and make it consistent to codegen. Both implementation raise the same error 
with new error class `INVALID_PARAMETER_VALUE.CHARSET`.

### Why are the changes needed?
To make codegen and non-codegen of the `Encode` expression consistent. So, 
users will observe the same behaviour in both modes.

### Does this PR introduce _any_ user-facing change?
Yes, if user code depends on error from `encode()`.

### How was this patch tested?
By running the following test suites:
```
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite -- -z string-functions.sql"
$ build/sbt "core/testOnly *SparkThrowableSuite"
$ build/sbt "test:testOnly *.StringFunctionsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43759 from MaxGekk/restrict-charsets-in-encode.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json|  5 
 ...nditions-invalid-parameter-value-error-class.md |  4 +++
 .../catalyst/expressions/stringExpressions.scala   | 19 
 .../spark/sql/errors/QueryExecutionErrors.scala|  9 ++
 .../analyzer-results/ansi/string-functions.sql.out | 15 ++
 .../analyzer-results/string-functions.sql.out  | 15 ++
 .../sql-tests/inputs/string-functions.sql  |  4 +++
 .../results/ansi/string-functions.sql.out  | 34 ++
 .../sql-tests/results/string-functions.sql.out | 34 ++
 9 files changed, 134 insertions(+), 5 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index b0621c44532..19b70307a1c 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -2042,6 +2042,11 @@
   "expects an integer value in [0, ), but got ."
 ]
   },
+  "CHARSET" : {
+"message" : [
+  "expects one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 
'UTF-16BE', 'UTF-16LE', 'UTF-16', but got ."
+]
+  },
   "DATETIME_UNIT" : {
 "message" : [
   "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, 
DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the 
string literal ."
diff --git a/docs/sql-error-conditions-invalid-parameter-value-error-class.md 
b/docs/sql-error-conditions-invalid-parameter-value-error-class.md
index d58d4fc2f59..8547d8b31f0 100644
--- a/docs/sql-error-conditions-invalid-parameter-value-error-class.md
+++ b/docs/sql-error-conditions-invalid-parameter-value-error-class.md
@@ -45,6 +45,10 @@ expects one of binary formats 'base64', 'hex', 'utf-8', but 
got `
 
 expects an integer value in [0, ``), but got ``.
 
+## CHARSET
+
+expects one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 
'UTF-16LE', 'UTF-16', but got ``.
+
 ## DATETIME_UNIT
 
 expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, 
DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string 
literal ``.
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 811d6e013ab..0d3239423b2 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
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
+import java.io.UnsupportedEncodingException
 import java.text.{BreakIterator, DecimalFormat, DecimalFormatSymbols}
 import java.util.{Base64 => JBase64}
 import java.util.{HashMap, Locale, Map => JMap}
@@ -2694,17 +2695,25 @@ case class Encode(value: Expression, charset: 
Expression)
 
   protected override def nullSafeEval(input1: Any, input2: Any): Any = {
 val toCharset = input2.asInstanceOf[UTF8String].toString
-inp

(spark) branch master updated: [SPARK-46070][SQL] Compile regex pattern in SparkDateTimeUtils.getZoneId outside the hot loop

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

maxgekk 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 240706600db [SPARK-46070][SQL] Compile regex pattern in 
SparkDateTimeUtils.getZoneId outside the hot loop
240706600db is described below

commit 240706600dbf257dfcf378acaadd608348d666aa
Author: Tanel Kiis 
AuthorDate: Thu Nov 23 18:30:37 2023 +0300

[SPARK-46070][SQL] Compile regex pattern in SparkDateTimeUtils.getZoneId 
outside the hot loop

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

Compile the regex patterns used in `SparkDateTimeUtils.getZoneId` outside 
of the method, that can be called for each dataset row..

### Why are the changes needed?

`String.replaceFirst` internally does 
`Pattern.compile(regex).matcher(this).replaceFirst(replacement)`. 
`Pattern.compile` is very expensive method, that should not be called in a loop.

When using method like `from_utc_timestamp` with non-literal timezone, the 
`SparkDateTimeUtils.getZoneId` is called for each loop. In one of my usecases 
adding `from_utc_timestamp` increased the runtime from 15min to 6h.

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

Performance improvement.

### How was this patch tested?

Existing UTs

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #43976 from tanelk/SPARK-46070_precompile_regex.

Authored-by: Tanel Kiis 
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala | 13 -
 1 file changed, 8 insertions(+), 5 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
index f8a9274a564..5e9fb0dd25f 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala
@@ -20,6 +20,7 @@ import java.sql.{Date, Timestamp}
 import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZonedDateTime, 
ZoneId, ZoneOffset}
 import java.util.TimeZone
 import java.util.concurrent.TimeUnit.{MICROSECONDS, NANOSECONDS}
+import java.util.regex.Pattern
 
 import scala.util.control.NonFatal
 
@@ -36,12 +37,14 @@ trait SparkDateTimeUtils {
 
   final val TimeZoneUTC = TimeZone.getTimeZone("UTC")
 
+  final val singleHourTz = Pattern.compile("(\\+|\\-)(\\d):")
+  final val singleMinuteTz = Pattern.compile("(\\+|\\-)(\\d\\d):(\\d)$")
+
   def getZoneId(timeZoneId: String): ZoneId = {
-val formattedZoneId = timeZoneId
-  // To support the (+|-)h:mm format because it was supported before Spark 
3.0.
-  .replaceFirst("(\\+|\\-)(\\d):", "$10$2:")
-  // To support the (+|-)hh:m format because it was supported before Spark 
3.0.
-  .replaceFirst("(\\+|\\-)(\\d\\d):(\\d)$", "$1$2:0$3")
+// To support the (+|-)h:mm format because it was supported before Spark 
3.0.
+var formattedZoneId = 
singleHourTz.matcher(timeZoneId).replaceFirst("$10$2:")
+// To support the (+|-)hh:m format because it was supported before Spark 
3.0.
+formattedZoneId = 
singleMinuteTz.matcher(formattedZoneId).replaceFirst("$1$2:0$3")
 
 ZoneId.of(formattedZoneId, ZoneId.SHORT_IDS)
   }


-
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] Pass `cause` in `CannotReplaceMissingTableException` costructor

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

maxgekk 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 49ca6aa6cb7 [MINOR][SQL] Pass `cause` in 
`CannotReplaceMissingTableException` costructor
49ca6aa6cb7 is described below

commit 49ca6aa6cb75b931d1c38dcffb4cd3dd63b0a2f3
Author: Max Gekk 
AuthorDate: Fri Nov 10 12:17:09 2023 +0300

[MINOR][SQL] Pass `cause` in `CannotReplaceMissingTableException` costructor

### What changes were proposed in this pull request?
In the PR, I propose to use the `cause` argument in the 
`CannotReplaceMissingTableException` constructor.

### Why are the changes needed?
To improve user experience with Spark SQL while troubleshooting issues. 
Currently, users don't see where the exception come from.

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

### How was this patch tested?
Manually.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43738 from MaxGekk/fix-missed-cause.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/analysis/CannotReplaceMissingTableException.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala
index 910bb9d3749..032cdca12c0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala
@@ -28,4 +28,5 @@ class CannotReplaceMissingTableException(
   extends AnalysisException(
   errorClass = "TABLE_OR_VIEW_NOT_FOUND",
   messageParameters = Map("relationName"
--> quoteNameParts(tableIdentifier.namespace :+ tableIdentifier.name)))
+-> quoteNameParts(tableIdentifier.namespace :+ tableIdentifier.name)),
+  cause = cause)


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



(spark) branch master updated: [SPARK-45841][SQL] Expose stack trace by `DataFrameQueryContext`

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

maxgekk 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 6abc4a1a58ef [SPARK-45841][SQL] Expose stack trace by 
`DataFrameQueryContext`
6abc4a1a58ef is described below

commit 6abc4a1a58ef4e5d896717b10b2314dae2af78af
Author: Max Gekk 
AuthorDate: Wed Nov 8 15:51:50 2023 +0300

[SPARK-45841][SQL] Expose stack trace by `DataFrameQueryContext`

### What changes were proposed in this pull request?
In the PR, I propose to change the case class `DataFrameQueryContext`, and 
add stack traces as a field and override `callSite`, `fragment` using the new 
field `stackTrace`.

### Why are the changes needed?
By exposing the stack trace, we give users opportunity to see all stack 
traces needed for debugging.

### Does this PR introduce _any_ user-facing change?
No, `DataFrameQueryContext` hasn't been released yet.

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "test:testOnly *DatasetSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43703 from MaxGekk/stack-traces-in-DataFrameQueryContext.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/trees/QueryContexts.scala   | 33 +-
 .../scala/org/apache/spark/sql/DatasetSuite.scala  | 13 +
 2 files changed, 22 insertions(+), 24 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala
index 8d885d07ca8b..874c834b7558 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/trees/QueryContexts.scala
@@ -134,9 +134,7 @@ case class SQLQueryContext(
   override def callSite: String = throw new UnsupportedOperationException
 }
 
-case class DataFrameQueryContext(
-override val fragment: String,
-override val callSite: String) extends QueryContext {
+case class DataFrameQueryContext(stackTrace: Seq[StackTraceElement]) extends 
QueryContext {
   override val contextType = QueryContextType.DataFrame
 
   override def objectType: String = throw new UnsupportedOperationException
@@ -144,6 +142,19 @@ case class DataFrameQueryContext(
   override def startIndex: Int = throw new UnsupportedOperationException
   override def stopIndex: Int = throw new UnsupportedOperationException
 
+  override val fragment: String = {
+stackTrace.headOption.map { firstElem =>
+  val methodName = firstElem.getMethodName
+  if (methodName.length > 1 && methodName(0) == '$') {
+methodName.substring(1)
+  } else {
+methodName
+  }
+}.getOrElse("")
+  }
+
+  override val callSite: String = 
stackTrace.tail.headOption.map(_.toString).getOrElse("")
+
   override lazy val summary: String = {
 val builder = new StringBuilder
 builder ++= "== DataFrame ==\n"
@@ -157,19 +168,3 @@ case class DataFrameQueryContext(
 builder.result()
   }
 }
-
-object DataFrameQueryContext {
-  def apply(elements: Array[StackTraceElement]): DataFrameQueryContext = {
-val fragment = elements.headOption.map { firstElem =>
-  val methodName = firstElem.getMethodName
-  if (methodName.length > 1 && methodName(0) == '$') {
-methodName.substring(1)
-  } else {
-methodName
-  }
-}.getOrElse("")
-val callSite = elements.tail.headOption.map(_.toString).getOrElse("")
-
-DataFrameQueryContext(fragment, callSite)
-  }
-}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 66105d2ac429..dcbd8948120c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -37,6 +37,7 @@ import 
org.apache.spark.sql.catalyst.encoders.{AgnosticEncoders, ExpressionEncod
 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.BoxedIntEncoder
 import org.apache.spark.sql.catalyst.expressions.{CodegenObjectFactoryMode, 
GenericRowWithSchema}
 import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi}
+import org.apache.spark.sql.catalyst.trees.DataFrameQueryContext
 import org.apache.spark.sql.catalyst.util.sideBySide
 import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution}
 import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
@@ -2668,16 +2669,18 @@ class DatasetSuite extends QueryTest
 withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") {
   val df = Seq(1

(spark) branch master updated: [SPARK-45824][SQL] Enforce the error class in `ParseException`

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

maxgekk 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 d582b74d97a7 [SPARK-45824][SQL] Enforce the error class in 
`ParseException`
d582b74d97a7 is described below

commit d582b74d97a7e44bdd2f0ae6c63121fc5e5466b7
Author: Max Gekk 
AuthorDate: Wed Nov 8 11:38:55 2023 +0300

[SPARK-45824][SQL] Enforce the error class in `ParseException`

### What changes were proposed in this pull request?
In the PR, I propose to enforce creation of `ParseException` with an error 
class always. In particular, it converts the constructor with a message to 
private one, so, callers have to create `ParseException` with an error class.

### Why are the changes needed?
This simplifies migration on error classes.

### Does this PR introduce _any_ user-facing change?
No since user code doesn't throw `ParseException` in regular cases.

### How was this patch tested?
By existing test suites, for instance:
```
$ build/sbt "sql/testOnly *QueryParsingErrorsSuite"
$ build/sbt "test:testOnly *SparkConnectClientSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

    Closes #43702 from MaxGekk/ban-message-ParseException.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../connect/client/SparkConnectClientSuite.scala   | 15 ++
 .../connect/client/GrpcExceptionConverter.scala|  3 +-
 .../apache/spark/sql/catalyst/parser/parsers.scala | 53 --
 .../spark/sql/errors/QueryParsingErrors.scala  |  8 +++-
 4 files changed, 51 insertions(+), 28 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
index b3ff4eb0bb29..d0c85da5f212 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
@@ -31,7 +31,6 @@ import org.apache.spark.{SparkException, SparkThrowable}
 import org.apache.spark.connect.proto
 import org.apache.spark.connect.proto.{AddArtifactsRequest, 
AddArtifactsResponse, AnalyzePlanRequest, AnalyzePlanResponse, 
ArtifactStatusesRequest, ArtifactStatusesResponse, ExecutePlanRequest, 
ExecutePlanResponse, SparkConnectServiceGrpc}
 import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.connect.common.config.ConnectCommon
 import org.apache.spark.sql.test.ConnectFunSuite
 
@@ -210,19 +209,15 @@ class SparkConnectClientSuite extends ConnectFunSuite 
with BeforeAndAfterEach {
   }
 
   for ((name, constructor) <- GrpcExceptionConverter.errorFactory) {
-test(s"error framework parameters - ${name}") {
+test(s"error framework parameters - $name") {
   val testParams = GrpcExceptionConverter.ErrorParams(
-message = "test message",
+message = "Found duplicate keys `abc`",
 cause = None,
-errorClass = Some("test error class"),
-messageParameters = Map("key" -> "value"),
+errorClass = Some("DUPLICATE_KEY"),
+messageParameters = Map("keyColumn" -> "`abc`"),
 queryContext = Array.empty)
   val error = constructor(testParams)
-  if (!error.isInstanceOf[ParseException]) {
-assert(error.getMessage == testParams.message)
-  } else {
-assert(error.getMessage == s"\n${testParams.message}")
-  }
+  assert(error.getMessage.contains(testParams.message))
   assert(error.getCause == null)
   error match {
 case sparkThrowable: SparkThrowable =>
diff --git 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
index 652797bc2e40..88cd2118ba75 100644
--- 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
+++ 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
@@ -191,10 +191,9 @@ private[client] object GrpcExceptionConverter {
 errorConstructor(params =>
   new ParseException(
 None,
-params.message,
 Origin(),
 Origin(),
-errorClass = params.errorClass,
+errorClass = params.errorClass.orNull,
 messageParameters = params.messageParameters,
   

(spark) branch master updated: [SPARK-45710][SQL] Assign names to error _LEGACY_ERROR_TEMP_21[59,60,61,62]

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

maxgekk 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 9cbc2d138a1 [SPARK-45710][SQL] Assign names to error 
_LEGACY_ERROR_TEMP_21[59,60,61,62]
9cbc2d138a1 is described below

commit 9cbc2d138a1de2d3ff399d224d817554ba0b9e18
Author: dengziming 
AuthorDate: Sun Nov 5 20:07:07 2023 +0300

[SPARK-45710][SQL] Assign names to error _LEGACY_ERROR_TEMP_21[59,60,61,62]

### What changes were proposed in this pull request?
This PR are removing `_LEGACY_ERROR_TEMP_21[59,60,61,62]` and 
`TOO_MANY_ARRAY_ELEMENTS`:
1. `_LEGACY_ERROR_TEMP_2159` is used in concat/array_insert;
2. `_LEGACY_ERROR_TEMP_2160` is only used in flatten;
3. `_LEGACY_ERROR_TEMP_2161` is used in 
array_repeat/array_insert/array_distinct/array_union/array_intersect/array_remove;
4. `_LEGACY_ERROR_TEMP_2162` is used in array_union/array_distinct;
5. There is another similar error class `TOO_MANY_ARRAY_ELEMENTS` which are 
used in `UnsafeArrayWriter.java`.

I removed these 5 similar error classes and create a new error class 
`COLLECTION_SIZE_LIMIT_EXCEEDED` with 3 sub-classes:
1. `PARAMETER` is used when the parameter exceed size limit, such as 
`array_repeat` with count too large;
6. `FUNCTION` is used when trying to create an array exceeding size limit 
in a function, for example, flatten 2 arrays to a larger array;
7. `INITIALIZE` is used in `UnsafeArrayWriter.java` when trying to 
initialize an array exceeding size limit.

### Why are the changes needed?
To assign proper name as a part of activity in SPARK-37935.

### Does this PR introduce _any_ user-facing change?
Yes, the error message will include the error class name.

### How was this patch tested?
1. `COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER` can be tested from use code;
2.  `COLLECTION_SIZE_LIMIT_EXCEEDED.FUNCTION` is tested using a 
`ColumnarArray` in `concat/flatten`, but can't be tested in 
`array_insert/array_distinct/array_union/array_intersect/array_remove` since we 
need to deduplicate the data and create an array which will cause OOM.
3. `INITIALIZE` is already tested in a existing case.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43567 from dengziming/SPARK-45710.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 49 ++-
 docs/_data/menu-sql.yaml   |  2 +
 ...s-collection-size-limit-exceeded-error-class.md | 38 +
 docs/sql-error-conditions.md   | 14 ++--
 .../expressions/codegen/UnsafeArrayWriter.java |  3 +-
 .../expressions/collectionOperations.scala | 95 --
 .../spark/sql/errors/QueryExecutionErrors.scala| 48 +--
 .../codegen/UnsafeArrayWriterSuite.scala   |  6 +-
 .../sql/errors/QueryExecutionErrorsSuite.scala | 59 +-
 9 files changed, 184 insertions(+), 130 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 8b0951a7b00..3e0743d366a 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -405,6 +405,29 @@
 ],
 "sqlState" : "42704"
   },
+  "COLLECTION_SIZE_LIMIT_EXCEEDED" : {
+"message" : [
+  "Can't create array with  elements which exceeding the 
array size limit ,"
+],
+"subClass" : {
+  "FUNCTION" : {
+"message" : [
+  "unsuccessful try to create arrays in the function ."
+]
+  },
+  "INITIALIZE" : {
+"message" : [
+  "cannot initialize an array with specified parameters."
+]
+  },
+  "PARAMETER" : {
+"message" : [
+  "the value of parameter(s)  in the function 
 is invalid."
+]
+  }
+},
+"sqlState" : "54000"
+  },
   "COLUMN_ALIASES_IS_NOT_ALLOWED" : {
 "message" : [
   "Columns aliases are not allowed in ."
@@ -3017,12 +3040,6 @@
 ],
 "sqlState" : "428EK"
   },
-  "TOO_MANY_ARRAY_ELEMENTS" : {
-"message" : [
-  "Cannot initialize array with  elements of size ."
-],
-"sqlState" : "54000"
-  },
   "UDTF_ALIAS_NUMBER_MISMATCH" : {
 "message" : [
   "The number of aliases supplied in the AS clause does not match the 
number of columns output by the UDTF.",
@@ -5765,26 +5782,6 @@
   "

(spark) branch master updated: [SPARK-38723][SS][TEST][FOLLOWUP] Deflake the newly added test in QueryExecutionErrorsSuite

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

maxgekk 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 97ba9d29cd1 [SPARK-38723][SS][TEST][FOLLOWUP] Deflake the newly added 
test in QueryExecutionErrorsSuite
97ba9d29cd1 is described below

commit 97ba9d29cd1ff83755b0d02251d249a625caace5
Author: Wei Liu 
AuthorDate: Tue Oct 31 09:38:50 2023 +0300

[SPARK-38723][SS][TEST][FOLLOWUP] Deflake the newly added test in 
QueryExecutionErrorsSuite

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

The newly added test in 
https://github.com/apache/spark/commit/7d7afb06f682c10f3900eb8adeab9fad6d49cb24 
could be flaky, this change deflakes it. Details see comments.

### Why are the changes needed?

Deflaky

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

Test only change

### How was this patch tested?

Test only change

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #43565 from WweiL/SPARK-38723-followup.

Authored-by: Wei Liu 
Signed-off-by: Max Gekk 
---
 .../spark/sql/errors/QueryExecutionErrorsSuite.scala | 20 +++-
 1 file changed, 11 insertions(+), 9 deletions(-)

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 945dd782da0..dd3f3dc6004 100644
--- 
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
@@ -22,6 +22,8 @@ import java.net.{URI, URL}
 import java.sql.{Connection, DatabaseMetaData, Driver, DriverManager, 
PreparedStatement, ResultSet, ResultSetMetaData}
 import java.util.{Locale, Properties, ServiceConfigurationError}
 
+import scala.jdk.CollectionConverters._
+
 import org.apache.hadoop.fs.{LocalFileSystem, Path}
 import org.apache.hadoop.fs.permission.FsPermission
 import org.mockito.Mockito.{mock, spy, when}
@@ -910,15 +912,15 @@ class QueryExecutionErrorsSuite
   }
   exception
 }
-  assert(exceptions.map(e => e.isDefined).reduceLeft(_ || _))
-  exceptions.map { e =>
-if (e.isDefined) {
-  checkError(
-e.get,
-errorClass = "CONCURRENT_QUERY",
-sqlState = Some("0A000")
-  )
-}
+  // Only check if errors exist to deflake. We couldn't guarantee that
+  // the above 50 runs must hit this error.
+  exceptions.flatten.map { e =>
+checkError(
+  e,
+  errorClass = "CONCURRENT_QUERY",
+  sqlState = Some("0A000"),
+  parameters = e.getMessageParameters.asScala.toMap
+)
   }
   spark.streams.active.foreach(_.stop())
 }


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



(spark) branch master updated: [SPARK-45328][DOCS][FOLLOWUP] Update docs for Hive metastore supported versions

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

maxgekk 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 efa891c4ea8 [SPARK-45328][DOCS][FOLLOWUP] Update docs for Hive 
metastore supported versions
efa891c4ea8 is described below

commit efa891c4ea869a7fac58eabf187ac8aeff1fcd38
Author: Cheng Pan 
AuthorDate: Fri Oct 27 13:03:40 2023 +0300

[SPARK-45328][DOCS][FOLLOWUP] Update docs for Hive metastore supported 
versions

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

A minor follow-up of https://github.com/apache/spark/pull/43116

### Why are the changes needed?

Correct the docs of `spark.sql.hive.metastore.version`

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

Yes, docs changed.

### How was this patch tested?

Review

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #43552 from pan3793/SPARK-45328-doc.

Authored-by: Cheng Pan 
Signed-off-by: Max Gekk 
---
 docs/sql-data-sources-hive-tables.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/sql-data-sources-hive-tables.md 
b/docs/sql-data-sources-hive-tables.md
index 0de573ec64b..a2c37e69f9a 100644
--- a/docs/sql-data-sources-hive-tables.md
+++ b/docs/sql-data-sources-hive-tables.md
@@ -130,7 +130,7 @@ The following options can be used to configure the version 
of Hive that is used
 2.3.9
 
   Version of the Hive metastore. Available
-  options are 0.12.0 through 2.3.9 and 
3.0.0 through 3.1.3.
+  options are 2.0.0 through 2.3.9 and 
3.0.0 through 3.1.3.
 
 1.4.0
   


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



(spark) branch master updated: [SPARK-45698][CORE][SQL][SS] Clean up the deprecated API usage related to `Buffer`

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

maxgekk 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 d072833d50a [SPARK-45698][CORE][SQL][SS] Clean up the deprecated API 
usage related to `Buffer`
d072833d50a is described below

commit d072833d50abd1a6630dd629e3560e3d0ad929cf
Author: yangjie01 
AuthorDate: Fri Oct 27 12:59:22 2023 +0300

[SPARK-45698][CORE][SQL][SS] Clean up the deprecated API usage related to 
`Buffer`

### What changes were proposed in this pull request?
This PR cleans up the use of the following APIs in `s.c.mutable.Buffer`, as 
they have been deprecated after Scala 2.13.0 or Scala 2.13.4:

- `trimStart` -> `dropInPlace`
- `trimEnd` -> `dropRightInPlace`
- `append(elems: A*)` -> `appendAll`
- `prepend(elems: A*)` -> `prependAll(elems)`

```
  deprecated("use dropInPlace instead", since = "2.13.4")
  def trimStart(n: Int): Unit = dropInPlace(n)

  deprecated("use dropRightInPlace instead", since = "2.13.4")
  def trimEnd(n: Int): Unit = dropRightInPlace(n)

  deprecated("Use appendAll instead", "2.13.0")
  `inline` final def append(elems: A*): this.type = addAll(elems)

  deprecated("Use prependAll instead", "2.13.0")
  `inline` final def prepend(elems: A*): this.type = prependAll(elems)
```

### Why are the changes needed?
Clean up deprecated API usage.

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

### How was this patch tested?
Pass GitHub Acitons.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #43551 from LuciferYang/buffer-deprecated.

Authored-by: yangjie01 
Signed-off-by: Max Gekk 
---
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala   | 4 ++--
 core/src/main/scala/org/apache/spark/util/SizeEstimator.scala | 2 +-
 .../org/apache/spark/util/collection/ExternalAppendOnlyMap.scala  | 2 +-
 core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala| 2 +-
 .../src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala | 4 ++--
 .../org/apache/spark/sql/execution/arrow/ArrowConverters.scala| 2 +-
 .../streaming/continuous/ContinuousTextSocketSource.scala | 2 +-
 .../scala/org/apache/spark/sql/execution/streaming/memory.scala   | 2 +-
 .../execution/streaming/sources/TextSocketMicroBatchStream.scala  | 2 +-
 sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala  | 8 
 10 files changed, 15 insertions(+), 15 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 46503e017ea..29022c7419b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -1042,7 +1042,7 @@ private[deploy] class Master(
 completedApps.take(toRemove).foreach { a =>
   applicationMetricsSystem.removeSource(a.appSource)
 }
-completedApps.trimStart(toRemove)
+completedApps.dropInPlace(toRemove)
   }
   completedApps += app // Remember it in our history
   waitingApps -= app
@@ -1204,7 +1204,7 @@ private[deploy] class Master(
 drivers -= driver
 if (completedDrivers.size >= retainedDrivers) {
   val toRemove = math.max(retainedDrivers / 10, 1)
-  completedDrivers.trimStart(toRemove)
+  completedDrivers.dropInPlace(toRemove)
 }
 completedDrivers += driver
 persistenceEngine.removeDriver(driver)
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala 
b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index 704aeaefa55..1447a3e752d 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -180,7 +180,7 @@ object SizeEstimator extends Logging {
 
 def dequeue(): AnyRef = {
   val elem = stack.last
-  stack.trimEnd(1)
+  stack.dropRightInPlace(1)
   elem
 }
   }
diff --git 
a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
 
b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 3afbe322b6e..8224472b754 100644
--- 
a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ 
b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -362,7 +362,7 @@ class ExternalAppendOnlyMap[K, V, C](
 private def removeFromBuffer[T](buffer: ArrayBuffer[T], index: Int): T = {
   val elem = buffer(index)
   buffer(index) 

[spark] branch master updated: [SPARK-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8]

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

maxgekk 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 23c4cce61b7 [SPARK-45614][SQL] Assign names to error 
_LEGACY_ERROR_TEMP_215[6,7,8]
23c4cce61b7 is described below

commit 23c4cce61b78c1f25e34850c9d0e242dd1fcffe8
Author: dengziming 
AuthorDate: Fri Oct 27 10:44:30 2023 +0300

[SPARK-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8]

### What changes were proposed in this pull request?
Replace the legacy error class `_LEGACY_ERROR_TEMP_2156` with an internal 
error.
Assign the name `INVALID_PARAMETER_VALUE.START` to the legacy error class 
`_LEGACY_ERROR_TEMP_2157`.
Assign the name `INVALID_PARAMETER_VALUE.LENGTH` to the legacy error class 
`_LEGACY_ERROR_TEMP_2158`.

### Why are the changes needed?
To assign proper name as a part of activity in SPARK-37935.

### Does this PR introduce _any_ user-facing change?
Yes, the error message will include the error class name.

### How was this patch tested?
`_LEGACY_ERROR_TEMP_2156` can't be produced from user code since it's an 
internal error, so I added a unit test in `CollectionExpressionsSuite.scala`.
`INVALID_PARAMETER_VALUE.START` and `INVALID_PARAMETER_VALUE.LENGTH` can be 
produced from user code so I add 2 tests in `QueryExecutionErrorsSuite.scala` 
to validate them.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #43481 from dengziming/SPARK-45614.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 25 ++-
 ...nditions-invalid-parameter-value-error-class.md |  8 +
 .../expressions/collectionOperations.scala |  5 +--
 .../spark/sql/errors/QueryExecutionErrors.scala| 22 +++--
 .../expressions/CollectionExpressionsSuite.scala   | 36 +++---
 .../sql/errors/QueryExecutionErrorsSuite.scala | 31 ++-
 6 files changed, 94 insertions(+), 33 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 5b756ad1077..f9cc0a86521 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1974,6 +1974,11 @@
   "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, 
DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the 
string literal ."
 ]
   },
+  "LENGTH" : {
+"message" : [
+  "Expects `length` greater than or equal to 0, but got ."
+]
+  },
   "NULL" : {
 "message" : [
   "expects a non-NULL value."
@@ -1989,6 +1994,11 @@
   "Expects group index between 0 and , but got 
."
 ]
   },
+  "START" : {
+"message" : [
+  "Expects a positive or a negative value for `start`, but got 0."
+]
+  },
   "ZERO_INDEX" : {
 "message" : [
   "expects %1$, %2$ and so on, but got %0$."
@@ -5749,21 +5759,6 @@
   " is not annotated with SQLUserDefinedType nor registered 
with UDTRegistration.}"
 ]
   },
-  "_LEGACY_ERROR_TEMP_2156" : {
-"message" : [
-  "The size function doesn't support the operand type ."
-]
-  },
-  "_LEGACY_ERROR_TEMP_2157" : {
-"message" : [
-  "Unexpected value for start in function : SQL array indices 
start at 1."
-]
-  },
-  "_LEGACY_ERROR_TEMP_2158" : {
-"message" : [
-  "Unexpected value for length in function : length must be 
greater than or equal to 0."
-]
-  },
   "_LEGACY_ERROR_TEMP_2159" : {
 "message" : [
   "Unsuccessful try to concat arrays with  elements due 
to exceeding the array size limit ."
diff --git a/docs/sql-error-conditions-invalid-parameter-value-error-class.md 
b/docs/sql-error-conditions-invalid-parameter-value-error-class.md
index 8186a56314d..d58d4fc2f59 100644
--- a/docs/sql-error-conditions-invalid-parameter-value-error-class.md
+++ b/docs/sql-error-conditions-invalid-parameter-value-error-class.md
@@ -49,6 +49,10 @@ expects an integer value in [0, ``), but got 
``.
 
 expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, 
DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string 
literal ``.
 
+## LENGTH
+
+Expects `length` greater than or equal to 0, but got ``.
+
 ## NULL
 
 expects a non-NULL value.
@@ -61,6 +65,10 @@ expects a non-NULL value.
 
 Expects 

[spark] branch master updated: [SPARK-45660] Re-use Literal objects in ComputeCurrentTime rule

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

maxgekk 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 344453761cb [SPARK-45660] Re-use Literal objects in ComputeCurrentTime 
rule
344453761cb is described below

commit 344453761cbca154a04a53d4c5d6c2b1eef59652
Author: Ole Sasse 
AuthorDate: Wed Oct 25 19:56:15 2023 +0300

[SPARK-45660] Re-use Literal objects in ComputeCurrentTime rule

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

The ComputeCurrentTime optimizer rule does produce unique timestamp 
Literals for current time expressions of a query. For CurrentDate and 
LocalTimestamp the Literal objects are not re-used though, but semantically 
equal objects are created for each instance. This can cost unnecessary much 
memory in case there are many such Literal objects.

This PR adds a map that caches timestamp literals in case they are used 
more than once.

### Why are the changes needed?

A query that has a lot of equal literals could use unnecessary high amounts 
of memory

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

No

### How was this patch tested?

Added a new Unit Test

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #43524 from olaky/unique-timestamp-replacement-literals.

Authored-by: Ole Sasse 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/optimizer/finishAnalysis.scala| 15 ---
 .../optimizer/ComputeCurrentTimeSuite.scala| 30 +-
 2 files changed, 40 insertions(+), 5 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
index 4052ccd6496..18c85999312 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
-import java.time.{Instant, LocalDateTime}
+import java.time.{Instant, LocalDateTime, ZoneId}
 
 import org.apache.spark.sql.catalyst.CurrentUserContext
 import org.apache.spark.sql.catalyst.expressions._
@@ -79,6 +79,8 @@ object ComputeCurrentTime extends Rule[LogicalPlan] {
 val currentTimestampMicros = instantToMicros(instant)
 val currentTime = Literal.create(currentTimestampMicros, TimestampType)
 val timezone = Literal.create(conf.sessionLocalTimeZone, StringType)
+val currentDates = collection.mutable.HashMap.empty[ZoneId, Literal]
+val localTimestamps = collection.mutable.HashMap.empty[ZoneId, Literal]
 
 def transformCondition(treePatternbits: TreePatternBits): Boolean = {
   treePatternbits.containsPattern(CURRENT_LIKE)
@@ -88,12 +90,17 @@ object ComputeCurrentTime extends Rule[LogicalPlan] {
   case subQuery =>
 subQuery.transformAllExpressionsWithPruning(transformCondition) {
   case cd: CurrentDate =>
-Literal.create(DateTimeUtils.microsToDays(currentTimestampMicros, 
cd.zoneId), DateType)
+currentDates.getOrElseUpdate(cd.zoneId, {
+  Literal.create(
+DateTimeUtils.microsToDays(currentTimestampMicros, cd.zoneId), 
DateType)
+})
   case CurrentTimestamp() | Now() => currentTime
   case CurrentTimeZone() => timezone
   case localTimestamp: LocalTimestamp =>
-val asDateTime = LocalDateTime.ofInstant(instant, 
localTimestamp.zoneId)
-Literal.create(localDateTimeToMicros(asDateTime), TimestampNTZType)
+localTimestamps.getOrElseUpdate(localTimestamp.zoneId, {
+  val asDateTime = LocalDateTime.ofInstant(instant, 
localTimestamp.zoneId)
+  Literal.create(localDateTimeToMicros(asDateTime), 
TimestampNTZType)
+})
 }
 }
   }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala
index 8b76cc383c5..447d77855fb 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ComputeCurrentTimeSuite.scala
@@ -23,7 +23,7 @@ import scala.concurrent.duration._
 import scala.jdk.CollectionConverters.MapHasAsScala
 
 import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentDate, 
CurrentTimestamp, CurrentTimeZone, InSubquery, ListQuery, Literal, 
LocalTimestamp, Now}
+import org.apache.spark.sql.catalyst.expressions.{Ali

[spark] branch master updated: [SPARK-45561][SQL] Add proper conversions for TINYINT in MySQLDialect

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

maxgekk 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 5092c8970246 [SPARK-45561][SQL] Add proper conversions for TINYINT in 
MySQLDialect
5092c8970246 is described below

commit 5092c8970246eb828a31154796c3b16f0b61bddd
Author: Michael Zhang 
AuthorDate: Tue Oct 24 14:51:45 2023 +0500

[SPARK-45561][SQL] Add proper conversions for TINYINT in MySQLDialect

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

Change MySql Dialect to convert catalyst TINYINT into MySQL TINYINT rather 
than BYTE and INTEGER. BYTE does not exist in MySQL. The same applies to 
MsSqlServerDialect.

### Why are the changes needed?

Since BYTE type does not exist in MySQL, any casts that could be pushed 
down involving BYTE type would fail.

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

No

### How was this patch tested?

UT pass.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #43390 from michaelzhan-db/SPARK-45561.

Lead-authored-by: Michael Zhang 
Co-authored-by: Wenchen Fan 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala   | 8 +---
 .../src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala   | 5 -
 sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 ++
 3 files changed, 11 insertions(+), 4 deletions(-)

diff --git 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
index 429404168d1e..7116bcc7de3e 100644
--- 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
+++ 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
@@ -56,10 +56,10 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 
 conn.prepareStatement("CREATE TABLE numbers (onebit BIT(1), tenbits 
BIT(10), "
   + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci 
DECIMAL(40,20), flt FLOAT, "
-  + "dbl DOUBLE)").executeUpdate()
+  + "dbl DOUBLE, tiny TINYINT)").executeUpdate()
 conn.prepareStatement("INSERT INTO numbers VALUES (b'0', b'1000100101', "
   + "17, 7, 123456789, 123456789012345, 
123456789012345.123456789012345, "
-  + "42.75, 1.0002)").executeUpdate()
+  + "42.75, 1.0002, -128)").executeUpdate()
 
 conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, dt DATETIME, ts 
TIMESTAMP, "
   + "yr YEAR)").executeUpdate()
@@ -89,7 +89,7 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 val rows = df.collect()
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
-assert(types.length == 9)
+assert(types.length == 10)
 assert(types(0).equals("class java.lang.Boolean"))
 assert(types(1).equals("class java.lang.Long"))
 assert(types(2).equals("class java.lang.Integer"))
@@ -99,6 +99,7 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 assert(types(6).equals("class java.math.BigDecimal"))
 assert(types(7).equals("class java.lang.Double"))
 assert(types(8).equals("class java.lang.Double"))
+assert(types(9).equals("class java.lang.Byte"))
 assert(rows(0).getBoolean(0) == false)
 assert(rows(0).getLong(1) == 0x225)
 assert(rows(0).getInt(2) == 17)
@@ -109,6 +110,7 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 assert(rows(0).getAs[BigDecimal](6).equals(bd))
 assert(rows(0).getDouble(7) == 42.75)
 assert(rows(0).getDouble(8) == 1.0002)
+assert(rows(0).getByte(9) == 0x80.toByte)
   }
 
   test("Date types") {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
index 3c6d02d86412..dd74c93bc2e1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.index.TableIndex
 import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, 
NamedReference, NullOrdering, SortDirection}
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
-import org.apache.spark.sql.types.{BooleanTy

[spark] branch branch-3.5 updated: [SPARK-45561][SQL] Add proper conversions for TINYINT in MySQLDialect

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new ddccf5add8f5 [SPARK-45561][SQL] Add proper conversions for TINYINT in 
MySQLDialect
ddccf5add8f5 is described below

commit ddccf5add8f5aa35693c4120f0b161a74379aec9
Author: Michael Zhang 
AuthorDate: Tue Oct 24 14:51:45 2023 +0500

[SPARK-45561][SQL] Add proper conversions for TINYINT in MySQLDialect

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

Change MySql Dialect to convert catalyst TINYINT into MySQL TINYINT rather 
than BYTE and INTEGER. BYTE does not exist in MySQL. The same applies to 
MsSqlServerDialect.

### Why are the changes needed?

Since BYTE type does not exist in MySQL, any casts that could be pushed 
down involving BYTE type would fail.

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

No

### How was this patch tested?

UT pass.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #43390 from michaelzhan-db/SPARK-45561.

Lead-authored-by: Michael Zhang 
Co-authored-by: Wenchen Fan 
Signed-off-by: Max Gekk 
(cherry picked from commit 5092c8970246eb828a31154796c3b16f0b61bddd)
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala   | 8 +---
 .../src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala   | 5 -
 sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 ++
 3 files changed, 11 insertions(+), 4 deletions(-)

diff --git 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
index dc3acb66ff1f..20fdc965874f 100644
--- 
a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
+++ 
b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala
@@ -56,10 +56,10 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 
 conn.prepareStatement("CREATE TABLE numbers (onebit BIT(1), tenbits 
BIT(10), "
   + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci 
DECIMAL(40,20), flt FLOAT, "
-  + "dbl DOUBLE)").executeUpdate()
+  + "dbl DOUBLE, tiny TINYINT)").executeUpdate()
 conn.prepareStatement("INSERT INTO numbers VALUES (b'0', b'1000100101', "
   + "17, 7, 123456789, 123456789012345, 
123456789012345.123456789012345, "
-  + "42.75, 1.0002)").executeUpdate()
+  + "42.75, 1.0002, -128)").executeUpdate()
 
 conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, dt DATETIME, ts 
TIMESTAMP, "
   + "yr YEAR)").executeUpdate()
@@ -89,7 +89,7 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 val rows = df.collect()
 assert(rows.length == 1)
 val types = rows(0).toSeq.map(x => x.getClass.toString)
-assert(types.length == 9)
+assert(types.length == 10)
 assert(types(0).equals("class java.lang.Boolean"))
 assert(types(1).equals("class java.lang.Long"))
 assert(types(2).equals("class java.lang.Integer"))
@@ -99,6 +99,7 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 assert(types(6).equals("class java.math.BigDecimal"))
 assert(types(7).equals("class java.lang.Double"))
 assert(types(8).equals("class java.lang.Double"))
+assert(types(9).equals("class java.lang.Byte"))
 assert(rows(0).getBoolean(0) == false)
 assert(rows(0).getLong(1) == 0x225)
 assert(rows(0).getInt(2) == 17)
@@ -109,6 +110,7 @@ class MySQLIntegrationSuite extends 
DockerJDBCIntegrationSuite {
 assert(rows(0).getAs[BigDecimal](6).equals(bd))
 assert(rows(0).getDouble(7) == 42.75)
 assert(rows(0).getDouble(8) == 1.0002)
+assert(rows(0).getByte(9) == 0x80.toByte)
   }
 
   test("Date types") {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
index a08c89318b66..c7e14cc78d5b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.index.TableIndex
 import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, 
NamedReference, NullOrdering, SortDirection}
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apa

[spark] branch master updated: [SPARK-45626][SQL] Convert _LEGACY_ERROR_TEMP_1055 to REQUIRES_SINGLE_PART_NAMESPACE

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

maxgekk 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 38222c4e8c58 [SPARK-45626][SQL] Convert _LEGACY_ERROR_TEMP_1055 to 
REQUIRES_SINGLE_PART_NAMESPACE
38222c4e8c58 is described below

commit 38222c4e8c581502e65a4224c3c4201136f7e390
Author: panbingkun 
AuthorDate: Tue Oct 24 12:47:42 2023 +0500

[SPARK-45626][SQL] Convert _LEGACY_ERROR_TEMP_1055 to 
REQUIRES_SINGLE_PART_NAMESPACE

### What changes were proposed in this pull request?
The pr aims to convert `_LEGACY_ERROR_TEMP_1055` to 
`REQUIRES_SINGLE_PART_NAMESPACE`
PS: In addition, the variable name (`quoted` -> `database`) was incorrect 
in the original error class message template, and this issue was resolved in 
the above `convert`.

### Why are the changes needed?
Fix bug.

- Before:
   https://github.com/apache/spark/assets/15246973/e7a59837-4f14-4f09-872a-913d78006ede;>

- After:
   https://github.com/apache/spark/assets/15246973/5d3928bf-e580-44b1-a86d-55654686e8d5;>

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

### How was this patch tested?
- Pass GA.
- Manually test:
```
./build/sbt "hive/testOnly 
org.apache.spark.sql.hive.execution.command.CreateNamespaceSuite -- -z 
\"REQUIRES_SINGLE_PART_NAMESPACE\""

[info] CreateNamespaceSuite:
[info] - CREATE NAMESPACE using Hive V1 catalog V1 command: 
REQUIRES_SINGLE_PART_NAMESPACE (392 milliseconds)
[info] - CREATE NAMESPACE using Hive V1 catalog V2 command: 
REQUIRES_SINGLE_PART_NAMESPACE (3 milliseconds)
15:24:14.648 WARN org.apache.hadoop.hive.metastore.ObjectStore: Version 
information not found in metastore. hive.metastore.schema.verification is not 
enabled so recording the schema version 2.3.0
15:24:14.648 WARN org.apache.hadoop.hive.metastore.ObjectStore: 
setMetaStoreSchemaVersion called but recording version is disabled: version = 
2.3.0, comment = Set by MetaStore panbingkun172.24.144.16
15:24:14.654 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to 
get database default, returning NoSuchObjectException
15:24:14.795 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to 
get database global_temp, returning NoSuchObjectException
15:24:15.007 WARN org.apache.hadoop.hive.ql.session.SessionState: 
METASTORE_FILTER_HOOK will be ignored, since 
hive.security.authorization.manager is set to instance of HiveAuthorizerFactory.
[info] Run completed in 4 seconds, 819 milliseconds.
[info] Total number of tests run: 2
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 2, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 186 s (03:06), completed Oct 24, 2023, 3:24:15 PM
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43479 from panbingkun/SPARK-45626.

Authored-by: panbingkun 
Signed-off-by: Max Gekk 
---
 common/utils/src/main/resources/error/error-classes.json  |  5 -
 docs/sql-error-conditions.md  |  2 --
 .../apache/spark/sql/errors/QueryCompilationErrors.scala  |  6 --
 .../sql/catalyst/analysis/ResolveSessionCatalog.scala |  2 +-
 .../sql/hive/execution/command/CreateNamespaceSuite.scala | 15 +++
 5 files changed, 16 insertions(+), 14 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 5ef70b583dfc..53c76b77c01e 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -4184,11 +4184,6 @@
   "ALTER COLUMN cannot find column  in v1 table. Available: 
."
 ]
   },
-  "_LEGACY_ERROR_TEMP_1055" : {
-"message" : [
-  "The database name is not valid: ."
-]
-  },
   "_LEGACY_ERROR_TEMP_1057" : {
 "message" : [
   "SHOW COLUMNS with conflicting databases: '' != ''."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index ce39fce85e79..20665e1018d5 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2310,5 +2310,3 @@ The operation `` requires a ``. 
But `` is a
 The `` requires `` parameters but the actual number 
is ``.
 
 For more details see 
[WRONG_NUM_ARGS](sql-error-conditions-wrong-num-args-error-class.html)
-
-
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 92b1ace67d4d..499fdc4d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql

[spark] branch master updated: [SPARK-45574][SQL] Add :: syntax as a shorthand for casting

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

maxgekk 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 936e98dbc07 [SPARK-45574][SQL] Add :: syntax as a shorthand for casting
936e98dbc07 is described below

commit 936e98dbc073fcfcb7e6c40720d55dac63a73d51
Author: Ivan Mitic 
AuthorDate: Sun Oct 22 11:23:01 2023 +0500

[SPARK-45574][SQL] Add :: syntax as a shorthand for casting

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

Adds the `::` syntax as syntactic sugar for casting columns. This is a 
pretty common syntax across many industry databases.

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

Yes, new casting syntax.

### How was this patch tested?

Unit tests.
SQL tests.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #43430 from mitkedb/master.

Authored-by: Ivan Mitic 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/parser/SqlBaseLexer.g4  |   1 +
 .../spark/sql/catalyst/parser/SqlBaseParser.g4 |   1 +
 .../spark/sql/catalyst/expressions/Cast.scala  |   5 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala |  11 +
 .../sql/catalyst/parser/CastingSyntaxSuite.scala   | 103 ++
 .../sql-tests/analyzer-results/ansi/cast.sql.out   | 234 +
 .../sql-tests/analyzer-results/cast.sql.out| 217 
 .../src/test/resources/sql-tests/inputs/cast.sql   |  31 ++
 .../resources/sql-tests/results/ansi/cast.sql.out  | 385 +
 .../test/resources/sql-tests/results/cast.sql.out  | 245 +
 .../spark/sql/errors/QueryParsingErrorsSuite.scala |   4 +-
 .../sql/expressions/ExpressionInfoSuite.scala  |   4 +-
 12 files changed, 1237 insertions(+), 4 deletions(-)

diff --git 
a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 
b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
index d9128de0f5d..e8b5cb012fc 100644
--- 
a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
+++ 
b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
@@ -447,6 +447,7 @@ PIPE: '|';
 CONCAT_PIPE: '||';
 HAT: '^';
 COLON: ':';
+DOUBLE_COLON: '::';
 ARROW: '->';
 FAT_ARROW : '=>';
 HENT_START: '/*+';
diff --git 
a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 
b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4
index 77a9108e063..84a31dafed9 100644
--- 
a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4
+++ 
b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4
@@ -957,6 +957,7 @@ primaryExpression
 | CASE whenClause+ (ELSE elseExpression=expression)? END   
#searchedCase
 | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END  
#simpleCase
 | name=(CAST | TRY_CAST) LEFT_PAREN expression AS dataType RIGHT_PAREN 
#cast
+| primaryExpression DOUBLE_COLON dataType  
#castByColon
 | STRUCT LEFT_PAREN (argument+=namedExpression (COMMA 
argument+=namedExpression)*)? RIGHT_PAREN #struct
 | FIRST LEFT_PAREN expression (IGNORE NULLS)? RIGHT_PAREN  
#first
 | ANY_VALUE LEFT_PAREN expression (IGNORE NULLS)? RIGHT_PAREN  
#any_value
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index b975dc3c7a5..99117d81b34 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -438,11 +438,14 @@ object Cast extends QueryErrorsBase {
  * session local timezone by an analyzer [[ResolveTimeZone]].
  */
 @ExpressionDescription(
-  usage = "_FUNC_(expr AS type) - Casts the value `expr` to the target data 
type `type`.",
+  usage = "_FUNC_(expr AS type) - Casts the value `expr` to the target data 
type `type`." +
+  " `expr` :: `type` alternative casting syntax is also supported.",
   examples = """
 Examples:
   > SELECT _FUNC_('10' as int);
10
+  > SELECT '10' :: int;
+   10
   """,
   since = "1.0.0",
   group = "conversion_funcs")
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 8ce58ef7688..7e0aafca31c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/sp

[spark] branch master updated: [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader

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

maxgekk 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 13b67ee8cc3 [SPARK-45604][SQL] Add LogicalType checking on INT64 -> 
DateTime conversion on Parquet Vectorized Reader
13b67ee8cc3 is described below

commit 13b67ee8cc377a5cc47d02b9addbc00eabfc8b6c
Author: Zamil Majdy 
AuthorDate: Sun Oct 22 10:53:22 2023 +0500

[SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion 
on Parquet Vectorized Reader

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

Currently, the read logical type is not checked while converting physical 
types INT64 into DateTime. One valid scenario where this can break is where the 
physical type is `timestamp_ntz`, and the logical type is 
`array`, since the logical type check does not happen, this 
conversion is allowed. However, the vectorized reader does not support this and 
will produce NPE on on-heap memory mode and SEGFAULT on off-heap memory mode. 
Segmentation fault on off-heap memory mode c [...]

### Why are the changes needed?
Prevent NPE or Segfault from happening.

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

No.

### How was this patch tested?

A new test is added in `ParquetSchemaSuite`.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #43451 from majdyz/SPARK-45604.

Lead-authored-by: Zamil Majdy 
Co-authored-by: Zamil Majdy 
Signed-off-by: Max Gekk 
---
 .../parquet/ParquetVectorUpdaterFactory.java| 10 --
 .../datasources/parquet/ParquetSchemaSuite.scala| 21 +
 2 files changed, 29 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
index d5675db4c3a..26bef0fe3a6 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
@@ -109,7 +109,8 @@ public class ParquetVectorUpdaterFactory {
   // For unsigned int64, it stores as plain signed int64 in Parquet 
when dictionary
   // fallbacks. We read them as decimal values.
   return new UnsignedLongUpdater();
-} else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
+} else if (isTimestamp(sparkType) &&
+isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
   validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongUpdater();
@@ -117,7 +118,8 @@ public class ParquetVectorUpdaterFactory {
 boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode);
 return new LongWithRebaseUpdater(failIfRebase, datetimeRebaseTz);
   }
-} else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
+} else if (isTimestamp(sparkType) &&
+isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
   validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongAsMicrosUpdater();
@@ -1149,6 +1151,10 @@ public class ParquetVectorUpdaterFactory {
 return false;
   }
 
+  private static boolean isTimestamp(DataType dt) {
+return dt == DataTypes.TimestampType || dt == DataTypes.TimestampNTZType;
+  }
+
   private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, 
DataType dt) {
 DecimalType d = (DecimalType) dt;
 LogicalTypeAnnotation typeAnnotation = 
descriptor.getPrimitiveType().getLogicalTypeAnnotation();
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
index ef06e64d2eb..19feb9b8bb5 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
@@ -1087,6 +1087,27 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
 }
   }
 
+  test("SPARK-45604: schema mismatch failure error on timestamp_ntz to 
array") {
+import testImplicits._
+
+withTempPath { dir =>
+  val path = dir.getCanonicalPath
+  val timestamp = java.time.LocalDateTime.of(1, 2, 3, 4, 5)
+  val df1 = Seq((1, timestamp)).toDF()
+  val df2 = Seq((2, Array(times

[spark] branch branch-3.4 updated: [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader

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

maxgekk 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 b24bc5c1a27 [SPARK-45604][SQL] Add LogicalType checking on INT64 -> 
DateTime conversion on Parquet Vectorized Reader
b24bc5c1a27 is described below

commit b24bc5c1a27e847143a8159d4291ddac87b7f387
Author: Zamil Majdy 
AuthorDate: Sun Oct 22 10:53:22 2023 +0500

[SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion 
on Parquet Vectorized Reader

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

Currently, the read logical type is not checked while converting physical 
types INT64 into DateTime. One valid scenario where this can break is where the 
physical type is `timestamp_ntz`, and the logical type is 
`array`, since the logical type check does not happen, this 
conversion is allowed. However, the vectorized reader does not support this and 
will produce NPE on on-heap memory mode and SEGFAULT on off-heap memory mode. 
Segmentation fault on off-heap memory mode c [...]

### Why are the changes needed?
Prevent NPE or Segfault from happening.

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

No.

### How was this patch tested?

A new test is added in `ParquetSchemaSuite`.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #43451 from majdyz/SPARK-45604.

Lead-authored-by: Zamil Majdy 
Co-authored-by: Zamil Majdy 
Signed-off-by: Max Gekk 
(cherry picked from commit 13b67ee8cc377a5cc47d02b9addbc00eabfc8b6c)
Signed-off-by: Max Gekk 
---
 .../parquet/ParquetVectorUpdaterFactory.java| 10 --
 .../datasources/parquet/ParquetSchemaSuite.scala| 21 +
 2 files changed, 29 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
index 15d58f0c757..42442cf8ea8 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
@@ -109,7 +109,8 @@ public class ParquetVectorUpdaterFactory {
   // For unsigned int64, it stores as plain signed int64 in Parquet 
when dictionary
   // fallbacks. We read them as decimal values.
   return new UnsignedLongUpdater();
-} else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
+} else if (isTimestamp(sparkType) &&
+isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
   validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongUpdater();
@@ -117,7 +118,8 @@ public class ParquetVectorUpdaterFactory {
 boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode);
 return new LongWithRebaseUpdater(failIfRebase, datetimeRebaseTz);
   }
-} else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
+} else if (isTimestamp(sparkType) &&
+isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
   validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongAsMicrosUpdater();
@@ -1150,6 +1152,10 @@ public class ParquetVectorUpdaterFactory {
 return false;
   }
 
+  private static boolean isTimestamp(DataType dt) {
+return dt == DataTypes.TimestampType || dt == DataTypes.TimestampNTZType;
+  }
+
   private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, 
DataType dt) {
 DecimalType d = (DecimalType) dt;
 LogicalTypeAnnotation typeAnnotation = 
descriptor.getPrimitiveType().getLogicalTypeAnnotation();
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
index 5589c61be7a..bf5c51b89bb 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
@@ -1067,6 +1067,27 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
 }
   }
 
+  test("SPARK-45604: schema mismatch failure error on timestamp_ntz to 
array") {
+import testImplicits._
+
+withTempPath { dir =>
+  val path = dir.getCanonicalPath
+  val timestamp = java.time.L

[spark] branch branch-3.5 updated: [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 6c55a6c0c68 [SPARK-45604][SQL] Add LogicalType checking on INT64 -> 
DateTime conversion on Parquet Vectorized Reader
6c55a6c0c68 is described below

commit 6c55a6c0c680f80a6cdef7f1a83045b6400b4d09
Author: Zamil Majdy 
AuthorDate: Sun Oct 22 10:53:22 2023 +0500

[SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion 
on Parquet Vectorized Reader

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

Currently, the read logical type is not checked while converting physical 
types INT64 into DateTime. One valid scenario where this can break is where the 
physical type is `timestamp_ntz`, and the logical type is 
`array`, since the logical type check does not happen, this 
conversion is allowed. However, the vectorized reader does not support this and 
will produce NPE on on-heap memory mode and SEGFAULT on off-heap memory mode. 
Segmentation fault on off-heap memory mode c [...]

### Why are the changes needed?
Prevent NPE or Segfault from happening.

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

No.

### How was this patch tested?

A new test is added in `ParquetSchemaSuite`.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #43451 from majdyz/SPARK-45604.

Lead-authored-by: Zamil Majdy 
Co-authored-by: Zamil Majdy 
Signed-off-by: Max Gekk 
(cherry picked from commit 13b67ee8cc377a5cc47d02b9addbc00eabfc8b6c)
Signed-off-by: Max Gekk 
---
 .../parquet/ParquetVectorUpdaterFactory.java| 10 --
 .../datasources/parquet/ParquetSchemaSuite.scala| 21 +
 2 files changed, 29 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
index 15d58f0c757..42442cf8ea8 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java
@@ -109,7 +109,8 @@ public class ParquetVectorUpdaterFactory {
   // For unsigned int64, it stores as plain signed int64 in Parquet 
when dictionary
   // fallbacks. We read them as decimal values.
   return new UnsignedLongUpdater();
-} else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
+} else if (isTimestamp(sparkType) &&
+isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) {
   validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongUpdater();
@@ -117,7 +118,8 @@ public class ParquetVectorUpdaterFactory {
 boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode);
 return new LongWithRebaseUpdater(failIfRebase, datetimeRebaseTz);
   }
-} else if 
(isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
+} else if (isTimestamp(sparkType) &&
+isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) {
   validateTimestampType(sparkType);
   if ("CORRECTED".equals(datetimeRebaseMode)) {
 return new LongAsMicrosUpdater();
@@ -1150,6 +1152,10 @@ public class ParquetVectorUpdaterFactory {
 return false;
   }
 
+  private static boolean isTimestamp(DataType dt) {
+return dt == DataTypes.TimestampType || dt == DataTypes.TimestampNTZType;
+  }
+
   private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, 
DataType dt) {
 DecimalType d = (DecimalType) dt;
 LogicalTypeAnnotation typeAnnotation = 
descriptor.getPrimitiveType().getLogicalTypeAnnotation();
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
index facc9b90ff7..3f47c5e506f 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
@@ -1087,6 +1087,27 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
 }
   }
 
+  test("SPARK-45604: schema mismatch failure error on timestamp_ntz to 
array") {
+import testImplicits._
+
+withTempPath { dir =>
+  val path = dir.getCanonicalPath
+  val timestamp = java.time.L

[spark] branch master updated: [SPARK-44837][SQL] Change ALTER TABLE ALTER PARTITION column error message

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

maxgekk 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 3c847c6c11f [SPARK-44837][SQL] Change ALTER TABLE ALTER PARTITION 
column error message
3c847c6c11f is described below

commit 3c847c6c11f6def15fced7bae55d0ccdeac1eb72
Author: Michael Zhang 
AuthorDate: Fri Oct 20 10:55:50 2023 +0500

[SPARK-44837][SQL] Change ALTER TABLE ALTER PARTITION column error message

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

Improve the error message for `ALTER TABLE ALTER COLUMN` command on 
partition columns.

### Why are the changes needed?

Currently, if a user executes `ALTER TABLE ALTER COLUMN` on a partition 
column, the error message provided is cryptic and does not help users. The 
improved error message makes it clear that the command is not supported for 
partition columns.

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

New output when the command is executed on a partition column.

> `org.apache.spark.sql.AnalysisException: [CANNOT_ALTER_PARTITION_COLUMN] 
ALTER TABLE (ALTER|CHANGE) COLUMN is not supported for partition columns, but 
found the partition column `i` in the table non-delta table 
spark_catalog.default.t.
>   at 
org.apache.spark.sql.errors.QueryCompilationErrors$.alterTableChangeColumnNotSupportedForPartitionColumn(QueryCompilationErrors.scala:2586)`

Old error message when the `ALTER TABLE ALTER COLUMN` is executed on a 
partition column

> `org.apache.spark.sql.AnalysisException: Can't find column `i` given 
table data columns [`k`].
>   at 
org.apache.spark.sql.errors.QueryCompilationErrors$.cannotFindColumnError(QueryCompilationErrors.scala:2843)
>   at 
org.apache.spark.sql.execution.command.AlterTableChangeColumnCommand.$anonfun$findColumnByName$2(ddl.scala:490)
>   at scala.Option.getOrElse(Option.scala:189)
>   at 
org.apache.spark.sql.execution.command.AlterTableChangeColumnCommand.findColumnByName(ddl.scala:490)
>   at 
org.apache.spark.sql.execution.command.AlterTableChangeColumnCommand.run(ddl.scala:447)`

### How was this patch tested?

All tests pass.

Closes #42524 from michaelzhan-db/alter-partition-column-error-message.

Lead-authored-by: Michael Zhang 
Co-authored-by: michaelzhan-db 
Signed-off-by: Max Gekk 
---
 common/utils/src/main/resources/error/error-classes.json  |  6 ++
 docs/sql-error-conditions.md  |  8 
 .../apache/spark/sql/errors/QueryCompilationErrors.scala  |  9 +
 .../org/apache/spark/sql/execution/command/ddl.scala  |  4 
 .../org/apache/spark/sql/execution/command/DDLSuite.scala | 15 +++
 .../scala/org/apache/spark/sql/sources/InsertSuite.scala  |  5 +++--
 6 files changed, 45 insertions(+), 2 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 5082165a4b3..5731022b6f4 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -104,6 +104,12 @@
 ],
 "sqlState" : "42KDE"
   },
+  "CANNOT_ALTER_PARTITION_COLUMN" : {
+"message" : [
+  "ALTER TABLE (ALTER|CHANGE) COLUMN is not supported for partition 
columns, but found the partition column  in the table ."
+],
+"sqlState" : "428FR"
+  },
   "CANNOT_CAST_DATATYPE" : {
 "message" : [
   "Cannot cast  to ."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index f22e5273746..d2100b5505b 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -118,6 +118,12 @@ Unable to find batch ``.
 
 The method `` can not be called on streaming Dataset/DataFrame.
 
+### CANNOT_ALTER_PARTITION_COLUMN
+
+[SQLSTATE: 
428FR](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+ALTER TABLE (ALTER|CHANGE) COLUMN is not supported for partition columns, but 
found the partition column `` in the table ``.
+
 ### CANNOT_CAST_DATATYPE
 
 [SQLSTATE: 
42846](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
@@ -2298,3 +2304,5 @@ The operation `` requires a ``. 
But `` is a
 The `` requires `` parameters but the actual number 
is ``.
 
 For more details see 
[WRONG_NUM_ARGS](sql-error-conditions-wrong-num-args-error-class.html)
+
+
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 5dfdd7757ad..1009c499aa3 100644
--- 
a/sql/ca

[spark] branch master updated: [SPARK-45569][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2152

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

maxgekk 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 69fa51cc21f [SPARK-45569][SQL] Assign name to the error 
_LEGACY_ERROR_TEMP_2152
69fa51cc21f is described below

commit 69fa51cc21f63c7ec1edfb79a535bd6024546489
Author: dengziming 
AuthorDate: Thu Oct 19 17:23:24 2023 +0500

[SPARK-45569][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2152

### What changes were proposed in this pull request?
Assign the name `EXPRESSION_ENCODING_FAILED` to the legacy error class 
`_LEGACY_ERROR_TEMP_2152`.

### Why are the changes needed?
To assign proper name as a part of activity in SPARK-37935.

### Does this PR introduce _any_ user-facing change?
Yes, the error message will include the error class name

### How was this patch tested?
Add a unit test to produce the error from user code.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43396 from dengziming/SPARK-45569.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 .../utils/src/main/resources/error/error-classes.json | 12 ++--
 docs/sql-error-conditions.md  |  6 ++
 .../spark/sql/errors/QueryExecutionErrors.scala   |  3 +--
 .../catalyst/encoders/ExpressionEncoderSuite.scala|  2 +-
 .../spark/sql/catalyst/encoders/RowEncoderSuite.scala | 12 ++--
 .../scala/org/apache/spark/sql/DatasetSuite.scala | 19 +--
 6 files changed, 37 insertions(+), 17 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index dc65d7347fe..5082165a4b3 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -985,6 +985,12 @@
 ],
 "sqlState" : "42846"
   },
+  "EXPRESSION_ENCODING_FAILED" : {
+"message" : [
+  "Failed to encode a value of the expressions:  to a row."
+],
+"sqlState" : "42846"
+  },
   "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : {
 "message" : [
   "Column expression  cannot be sorted because its type  
is not orderable."
@@ -5718,12 +5724,6 @@
   "Due to Scala's limited support of tuple, tuple with more than 22 
elements are not supported."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2152" : {
-"message" : [
-  "Error while encoding: ",
-  "."
-]
-  },
   "_LEGACY_ERROR_TEMP_2154" : {
 "message" : [
   "Failed to get outer pointer for ."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index e555ae7476b..f22e5273746 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -574,6 +574,12 @@ For more details see 
[EXPECT_VIEW_NOT_TABLE](sql-error-conditions-expect-view-no
 
 Failed to decode a row to a value of the expressions: ``.
 
+### EXPRESSION_ENCODING_FAILED
+
+[SQLSTATE: 
42846](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+Failed to encode a value of the expressions: `` to a row.
+
 ### EXPRESSION_TYPE_IS_NOT_ORDERABLE
 
 [SQLSTATE: 
42822](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
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 957e87b4d3f..a11b929919d 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
@@ -1351,9 +1351,8 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
 
   def expressionEncodingError(e: Exception, expressions: Seq[Expression]): 
SparkRuntimeException = {
 new SparkRuntimeException(
-  errorClass = "_LEGACY_ERROR_TEMP_2152",
+  errorClass = "EXPRESSION_ENCODING_FAILED",
   messageParameters = Map(
-"e" -> e.toString(),
 "expressions" -> expressions.map(
   _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")),
   cause = e)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
index a3ea3a462b1..8373f53446f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark

[spark] branch master updated: [SPARK-45569][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2153

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

maxgekk 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 7057952f6bc [SPARK-45569][SQL] Assign name to the error 
_LEGACY_ERROR_TEMP_2153
7057952f6bc is described below

commit 7057952f6bc2c5cf97dd408effd1b18bee1cb8f4
Author: dengziming 
AuthorDate: Thu Oct 19 11:14:55 2023 +0500

[SPARK-45569][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2153

### What changes were proposed in this pull request?
Assign the name `UNEXPECTED_SERIALIZER_FOR_CLASS` to the legacy error class 
`_LEGACY_ERROR_TEMP_2153`.

### Why are the changes needed?
To assign proper name as a part of activity in SPARK-37935.

### Does this PR introduce _any_ user-facing change?
Yes, the error message will include the error class name

### How was this patch tested?
Add a unit test to produce the error from user code.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43414 from dengziming/SPARK-45573.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 11 
 docs/sql-error-conditions.md   |  6 +
 .../spark/sql/errors/QueryExecutionErrors.scala|  6 ++---
 .../catalyst/encoders/ExpressionEncoderSuite.scala | 29 +++---
 4 files changed, 40 insertions(+), 12 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 9e442f4c666..dc65d7347fe 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3017,6 +3017,12 @@
 ],
 "sqlState" : "4274K"
   },
+  "UNEXPECTED_SERIALIZER_FOR_CLASS" : {
+"message" : [
+  "The class  has an unexpected expression serializer. Expects 
\"STRUCT\" or \"IF\" which returns \"STRUCT\" but found ."
+],
+"sqlState" : "42846"
+  },
   "UNKNOWN_PROTOBUF_MESSAGE_TYPE" : {
 "message" : [
   "Attempting to treat  as a Message, but it was 
."
@@ -5718,11 +5724,6 @@
   "."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2153" : {
-"message" : [
-  "class  has unexpected serializer: ."
-]
-  },
   "_LEGACY_ERROR_TEMP_2154" : {
 "message" : [
   "Failed to get outer pointer for ."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index d8f5193c9bc..e555ae7476b 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -1941,6 +1941,12 @@ Parameter `` of function `` 
requires the `` because it contains positional 
argument(s) following the named argument assigned to ``; please 
rearrange them so the positional arguments come first and then retry the query 
again.
 
+### UNEXPECTED_SERIALIZER_FOR_CLASS
+
+[SQLSTATE: 
42846](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+The class `` has an unexpected expression serializer. Expects 
"STRUCT" or "IF" which returns "STRUCT" but found ``.
+
 ### UNKNOWN_PROTOBUF_MESSAGE_TYPE
 
 [SQLSTATE: 
42K0G](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
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 8d7819de052..957e87b4d3f 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
@@ -1362,10 +1362,10 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
   def classHasUnexpectedSerializerError(
   clsName: String, objSerializer: Expression): SparkRuntimeException = {
 new SparkRuntimeException(
-  errorClass = "_LEGACY_ERROR_TEMP_2153",
+  errorClass = "UNEXPECTED_SERIALIZER_FOR_CLASS",
   messageParameters = Map(
-"clsName" -> clsName,
-"objSerializer" -> objSerializer.toString()))
+"className" -> clsName,
+"expr" -> toSQLExpr(objSerializer)))
   }
 
   def unsupportedOperandTypeForSizeFunctionError(
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
index 9d2051b01d6..a3ea3a462b1 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/c

[spark] branch master updated: [SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error

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

maxgekk 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 11e7ea4f11d [SPARK-45035][SQL] Fix 
ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will report error
11e7ea4f11d is described below

commit 11e7ea4f11df71e2942322b01fcaab57dac20c83
Author: Jia Fan 
AuthorDate: Wed Oct 18 11:06:43 2023 +0500

[SPARK-45035][SQL] Fix ignoreCorruptFiles/ignoreMissingFiles with multiline 
CSV/JSON will report error

### What changes were proposed in this pull request?
Fix ignoreCorruptFiles/ignoreMissingFiles with multiline CSV/JSON will 
report error, it would be like:
```log
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 
in stage 4940.0 failed 4 times, most recent failure: Lost task 0.3 in stage 
4940.0 (TID 4031) (10.68.177.106 executor 0): 
com.univocity.parsers.common.TextParsingException: 
java.lang.IllegalStateException - Error reading from input
Parser Configuration: CsvParserSettings:
Auto configuration enabled=true
Auto-closing enabled=true
Autodetect column delimiter=false
Autodetect quotes=false
Column reordering enabled=true
Delimiters for detection=null
Empty value=
Escape unquoted values=false
Header extraction enabled=null
Headers=null
Ignore leading whitespaces=false
Ignore leading whitespaces in quotes=false
Ignore trailing whitespaces=false
Ignore trailing whitespaces in quotes=false
Input buffer size=1048576
Input reading on separate thread=false
Keep escape sequences=false
Keep quotes=false
Length of content displayed on error=1000
Line separator detection enabled=true
Maximum number of characters per column=-1
Maximum number of columns=20480
Normalize escaped line separators=true
Null value=
Number of records to read=all
Processor=none
Restricting data in exceptions=false
RowProcessor error handler=null
Selected fields=none
Skip bits as whitespace=true
Skip empty lines=true
Unescaped quote handling=STOP_AT_DELIMITERFormat configuration:
CsvFormat:
Comment character=#
Field delimiter=,
Line separator (normalized)=\n
Line separator sequence=\n
Quote character="
Quote escape character=\
Quote escape escape character=null
Internal state when error was thrown: line=0, column=0, record=0
at 
com.univocity.parsers.common.AbstractParser.handleException(AbstractParser.java:402)
at 
com.univocity.parsers.common.AbstractParser.beginParsing(AbstractParser.java:277)
at 
com.univocity.parsers.common.AbstractParser.beginParsing(AbstractParser.java:843)
at 
org.apache.spark.sql.catalyst.csv.UnivocityParser$$anon$1.(UnivocityParser.scala:463)
at 
org.apache.spark.sql.catalyst.csv.UnivocityParser$.convertStream(UnivocityParser.scala:46...
```
Because multiline CSV/JSON use `BinaryFileRDD` not `FileScanRDD`. Unlike 
`FileScanRDD`, when met corrupt files will check `ignoreCorruptFiles` config to 
avoid report IOException, `BinaryFileRDD` will not report error because it 
return normal `PortableDataStream`. So we should catch it when infer schema in 
lambda function. Also do same thing for `ignoreMissingFiles`.

### Why are the changes needed?
Fix the bug when use mulitline mode with 
ignoreCorruptFiles/ignoreMissingFiles config.

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

### How was this patch tested?
add new test.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #42979 from Hisoka-X/SPARK-45035_csv_multi_line.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/json/JsonInferSchema.scala  | 18 +--
 .../execution/datasources/csv/CSVDataSource.scala  | 28 ---
 .../datasources/CommonFileDataSourceSuite.scala| 28 +++
 .../sql/execution/datasources/csv/CSVSuite.scala   | 58 +-
 .../sql/execution/datasources/json/JsonSuite.scala | 46 -
 5 files changed, 142 insertions(+), 36 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
index 4123c5290b6..4d04b34876c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spar

[spark] branch master updated: [SPARK-44262][SQL] Add `dropTable` and `getInsertStatement` to JdbcDialect

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

maxgekk 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 6994bad5e6e [SPARK-44262][SQL] Add `dropTable` and 
`getInsertStatement` to JdbcDialect
6994bad5e6e is described below

commit 6994bad5e6ea8700d48cbe20e9b406b89925adc7
Author: Jia Fan 
AuthorDate: Mon Oct 16 13:55:24 2023 +0500

[SPARK-44262][SQL] Add `dropTable` and `getInsertStatement` to JdbcDialect

### What changes were proposed in this pull request?
1. This PR add `dropTable` function to `JdbcDialect`. So user can override 
dropTable SQL by other JdbcDialect like Neo4J
Neo4J Drop case
```sql
MATCH (m:Person {name: 'Mark'})
DELETE m
```
2. Also add `getInsertStatement` for same reason.
Neo4J Insert case
```sql
MATCH (p:Person {name: 'Jennifer'})
SET p.birthdate = date('1980-01-01')
RETURN p
```
Neo4J SQL(in fact named `CQL`) not like normal SQL, but it have JDBC driver.

### Why are the changes needed?
Make JdbcDialect more useful

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

### How was this patch tested?
exist test

Closes #41855 from Hisoka-X/SPARK-44262_JDBCUtils_improve.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../sql/execution/datasources/jdbc/JdbcUtils.scala | 14 +--
 .../org/apache/spark/sql/jdbc/JdbcDialects.scala   | 29 ++
 2 files changed, 35 insertions(+), 8 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index fb9e11df188..f2b84810175 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -78,7 +78,8 @@ object JdbcUtils extends Logging with SQLConfHelper {
* Drops a table from the JDBC database.
*/
   def dropTable(conn: Connection, table: String, options: JDBCOptions): Unit = 
{
-executeStatement(conn, options, s"DROP TABLE $table")
+val dialect = JdbcDialects.get(options.url)
+executeStatement(conn, options, dialect.dropTable(table))
   }
 
   /**
@@ -114,22 +115,19 @@ object JdbcUtils extends Logging with SQLConfHelper {
   isCaseSensitive: Boolean,
   dialect: JdbcDialect): String = {
 val columns = if (tableSchema.isEmpty) {
-  rddSchema.fields.map(x => dialect.quoteIdentifier(x.name)).mkString(",")
+  rddSchema.fields
 } else {
   // The generated insert statement needs to follow rddSchema's column 
sequence and
   // tableSchema's column names. When appending data into some 
case-sensitive DBMSs like
   // PostgreSQL/Oracle, we need to respect the existing case-sensitive 
column names instead of
   // RDD column names for user convenience.
-  val tableColumnNames = tableSchema.get.fieldNames
   rddSchema.fields.map { col =>
-val normalizedName = tableColumnNames.find(f => conf.resolver(f, 
col.name)).getOrElse {
+tableSchema.get.find(f => conf.resolver(f.name, col.name)).getOrElse {
   throw QueryCompilationErrors.columnNotFoundInSchemaError(col, 
tableSchema)
 }
-dialect.quoteIdentifier(normalizedName)
-  }.mkString(",")
+  }
 }
-val placeholders = rddSchema.fields.map(_ => "?").mkString(",")
-s"INSERT INTO $table ($columns) VALUES ($placeholders)"
+dialect.insertIntoTable(table, columns)
   }
 
   /**
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
index 22625523a04..37c378c294c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
@@ -193,6 +193,24 @@ abstract class JdbcDialect extends Serializable with 
Logging {
 statement.executeUpdate(s"CREATE TABLE $tableName ($strSchema) 
$createTableOptions")
   }
 
+  /**
+   * Returns an Insert SQL statement template for inserting a row into the 
target table via JDBC
+   * conn. Use "?" as placeholder for each value to be inserted.
+   * E.g. `INSERT INTO t ("name", "age", "gender") VALUES (?, ?, ?)`
+   *
+   * @param table The name of the table.
+   * @param fields The fields of the row that will be inserted.
+   * @return The SQL query to use for insert data into table.
+   */
+  @Since("4.0.0")
+  def insertIntoTable(
+  table: String,
+  fields: Array[StructField]): String = {
+val placeholders = fields.map(_ =>

[spark] branch branch-3.4 updated: [SPARK-45433][SQL][3.4] Fix CSV/JSON schema inference when timestamps do not match specified timestampFormat

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

maxgekk 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 f985d716e164 [SPARK-45433][SQL][3.4] Fix CSV/JSON schema inference 
when timestamps do not match specified timestampFormat
f985d716e164 is described below

commit f985d716e164885575ec7f36a7782694411da024
Author: Jia Fan 
AuthorDate: Thu Oct 12 17:09:48 2023 +0500

[SPARK-45433][SQL][3.4] Fix CSV/JSON schema inference when timestamps do 
not match specified timestampFormat

### What changes were proposed in this pull request?
This is a backport PR of #43243. Fix the bug of schema inference when 
timestamps do not match specified timestampFormat. Please check #43243 for 
detail.

### Why are the changes needed?
Fix schema inference bug on 3.4.

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

### How was this patch tested?
add new test.

### Was this patch authored or co-authored using generative AI tooling?

Closes #43343 from Hisoka-X/backport-SPARK-45433-inference-schema.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala |  8 ++--
 .../org/apache/spark/sql/catalyst/json/JsonInferSchema.scala   |  7 +--
 .../apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala| 10 ++
 .../apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala  |  8 
 4 files changed, 29 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
index 51586a0065e9..dd8ac3985f19 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.util.{DateFormatter, 
TimestampFormatter}
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
 import org.apache.spark.sql.types._
 
 class CSVInferSchema(val options: CSVOptions) extends Serializable {
@@ -202,8 +203,11 @@ class CSVInferSchema(val options: CSVOptions) extends 
Serializable {
 // We can only parse the value as TimestampNTZType if it does not have 
zone-offset or
 // time-zone component and can be parsed with the timestamp formatter.
 // Otherwise, it is likely to be a timestamp with timezone.
-if (timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
-  SQLConf.get.timestampType
+val timestampType = SQLConf.get.timestampType
+if ((SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY ||
+timestampType == TimestampNTZType) &&
+timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
+  timestampType
 } else {
   tryParseTimestamp(field)
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
index 5385afe8c935..7e4767750fd3 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils
 
@@ -148,11 +149,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) 
extends Serializable {
   val bigDecimal = decimalParser(field)
 DecimalType(bigDecimal.precision, bigDecimal.scale)
 }
+val timestampType = SQLConf.get.timestampType
 if (options.prefersDecimal && decimalTry.isDefined) {
   decimalTry.get
-} else if (options.inferTimestamp &&
+} else if (options.inferTimestamp && 
(SQLConf.get.legacyTimeParserPolicy ==
+  LegacyBehaviorPolicy.LEGACY || timestampType == TimestampNTZType) &&
 timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
-  SQLConf.get.timestampType
+  timestampType
 } else if (options.inferTimestamp &&
 timestampFormatter.parseOptional(field).isDefined) {
   

[spark] branch branch-3.5 updated: [SPARK-45433][SQL] Fix CSV/JSON schema inference when timestamps do not match specified timestampFormat

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 7e3ddc1e582 [SPARK-45433][SQL] Fix CSV/JSON schema inference when 
timestamps do not match specified timestampFormat
7e3ddc1e582 is described below

commit 7e3ddc1e582a6e4fa96bab608c4c2bbc2c93b449
Author: Jia Fan 
AuthorDate: Wed Oct 11 19:33:23 2023 +0300

[SPARK-45433][SQL] Fix CSV/JSON schema inference when timestamps do not 
match specified timestampFormat

### What changes were proposed in this pull request?
This PR fix CSV/JSON schema inference when timestamps do not match 
specified timestampFormat will report error.
```scala
//eg
val csv = spark.read.option("timestampFormat", "-MM-dd'T'HH:mm:ss")
  .option("inferSchema", true).csv(Seq("2884-06-24T02:45:51.138").toDS())
csv.show()
//error
Caused by: java.time.format.DateTimeParseException: Text 
'2884-06-24T02:45:51.138' could not be parsed, unparsed text found at index 19
```
This bug only happend when partition had one row. The data type should be 
`StringType` not `TimestampType` because the value not match `timestampFormat`.

Use csv as eg, in `CSVInferSchema::tryParseTimestampNTZ`, first, use 
`timestampNTZFormatter.parseWithoutTimeZoneOptional` to inferring return 
`TimestampType`, if same partition had another row, it will use 
`tryParseTimestamp` to parse row with user defined `timestampFormat`, then 
found it can't be convert to timestamp with `timestampFormat`. Finally return 
`StringType`. But when only one row, we use 
`timestampNTZFormatter.parseWithoutTimeZoneOptional` to parse normally 
timestamp not r [...]

### Why are the changes needed?
Fix schema inference bug.

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

### How was this patch tested?
add new test.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #43243 from 
Hisoka-X/SPARK-45433-inference-mismatch-timestamp-one-row.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
(cherry picked from commit eae5c0e1efce83c2bb08754784db070be285285a)
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala |  9 ++---
 .../org/apache/spark/sql/catalyst/json/JsonInferSchema.scala   |  8 +---
 .../apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala| 10 ++
 .../apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala  |  8 
 4 files changed, 29 insertions(+), 6 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
index 51586a0065e..ec01b56f9eb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExprUtils
 import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter}
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT
 import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
 import org.apache.spark.sql.types._
 
 class CSVInferSchema(val options: CSVOptions) extends Serializable {
@@ -202,8 +202,11 @@ class CSVInferSchema(val options: CSVOptions) extends 
Serializable {
 // We can only parse the value as TimestampNTZType if it does not have 
zone-offset or
 // time-zone component and can be parsed with the timestamp formatter.
 // Otherwise, it is likely to be a timestamp with timezone.
-if (timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
-  SQLConf.get.timestampType
+val timestampType = SQLConf.get.timestampType
+if ((SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY ||
+timestampType == TimestampNTZType) &&
+timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
+  timestampType
 } else {
   tryParseTimestamp(field)
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
index 5385afe8c93..4123c5290b6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
@@ -32,7 +32,7 @@ import 
org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil
 imp

[spark] branch master updated: [SPARK-45433][SQL] Fix CSV/JSON schema inference when timestamps do not match specified timestampFormat

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

maxgekk 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 eae5c0e1efc [SPARK-45433][SQL] Fix CSV/JSON schema inference when 
timestamps do not match specified timestampFormat
eae5c0e1efc is described below

commit eae5c0e1efce83c2bb08754784db070be285285a
Author: Jia Fan 
AuthorDate: Wed Oct 11 19:33:23 2023 +0300

[SPARK-45433][SQL] Fix CSV/JSON schema inference when timestamps do not 
match specified timestampFormat

### What changes were proposed in this pull request?
This PR fix CSV/JSON schema inference when timestamps do not match 
specified timestampFormat will report error.
```scala
//eg
val csv = spark.read.option("timestampFormat", "-MM-dd'T'HH:mm:ss")
  .option("inferSchema", true).csv(Seq("2884-06-24T02:45:51.138").toDS())
csv.show()
//error
Caused by: java.time.format.DateTimeParseException: Text 
'2884-06-24T02:45:51.138' could not be parsed, unparsed text found at index 19
```
This bug only happend when partition had one row. The data type should be 
`StringType` not `TimestampType` because the value not match `timestampFormat`.

Use csv as eg, in `CSVInferSchema::tryParseTimestampNTZ`, first, use 
`timestampNTZFormatter.parseWithoutTimeZoneOptional` to inferring return 
`TimestampType`, if same partition had another row, it will use 
`tryParseTimestamp` to parse row with user defined `timestampFormat`, then 
found it can't be convert to timestamp with `timestampFormat`. Finally return 
`StringType`. But when only one row, we use 
`timestampNTZFormatter.parseWithoutTimeZoneOptional` to parse normally 
timestamp not r [...]

### Why are the changes needed?
Fix schema inference bug.

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

### How was this patch tested?
add new test.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #43243 from 
Hisoka-X/SPARK-45433-inference-mismatch-timestamp-one-row.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala |  9 ++---
 .../org/apache/spark/sql/catalyst/json/JsonInferSchema.scala   |  8 +---
 .../apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala| 10 ++
 .../apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala  |  8 
 4 files changed, 29 insertions(+), 6 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
index 51586a0065e..ec01b56f9eb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExprUtils
 import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter}
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT
 import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
 import org.apache.spark.sql.types._
 
 class CSVInferSchema(val options: CSVOptions) extends Serializable {
@@ -202,8 +202,11 @@ class CSVInferSchema(val options: CSVOptions) extends 
Serializable {
 // We can only parse the value as TimestampNTZType if it does not have 
zone-offset or
 // time-zone component and can be parsed with the timestamp formatter.
 // Otherwise, it is likely to be a timestamp with timezone.
-if (timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
-  SQLConf.get.timestampType
+val timestampType = SQLConf.get.timestampType
+if ((SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY ||
+timestampType == TimestampNTZType) &&
+timestampNTZFormatter.parseWithoutTimeZoneOptional(field, 
false).isDefined) {
+  timestampType
 } else {
   tryParseTimestamp(field)
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
index 5385afe8c93..4123c5290b6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
@@ -32,7 +32,7 @@ import 
org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.catalyst

[spark] branch master updated: [SPARK-42881][SQL] Codegen Support for get_json_object

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

maxgekk 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 c2525308330 [SPARK-42881][SQL] Codegen Support for get_json_object
c2525308330 is described below

commit c252530833097759b1f943ff89b05f22025f0dd0
Author: panbingkun 
AuthorDate: Wed Oct 11 17:42:48 2023 +0300

[SPARK-42881][SQL] Codegen Support for get_json_object

### What changes were proposed in this pull request?
The PR adds Codegen Support for get_json_object.

### Why are the changes needed?
Improve codegen coverage and performance.
Github benchmark 
data(https://github.com/panbingkun/spark/actions/runs/4497396473/jobs/7912952710):
https://user-images.githubusercontent.com/15246973/227117793-bab38c42-dcc1-46de-a689-25a87b8f3561.png;>

Local benchmark data:
https://user-images.githubusercontent.com/15246973/227098745-9b360e60-fe84-4419-8b7d-073a0530816a.png;>

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

### How was this patch tested?
Add new UT.
Pass GA.

Closes #40506 from panbingkun/json_code_gen.

Authored-by: panbingkun 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/expressions/jsonExpressions.scala | 121 +---
 sql/core/benchmarks/JsonBenchmark-results.txt  | 127 +++--
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  |  28 +
 .../execution/datasources/json/JsonBenchmark.scala |  15 ++-
 4 files changed, 208 insertions(+), 83 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
index e7df542ddab..04bc457b66a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
@@ -28,7 +28,8 @@ import com.fasterxml.jackson.core.json.JsonReadFeature
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, CodegenFallback, ExprCode}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
 import org.apache.spark.sql.catalyst.json._
 import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, 
TreePattern}
 import org.apache.spark.sql.catalyst.util._
@@ -125,13 +126,7 @@ private[this] object SharedFactory {
   group = "json_funcs",
   since = "1.5.0")
 case class GetJsonObject(json: Expression, path: Expression)
-  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
-
-  import com.fasterxml.jackson.core.JsonToken._
-
-  import PathInstruction._
-  import SharedFactory._
-  import WriteStyle._
+  extends BinaryExpression with ExpectsInputTypes {
 
   override def left: Expression = json
   override def right: Expression = path
@@ -140,18 +135,114 @@ case class GetJsonObject(json: Expression, path: 
Expression)
   override def nullable: Boolean = true
   override def prettyName: String = "get_json_object"
 
-  @transient private lazy val parsedPath = 
parsePath(path.eval().asInstanceOf[UTF8String])
+  @transient
+  private lazy val evaluator = if (path.foldable) {
+new GetJsonObjectEvaluator(path.eval().asInstanceOf[UTF8String])
+  } else {
+new GetJsonObjectEvaluator()
+  }
 
   override def eval(input: InternalRow): Any = {
-val jsonStr = json.eval(input).asInstanceOf[UTF8String]
+evaluator.setJson(json.eval(input).asInstanceOf[UTF8String])
+if (!path.foldable) {
+  evaluator.setPath(path.eval(input).asInstanceOf[UTF8String])
+}
+evaluator.evaluate()
+  }
+
+  protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+val evaluatorClass = classOf[GetJsonObjectEvaluator].getName
+val initEvaluator = path.foldable match {
+  case true if path.eval() != null =>
+val cachedPath = path.eval().asInstanceOf[UTF8String]
+val refCachedPath = ctx.addReferenceObj("cachedPath", cachedPath)
+s"new $evaluatorClass($refCachedPath)"
+  case _ => s"new $evaluatorClass()"
+}
+val evaluator = ctx.addMutableState(evaluatorClass, "evaluator",
+  v => s"""$v = $initEvaluator;""", forceInline = true)
+
+val jsonEval = json.genCode(ctx)
+val pathEval = path.genCode(ctx)
+
+val setJson =
+  s"""
+ |if 

[spark] branch master updated (e1a7b84f47b -> ae112e4279f)

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

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


from e1a7b84f47b [SPARK-45397][ML][CONNECT] Add array assembler feature 
transformer
 add ae112e4279f [SPARK-45116][SQL] Add some comment for param of 
JdbcDialect `createTable`

No new revisions were added by this update.

Summary of changes:
 .../main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala| 10 ++
 1 file changed, 6 insertions(+), 4 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-45213][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2151

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

maxgekk 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 6373f19f537 [SPARK-45213][SQL] Assign name to the error 
_LEGACY_ERROR_TEMP_2151
6373f19f537 is described below

commit 6373f19f537f69c6460b2e4097f19903c01a608f
Author: dengziming 
AuthorDate: Tue Oct 10 15:36:18 2023 +0300

[SPARK-45213][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2151

### What changes were proposed in this pull request?
Assign the name `EXPRESSION_DECODING_FAILED` to the legacy error class 
`_LEGACY_ERROR_TEMP_2151`.

### Why are the changes needed?
To assign proper name as a part of activity in SPARK-37935.

### Does this PR introduce _any_ user-facing change?
Yes, the error message will include the error class name

### How was this patch tested?
An existing unit test to produce the error from user code.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43029 from dengziming/SPARK-45213.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 common/utils/src/main/resources/error/error-classes.json  | 11 +--
 docs/sql-error-conditions.md  |  6 ++
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala|  3 +--
 .../spark/sql/catalyst/encoders/EncoderResolutionSuite.scala  |  2 +-
 .../src/test/scala/org/apache/spark/sql/DatasetSuite.scala|  5 ++---
 5 files changed, 15 insertions(+), 12 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 690d1ae1a14..1239793b3f9 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -921,6 +921,11 @@
   }
 }
   },
+  "EXPRESSION_DECODING_FAILED" : {
+"message" : [
+  "Failed to decode a row to a value of the expressions: ."
+]
+  },
   "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : {
 "message" : [
   "Column expression  cannot be sorted because its type  
is not orderable."
@@ -5524,12 +5529,6 @@
   "Due to Scala's limited support of tuple, tuple with more than 22 
elements are not supported."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2151" : {
-"message" : [
-  "Error while decoding: ",
-  "."
-]
-  },
   "_LEGACY_ERROR_TEMP_2152" : {
 "message" : [
   "Error while encoding: ",
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index fda10eceb97..b4ee7358b52 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -551,6 +551,12 @@ The table `` does not support ``.
 
 For more details see 
[EXPECT_VIEW_NOT_TABLE](sql-error-conditions-expect-view-not-table-error-class.html)
 
+### EXPRESSION_DECODING_FAILED
+
+SQLSTATE: none assigned
+
+Failed to decode a row to a value of the expressions: ``.
+
 ### EXPRESSION_TYPE_IS_NOT_ORDERABLE
 
 SQLSTATE: none assigned
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 bd4d7a3be7f..5396ae5ff70 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
@@ -1342,9 +1342,8 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
 
   def expressionDecodingError(e: Exception, expressions: Seq[Expression]): 
SparkRuntimeException = {
 new SparkRuntimeException(
-  errorClass = "_LEGACY_ERROR_TEMP_2151",
+  errorClass = "EXPRESSION_DECODING_FAILED",
   messageParameters = Map(
-"e" -> e.toString(),
 "expressions" -> expressions.map(
   _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")),
   cause = e)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
index f4106e65e7c..7f54987ee7e 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
@@ -172,7 +172,7 @@ class EncoderResolutionSuite extends PlanTest {
 val e = intercept[RuntimeException] {
   fromRow(InternalRow(new GenericArrayData(Array(1, null
 }
-assert(e.getMessage.contains("Null value appeared 

[spark] branch master updated (f378b506bf1 -> 76230765674)

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

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


from f378b506bf1 [SPARK-45470][SQL] Avoid paste string value of hive orc 
compression kind
 add 76230765674 [SPARK-45458][SQL] Convert IllegalArgumentException to 
SparkIllegalArgumentException in bitwiseExpressions

No new revisions were added by this update.

Summary of changes:
 .../src/main/resources/error/error-classes.json|  5 +++
 ...nditions-invalid-parameter-value-error-class.md |  4 +++
 .../catalyst/expressions/bitwiseExpressions.scala  | 15 
 .../spark/sql/errors/QueryExecutionErrors.scala| 10 ++
 .../expressions/BitwiseExpressionsSuite.scala  | 42 --
 .../resources/sql-tests/results/bitwise.sql.out| 26 +++---
 6 files changed, 79 insertions(+), 23 deletions(-)


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



[spark] branch branch-3.5 updated: [SPARK-45383][SQL] Fix error message for time travel with non-existing table

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 8bf5a5bca3f [SPARK-45383][SQL] Fix error message for time travel with 
non-existing table
8bf5a5bca3f is described below

commit 8bf5a5bca3f9f7db78182d14e56476d384f442fa
Author: Wenchen Fan 
AuthorDate: Mon Oct 9 22:15:45 2023 +0300

[SPARK-45383][SQL] Fix error message for time travel with non-existing table

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

Fixes a small bug to report `TABLE_OR_VIEW_NOT_FOUND` error correctly for 
time travel. It was missed before because `RelationTimeTravel` is a leaf node 
but it may contain `UnresolvedRelation`.

### Why are the changes needed?

bug fix

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

Yes, the error message becomes reasonable

### How was this patch tested?

new tests

### Was this patch authored or co-authored using generative AI tooling?

no

Closes #43298 from cloud-fan/time-travel.

Authored-by: Wenchen Fan 
Signed-off-by: Max Gekk 
(cherry picked from commit ced321c8b5a32c69dfb2841d4bec8a03f21b8038)
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala|  4 
 .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 11 +++
 2 files changed, 15 insertions(+)

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 511f3622e7e..533ea8a2b79 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
@@ -365,6 +365,9 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog with QueryErrorsB
 })
 
 operator match {
+  case RelationTimeTravel(u: UnresolvedRelation, _, _) =>
+u.tableNotFound(u.multipartIdentifier)
+
   case etw: EventTimeWatermark =>
 etw.eventTime.dataType match {
   case s: StructType
@@ -377,6 +380,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog with QueryErrorsB
 "eventName" -> toSQLId(etw.eventTime.name),
 "eventType" -> toSQLType(etw.eventTime.dataType)))
 }
+
   case f: Filter if f.condition.dataType != BooleanType =>
 f.failAnalysis(
   errorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index 06f5600e0d1..7745e9c0a4e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -3014,6 +3014,17 @@ class DataSourceV2SQLSuiteV1Filter
 sqlState = None,
 parameters = Map("relationId" -> "`x`"))
 
+  checkError(
+exception = intercept[AnalysisException] {
+  sql("SELECT * FROM non_exist VERSION AS OF 1")
+},
+errorClass = "TABLE_OR_VIEW_NOT_FOUND",
+parameters = Map("relationName" -> "`non_exist`"),
+context = ExpectedContext(
+  fragment = "non_exist",
+  start = 14,
+  stop = 22))
+
   val subquery1 = "SELECT 1 FROM non_exist"
   checkError(
 exception = intercept[AnalysisException] {


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



[spark] branch master updated: [SPARK-45383][SQL] Fix error message for time travel with non-existing table

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

maxgekk 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 ced321c8b5a [SPARK-45383][SQL] Fix error message for time travel with 
non-existing table
ced321c8b5a is described below

commit ced321c8b5a32c69dfb2841d4bec8a03f21b8038
Author: Wenchen Fan 
AuthorDate: Mon Oct 9 22:15:45 2023 +0300

[SPARK-45383][SQL] Fix error message for time travel with non-existing table

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

Fixes a small bug to report `TABLE_OR_VIEW_NOT_FOUND` error correctly for 
time travel. It was missed before because `RelationTimeTravel` is a leaf node 
but it may contain `UnresolvedRelation`.

### Why are the changes needed?

bug fix

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

Yes, the error message becomes reasonable

### How was this patch tested?

new tests

### Was this patch authored or co-authored using generative AI tooling?

no

Closes #43298 from cloud-fan/time-travel.

Authored-by: Wenchen Fan 
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala|  4 
 .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 11 +++
 2 files changed, 15 insertions(+)

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 e140625f47a..611dd7b3009 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
@@ -384,6 +384,9 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog with QueryErrorsB
 })
 
 operator match {
+  case RelationTimeTravel(u: UnresolvedRelation, _, _) =>
+u.tableNotFound(u.multipartIdentifier)
+
   case etw: EventTimeWatermark =>
 etw.eventTime.dataType match {
   case s: StructType
@@ -396,6 +399,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog with QueryErrorsB
 "eventName" -> toSQLId(etw.eventTime.name),
 "eventType" -> toSQLType(etw.eventTime.dataType)))
 }
+
   case f: Filter if f.condition.dataType != BooleanType =>
 f.failAnalysis(
   errorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index ae639b272a2..047bc8de739 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -3014,6 +3014,17 @@ class DataSourceV2SQLSuiteV1Filter
 sqlState = None,
 parameters = Map("relationId" -> "`x`"))
 
+  checkError(
+exception = intercept[AnalysisException] {
+  sql("SELECT * FROM non_exist VERSION AS OF 1")
+},
+errorClass = "TABLE_OR_VIEW_NOT_FOUND",
+parameters = Map("relationName" -> "`non_exist`"),
+context = ExpectedContext(
+  fragment = "non_exist",
+  start = 14,
+  stop = 22))
+
   val subquery1 = "SELECT 1 FROM non_exist"
   checkError(
 exception = intercept[AnalysisException] {


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



[spark] branch branch-3.5 updated: [SPARK-45459][SQL][TESTS][DOCS] Remove the last 2 extra spaces in the automatically generated `sql-error-conditions.md` file

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 4841a404be3 [SPARK-45459][SQL][TESTS][DOCS] Remove the last 2 extra 
spaces in the automatically generated `sql-error-conditions.md` file
4841a404be3 is described below

commit 4841a404be3c37fc16031a0119b321eefcb2faab
Author: panbingkun 
AuthorDate: Mon Oct 9 12:32:14 2023 +0300

[SPARK-45459][SQL][TESTS][DOCS] Remove the last 2 extra spaces in the 
automatically generated `sql-error-conditions.md` file

### What changes were proposed in this pull request?
The pr aims to remove the last 2 extra spaces in the automatically 
generated `sql-error-conditions.md` file.

### Why are the changes needed?
- When I am work on another PR, I use the following command:
```
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt \
"core/testOnly *SparkThrowableSuite -- -t \"Error classes match 
with document\""
```
  I found that in the automatically generated `sql-error-conditions.md` 
file, there are 2 extra spaces added at the end,
Obviously, this is not what we expected, otherwise we would need to 
manually remove it, which is not in line with automation.

- The git tells us this difference, as follows:
https://github.com/apache/spark/assets/15246973/a68b657f-3a00-4405-9623-1f7ab9d44d82;>

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

### How was this patch tested?
- Pass GA.
- Manually test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43274 from panbingkun/SPARK-45459.

Authored-by: panbingkun 
Signed-off-by: Max Gekk 
(cherry picked from commit af800b505956ff26e03c5fc56b6cb4ac5c0efe2f)
Signed-off-by: Max Gekk 
---
 core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala 
b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala
index 0249cde5488..299bcea3f9e 100644
--- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala
@@ -253,8 +253,7 @@ class SparkThrowableSuite extends SparkFunSuite {
  |
  |Also see [SQLSTATE Codes](sql-error-conditions-sqlstates.html).
  |
- |$sqlErrorParentDocContent
- |""".stripMargin
+ |$sqlErrorParentDocContent""".stripMargin
 
 errors.filter(_._2.subClass.isDefined).foreach(error => {
   val name = error._1
@@ -316,7 +315,7 @@ class SparkThrowableSuite extends SparkFunSuite {
 }
 FileUtils.writeStringToFile(
   parentDocPath.toFile,
-  sqlErrorParentDoc + lineSeparator,
+  sqlErrorParentDoc,
   StandardCharsets.UTF_8)
   }
 } else {


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



[spark] branch master updated (4493b431192 -> af800b50595)

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

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


from 4493b431192 [SPARK-45424][SQL] Fix TimestampFormatter return optional 
parse results when only prefix match
 add af800b50595 [SPARK-45459][SQL][TESTS][DOCS] Remove the last 2 extra 
spaces in the automatically generated `sql-error-conditions.md` file

No new revisions were added by this update.

Summary of changes:
 core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)


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



[spark] branch branch-3.5 updated: [SPARK-45424][SQL] Fix TimestampFormatter return optional parse results when only prefix match

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 5f8ae9a3dbd [SPARK-45424][SQL] Fix TimestampFormatter return optional 
parse results when only prefix match
5f8ae9a3dbd is described below

commit 5f8ae9a3dbd2c7624bffd588483c9916c302c081
Author: Jia Fan 
AuthorDate: Mon Oct 9 12:30:20 2023 +0300

[SPARK-45424][SQL] Fix TimestampFormatter return optional parse results 
when only prefix match

### What changes were proposed in this pull request?
When use custom pattern to parse timestamp, if there have matched prefix, 
not matched all. The `Iso8601TimestampFormatter::parseOptional` and 
`Iso8601TimestampFormatter::parseWithoutTimeZoneOptional` should not return not 
empty result.
eg: pattern = `-MM-dd HH:mm:ss`, value = `-12-31 23:59:59.999`. If 
fact, `-MM-dd HH:mm:ss` can parse `-12-31 23:59:59`  normally, but 
value have suffix `.999`. so we can't return not empty result.
This bug will affect inference the schema in CSV/JSON.

### Why are the changes needed?
Fix inference the schema bug.

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

### How was this patch tested?
add new test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43245 from Hisoka-X/SPARK-45424-inference-schema-unresolved.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
(cherry picked from commit 4493b431192fcdbab1379b7ffb89eea0cdaa19f1)
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/util/TimestampFormatter.scala| 10 ++
 .../spark/sql/catalyst/util/TimestampFormatterSuite.scala  | 10 ++
 2 files changed, 16 insertions(+), 4 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
index 8a288d0e9f3..55eee41c14c 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
@@ -167,8 +167,9 @@ class Iso8601TimestampFormatter(
 
   override def parseOptional(s: String): Option[Long] = {
 try {
-  val parsed = formatter.parseUnresolved(s, new ParsePosition(0))
-  if (parsed != null) {
+  val parsePosition = new ParsePosition(0)
+  val parsed = formatter.parseUnresolved(s, parsePosition)
+  if (parsed != null && s.length == parsePosition.getIndex) {
 Some(extractMicros(parsed))
   } else {
 None
@@ -196,8 +197,9 @@ class Iso8601TimestampFormatter(
 
   override def parseWithoutTimeZoneOptional(s: String, allowTimeZone: 
Boolean): Option[Long] = {
 try {
-  val parsed = formatter.parseUnresolved(s, new ParsePosition(0))
-  if (parsed != null) {
+  val parsePosition = new ParsePosition(0)
+  val parsed = formatter.parseUnresolved(s, parsePosition)
+  if (parsed != null && s.length == parsePosition.getIndex) {
 Some(extractMicrosNTZ(s, parsed, allowTimeZone))
   } else {
 None
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
index eb173bc7f8c..2134a0d6ecd 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
@@ -507,4 +507,14 @@ class TimestampFormatterSuite extends 
DatetimeFormatterSuite {
 assert(simpleFormatter.parseOptional("abc").isEmpty)
 
   }
+
+  test("SPARK-45424: do not return optional parse results when only prefix 
match") {
+val formatter = new Iso8601TimestampFormatter(
+  "-MM-dd HH:mm:ss",
+  locale = DateFormatter.defaultLocale,
+  legacyFormat = LegacyDateFormats.SIMPLE_DATE_FORMAT,
+  isParsing = true, zoneId = DateTimeTestUtils.LA)
+assert(formatter.parseOptional("-12-31 23:59:59.999").isEmpty)
+assert(formatter.parseWithoutTimeZoneOptional("-12-31 23:59:59.999", 
true).isEmpty)
+  }
 }


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



[spark] branch master updated: [SPARK-45424][SQL] Fix TimestampFormatter return optional parse results when only prefix match

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

maxgekk 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 4493b431192 [SPARK-45424][SQL] Fix TimestampFormatter return optional 
parse results when only prefix match
4493b431192 is described below

commit 4493b431192fcdbab1379b7ffb89eea0cdaa19f1
Author: Jia Fan 
AuthorDate: Mon Oct 9 12:30:20 2023 +0300

[SPARK-45424][SQL] Fix TimestampFormatter return optional parse results 
when only prefix match

### What changes were proposed in this pull request?
When use custom pattern to parse timestamp, if there have matched prefix, 
not matched all. The `Iso8601TimestampFormatter::parseOptional` and 
`Iso8601TimestampFormatter::parseWithoutTimeZoneOptional` should not return not 
empty result.
eg: pattern = `-MM-dd HH:mm:ss`, value = `-12-31 23:59:59.999`. If 
fact, `-MM-dd HH:mm:ss` can parse `-12-31 23:59:59`  normally, but 
value have suffix `.999`. so we can't return not empty result.
This bug will affect inference the schema in CSV/JSON.

### Why are the changes needed?
Fix inference the schema bug.

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

### How was this patch tested?
add new test.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43245 from Hisoka-X/SPARK-45424-inference-schema-unresolved.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/util/TimestampFormatter.scala| 10 ++
 .../spark/sql/catalyst/util/TimestampFormatterSuite.scala  | 10 ++
 2 files changed, 16 insertions(+), 4 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
index 8a288d0e9f3..55eee41c14c 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
@@ -167,8 +167,9 @@ class Iso8601TimestampFormatter(
 
   override def parseOptional(s: String): Option[Long] = {
 try {
-  val parsed = formatter.parseUnresolved(s, new ParsePosition(0))
-  if (parsed != null) {
+  val parsePosition = new ParsePosition(0)
+  val parsed = formatter.parseUnresolved(s, parsePosition)
+  if (parsed != null && s.length == parsePosition.getIndex) {
 Some(extractMicros(parsed))
   } else {
 None
@@ -196,8 +197,9 @@ class Iso8601TimestampFormatter(
 
   override def parseWithoutTimeZoneOptional(s: String, allowTimeZone: 
Boolean): Option[Long] = {
 try {
-  val parsed = formatter.parseUnresolved(s, new ParsePosition(0))
-  if (parsed != null) {
+  val parsePosition = new ParsePosition(0)
+  val parsed = formatter.parseUnresolved(s, parsePosition)
+  if (parsed != null && s.length == parsePosition.getIndex) {
 Some(extractMicrosNTZ(s, parsed, allowTimeZone))
   } else {
 None
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
index ecd849dd3af..d2fc89a034f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala
@@ -491,4 +491,14 @@ class TimestampFormatterSuite extends 
DatetimeFormatterSuite {
 assert(simpleFormatter.parseOptional("abc").isEmpty)
 
   }
+
+  test("SPARK-45424: do not return optional parse results when only prefix 
match") {
+val formatter = new Iso8601TimestampFormatter(
+  "-MM-dd HH:mm:ss",
+  locale = DateFormatter.defaultLocale,
+  legacyFormat = LegacyDateFormats.SIMPLE_DATE_FORMAT,
+  isParsing = true, zoneId = DateTimeTestUtils.LA)
+assert(formatter.parseOptional("-12-31 23:59:59.999").isEmpty)
+assert(formatter.parseWithoutTimeZoneOptional("-12-31 23:59:59.999", 
true).isEmpty)
+  }
 }


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



[spark] branch master updated: [SPARK-45262][SQL][TESTS][DOCS] Improve examples for regexp parameters

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

maxgekk 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 e3b1bb117fe9 [SPARK-45262][SQL][TESTS][DOCS] Improve examples for 
regexp parameters
e3b1bb117fe9 is described below

commit e3b1bb117fe9bf0b17321e6359b7aa90f70a24b5
Author: Max Gekk 
AuthorDate: Fri Oct 6 22:34:40 2023 +0300

[SPARK-45262][SQL][TESTS][DOCS] Improve examples for regexp parameters

### What changes were proposed in this pull request?
In the PR, I propose to add a few more examples for `LIKE`, `ILIKE`, 
`RLIKE`, `regexp_instr()`, `regexp_extract_all()` that highlight correctness of 
current description and test a couple more of corner cases.

### Why are the changes needed?
The description of `LIKE` says:
```
... in order to match "\abc", the pattern should be "\\abc"
```
but in Spark SQL shell:
```sql
spark-sql (default)> SELECT c FROM t;
\abc
spark-sql (default)> SELECT c LIKE "\\abc" FROM t;
[INVALID_FORMAT.ESC_IN_THE_MIDDLE] The format is invalid: '\\abc'. The 
escape character is not allowed to precede 'a'.
spark-sql (default)> SELECT c LIKE "abc" FROM t;
true
```
So, the description might confuse users since the pattern must contain 4 
slashes when the pattern is a regular SQL string.

New example shows that the pattern "\\abc" is correct if we take into 
account the string as a raw string:
```sql
spark-sql (default)> SELECT c LIKE R"\\abc" FROM t;
true
```

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

### How was this patch tested?
By running new and modified tests:
```
$ build/sbt "test:testOnly *.StringFunctionsSuite"
$ build/sbt "sql/test:testOnly 
org.apache.spark.sql.expressions.ExpressionInfoSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43037 from MaxGekk/fix-like-doc.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/expressions/regexpExpressions.scala   | 18 --
 .../resources/sql-functions/sql-expression-schema.md   |  2 +-
 .../org/apache/spark/sql/StringFunctionsSuite.scala|  5 +
 3 files changed, 22 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index 87ea8b5a102a..b33de303b5d5 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -108,13 +108,15 @@ abstract class StringRegexExpression extends 
BinaryExpression
 Examples:
   > SELECT _FUNC_('Spark', '_park');
   true
+  > SELECT '\\abc' AS S, S _FUNC_ r'\\abc', S _FUNC_ 'abc';
+  \abc truetrue
   > SET spark.sql.parser.escapedStringLiterals=true;
   spark.sql.parser.escapedStringLiterals   true
   > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%';
   true
   > SET spark.sql.parser.escapedStringLiterals=false;
   spark.sql.parser.escapedStringLiterals   false
-  > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '\%SystemDrive\%Users%';
+  > SELECT '%SystemDrive%\\Users\\John' _FUNC_ r'%SystemDrive%\\Users%';
   true
   > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SystemDrive/%//Users%' 
ESCAPE '/';
   true
@@ -226,13 +228,15 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
 Examples:
   > SELECT _FUNC_('Spark', '_Park');
   true
+  > SELECT '\\abc' AS S, S _FUNC_ r'\\abc', S _FUNC_ 'abc';
+  \abc truetrue
   > SET spark.sql.parser.escapedStringLiterals=true;
   spark.sql.parser.escapedStringLiterals   true
   > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\users%';
   true
   > SET spark.sql.parser.escapedStringLiterals=false;
   spark.sql.parser.escapedStringLiterals   false
-  > SELECT '%SystemDrive%\\USERS\\John' _FUNC_ '\%SystemDrive\%Users%';
+  > SELECT '%SystemDrive%\\USERS\\John' _FUNC_ r'%SystemDrive%\\Users%';
   true
   > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SYSTEMDrive/%//Users%' 
ESCAPE '/';
   true
@@ -446,6 +450,8 @@ case class NotLikeAny(child: Expression, patterns: 
Seq[UTF8String]) extends Like
   spark.sql.parser.escapedStringLiterals   false
   > SELECT _FUNC_('%SystemDrive%\\Users\\John', 
'%SystemDrive%Users.*');
   true
+  > SELECT _FUNC_('%Syst

[spark] branch master updated: [SPARK-45400][SQL][DOCS] Refer to the unescaping rules from expression descriptions

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

maxgekk 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 c0d9ca3be14c [SPARK-45400][SQL][DOCS] Refer to the unescaping rules 
from expression descriptions
c0d9ca3be14c is described below

commit c0d9ca3be14cb0ec8d8f9920d3ecc4aac3cf5adc
Author: Max Gekk 
AuthorDate: Thu Oct 5 22:22:29 2023 +0300

[SPARK-45400][SQL][DOCS] Refer to the unescaping rules from expression 
descriptions

### What changes were proposed in this pull request?
In the PR, I propose to refer to the unescaping rules added by 
https://github.com/apache/spark/pull/43152 from expression descriptions like in 
`Like`, see
https://github.com/apache/spark/assets/1580697/6a332b50-f2c8-4549-848a-61519c9f964e;>

### Why are the changes needed?
To improve user experience w/ Spark SQL.

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

### How was this patch tested?
Manually generated docs and checked by eyes.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43203 from MaxGekk/link-to-escape-doc.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 docs/sql-ref-literals.md   |  2 +
 .../catalyst/expressions/regexpExpressions.scala   | 70 ++
 2 files changed, 47 insertions(+), 25 deletions(-)

diff --git a/docs/sql-ref-literals.md b/docs/sql-ref-literals.md
index e9447af71c54..2a02a22bd6f0 100644
--- a/docs/sql-ref-literals.md
+++ b/docs/sql-ref-literals.md
@@ -62,6 +62,8 @@ The following escape sequences are recognized in regular 
string literals (withou
 - `\_` -> `\_`;
 - `\` -> ``, skip the slash and leave the character as 
is.
 
+The unescaping rules above can be turned off by setting the SQL config 
`spark.sql.parser.escapedStringLiterals` to `true`.
+
  Examples
 
 ```sql
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index 69d90296d7ff..87ea8b5a102a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -77,7 +77,7 @@ abstract class StringRegexExpression extends BinaryExpression
   }
 }
 
-// scalastyle:off line.contains.tab
+// scalastyle:off line.contains.tab line.size.limit
 /**
  * Simple RegEx pattern matching function
  */
@@ -92,11 +92,14 @@ abstract class StringRegexExpression extends 
BinaryExpression
   _ matches any one character in the input (similar to . in posix 
regular expressions)\
   % matches zero or more characters in the input (similar to .* in 
posix regular
   expressions)
-  Since Spark 2.0, string literals are unescaped in our SQL parser. 
For example, in order
-  to match "\abc", the pattern should be "\\abc".
+  Since Spark 2.0, string literals are unescaped in our SQL parser, 
see the unescaping
+  rules at https://spark.apache.org/docs/latest/sql-ref-literals.html#string-literal;>String
 Literal.
+  For example, in order to match "\abc", the pattern should be 
"\\abc".
   When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, 
it falls back
   to Spark 1.6 behavior regarding string literal parsing. For example, 
if the config is
-  enabled, the pattern to match "\abc" should be "\abc".
+  enabled, the pattern to match "\abc" should be "\abc".
+  It's recommended to use a raw string literal (with the `r` prefix) 
to avoid escaping
+  special characters in the pattern string if exists.
   * escape - an character added since Spark 3.0. The default escape 
character is the '\'.
   If an escape character precedes a special symbol or another escape 
character, the
   following character is matched literally. It is invalid to escape 
any other character.
@@ -121,7 +124,7 @@ abstract class StringRegexExpression extends 
BinaryExpression
   """,
   since = "1.0.0",
   group = "predicate_funcs")
-// scalastyle:on line.contains.tab
+// scalastyle:on line.contains.tab line.size.limit
 case class Like(left: Expression, right: Expression, escapeChar: Char)
   extends StringRegexExpression {
 
@@ -207,11 +210,14 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
   _ matches any one character in the input (similar to . in posix 
regular expressions)
   % matches zero or more characters in the input (similar to .* in 
posix regu

[spark] branch master updated: [SPARK-45398][SQL] Append `ESCAPE` in `sql()` of the `Like` expression

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

maxgekk 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 cc4ecb5104e [SPARK-45398][SQL] Append `ESCAPE` in `sql()` of the 
`Like` expression
cc4ecb5104e is described below

commit cc4ecb5104e37d5e530d44b41fc1d8f8116e37d8
Author: Max Gekk 
AuthorDate: Wed Oct 4 11:35:05 2023 +0300

[SPARK-45398][SQL] Append `ESCAPE` in `sql()` of the `Like` expression

### What changes were proposed in this pull request?
In the PR, I propose to fix the `sql()` method of the `Like` expression, 
and append the `ESCAPE` clause when the `escapeChar` is not the default one 
`\\`.

### Why are the changes needed?
1. To be consistent to the `toString()` method
2. To distinguish column names when the escape argument is set. Before the 
changes, columns might conflict like the example below, and that could confuse 
users:
```sql
spark-sql (default)> create temp view tbl as (SELECT 'a|_' like 'a||_' 
escape '|', 'a|_' like 'a||_' escape 'a');
[COLUMN_ALREADY_EXISTS] The column `a|_ like a||_` already exists. Consider 
to choose another name or rename the existing column.
```

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

### How was this patch tested?
Manually checking the column name by:
```sql
spark-sql (default)> create temp view tbl as (SELECT 'a|_' like 'a||_' 
escape '|', 'a|_' like 'a||_' escape 'a');
Time taken: 0.531 seconds
spark-sql (default)> describe extended tbl;
a|_ LIKE a||_ ESCAPE '|'boolean
a|_ LIKE a||_ ESCAPE 'a'boolean
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43196 from MaxGekk/fix-like-sql.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../explain-results/function_like_with_escape.explain |  2 +-
 .../spark/sql/catalyst/expressions/regexpExpressions.scala| 11 +++
 2 files changed, 8 insertions(+), 5 deletions(-)

diff --git 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_like_with_escape.explain
 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_like_with_escape.explain
index 471a3a4bd52..1a15a27d97e 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_like_with_escape.explain
+++ 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_like_with_escape.explain
@@ -1,2 +1,2 @@
-Project [g#0 LIKE g#0 ESCAPE '/' AS g LIKE g#0]
+Project [g#0 LIKE g#0 ESCAPE '/' AS g LIKE g ESCAPE '/'#0]
 +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index 5ebfdd919b8..69d90296d7f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -133,12 +133,15 @@ case class Like(left: Expression, right: Expression, 
escapeChar: Char)
 
   final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
 
-  override def toString: String = escapeChar match {
-case '\\' => s"$left LIKE $right"
-case c => s"$left LIKE $right ESCAPE '$c'"
+  override def toString: String = {
+val escapeSuffix = if (escapeChar == '\\') "" else s" ESCAPE '$escapeChar'"
+s"$left ${prettyName.toUpperCase(Locale.ROOT)} $right" + escapeSuffix
   }
 
-  override def sql: String = s"${left.sql} 
${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}"
+  override def sql: String = {
+val escapeSuffix = if (escapeChar == '\\') "" else s" ESCAPE 
${Literal(escapeChar).sql}"
+s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}" + 
escapeSuffix
+  }
 
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
 val patternClass = classOf[Pattern].getName


-
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] Remove duplicate cases of escaping characters in string literals

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

maxgekk 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 ee21b12c395 [MINOR][SQL] Remove duplicate cases of escaping characters 
in string literals
ee21b12c395 is described below

commit ee21b12c395ac184c8ddc2f74b66f6e6285de5fa
Author: Max Gekk 
AuthorDate: Thu Sep 28 21:18:40 2023 +0300

[MINOR][SQL] Remove duplicate cases of escaping characters in string 
literals

### What changes were proposed in this pull request?
In the PR, I propose to remove some cases in `appendEscapedChar()` because 
they fall to the default case.

The following tests check the cases:
- 
https://github.com/apache/spark/blob/187e9a851758c0e9cec11edab2bc07d6f4404001/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala#L97-L98
- 
https://github.com/apache/spark/blob/187e9a851758c0e9cec11edab2bc07d6f4404001/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala#L104

### Why are the changes needed?
To improve code maintainability.

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

### How was this patch tested?
By running the affected test suite:
```
$ build/sbt "test:testOnly *.ParserUtilsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43170 from MaxGekk/cleanup-escaping.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala| 3 ---
 1 file changed, 3 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
index c318f208255..a4ce5fb1203 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkParserUtils.scala
@@ -38,14 +38,11 @@ trait SparkParserUtils {
 def appendEscapedChar(n: Char): Unit = {
   n match {
 case '0' => sb.append('\u')
-case '\'' => sb.append('\'')
-case '"' => sb.append('\"')
 case 'b' => sb.append('\b')
 case 'n' => sb.append('\n')
 case 'r' => sb.append('\r')
 case 't' => sb.append('\t')
 case 'Z' => sb.append('\u001A')
-case '\\' => sb.append('\\')
 // The following 2 lines are exactly what MySQL does TODO: why do we 
do this?
 case '%' => sb.append("\\%")
 case '_' => sb.append("\\_")


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



[spark] branch master updated: [SPARK-45340][SQL] Remove the SQL config `spark.sql.hive.verifyPartitionPath`

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

maxgekk 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 eff46ea77e9 [SPARK-45340][SQL] Remove the SQL config 
`spark.sql.hive.verifyPartitionPath`
eff46ea77e9 is described below

commit eff46ea77e9bebef3076277bef1e086833dd
Author: Max Gekk 
AuthorDate: Wed Sep 27 08:28:45 2023 +0300

[SPARK-45340][SQL] Remove the SQL config 
`spark.sql.hive.verifyPartitionPath`

### What changes were proposed in this pull request?
In the PR, I propose to remove already deprecated SQL config 
`spark.sql.hive.verifyPartitionPath`, and the code under the config. The config 
has been deprecated since Spark 3.0.

### Why are the changes needed?
To improve code maintainability by remove unused code.

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

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "test:testOnly *SQLConfSuite"
$ build/sbt "test:testOnly *QueryPartitionSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

    Closes #43130 from MaxGekk/remove-verifyPartitionPath.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/internal/SQLConf.scala| 17 ++---
 .../apache/spark/sql/internal/SQLConfSuite.scala   |  4 +--
 .../org/apache/spark/sql/hive/TableReader.scala| 41 +-
 .../spark/sql/hive/QueryPartitionSuite.scala   | 12 ++-
 4 files changed, 8 insertions(+), 66 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 43eb0756d8d..aeef531dbcd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.Path
 import org.apache.spark.{ErrorMessageFormat, SparkConf, SparkContext, 
TaskContext}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
-import org.apache.spark.internal.config.{IGNORE_MISSING_FILES => 
SPARK_IGNORE_MISSING_FILES}
 import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver}
@@ -1261,14 +1260,6 @@ object SQLConf {
 .booleanConf
 .createWithDefault(false)
 
-  val HIVE_VERIFY_PARTITION_PATH = 
buildConf("spark.sql.hive.verifyPartitionPath")
-.doc("When true, check all the partition paths under the table\'s root 
directory " +
- "when reading data stored in HDFS. This configuration will be 
deprecated in the future " +
- s"releases and replaced by ${SPARK_IGNORE_MISSING_FILES.key}.")
-.version("1.4.0")
-.booleanConf
-.createWithDefault(false)
-
   val HIVE_METASTORE_DROP_PARTITION_BY_NAME =
 buildConf("spark.sql.hive.dropPartitionByName.enabled")
   .doc("When true, Spark will get partition name rather than partition 
object " +
@@ -4472,8 +4463,6 @@ object SQLConf {
 PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key, "2.4",
 "The config allows to switch to the behaviour before Spark 2.4 " +
   "and will be removed in the future releases."),
-  DeprecatedConfig(HIVE_VERIFY_PARTITION_PATH.key, "3.0",
-s"This config is replaced by '${SPARK_IGNORE_MISSING_FILES.key}'."),
   DeprecatedConfig(ARROW_EXECUTION_ENABLED.key, "3.0",
 s"Use '${ARROW_PYSPARK_EXECUTION_ENABLED.key}' instead of it."),
   DeprecatedConfig(ARROW_FALLBACK_ENABLED.key, "3.0",
@@ -4552,7 +4541,9 @@ object SQLConf {
   RemovedConfig("spark.sql.ansi.strictIndexOperator", "3.4.0", "true",
 "This was an internal configuration. It is not needed anymore since 
Spark SQL always " +
   "returns null when getting a map value with a non-existing key. See 
SPARK-40066 " +
-  "for more details.")
+  "for more details."),
+  RemovedConfig("spark.sql.hive.verifyPartitionPath", "4.0.0", "false",
+s"This config was replaced by '${IGNORE_MISSING_FILES.key}'.")
 )
 
 Map(configs.map { cfg => cfg.key -> cfg } : _*)
@@ -4766,8 +4757,6 @@ class SQLConf extends Serializable with Logging with 
SqlApiConf {
 
   def isOrcSchemaMergingEnabled: Boolean = getConf(ORC_SCHEMA_MERGING_ENABLED)
 
-  def verifyPartitionPath: Boolean = getConf(HIVE_VERIFY_PARTITION_PATH)
-
   def metastoreDrop

[spark] branch master updated: [SPARK-45316][CORE][SQL] Add new parameters `ignoreCorruptFiles`/`ignoreMissingFiles` to `HadoopRDD` and `NewHadoopRDD`

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

maxgekk 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 60d02b444e2 [SPARK-45316][CORE][SQL] Add new parameters 
`ignoreCorruptFiles`/`ignoreMissingFiles` to `HadoopRDD` and `NewHadoopRDD`
60d02b444e2 is described below

commit 60d02b444e2225b3afbe4955dabbea505e9f769c
Author: Max Gekk 
AuthorDate: Tue Sep 26 17:33:07 2023 +0300

[SPARK-45316][CORE][SQL] Add new parameters 
`ignoreCorruptFiles`/`ignoreMissingFiles` to `HadoopRDD` and `NewHadoopRDD`

### What changes were proposed in this pull request?
In the PR, I propose to add new parameters 
`ignoreCorruptFiles`/`ignoreMissingFiles` to `HadoopRDD` and `NewHadoopRDD`, 
and set it to the current value of:
- `spark.files.ignoreCorruptFiles`/`ignoreMissingFiles` in Spark `core`,
- `spark.sql.files.ignoreCorruptFiles`/`ignoreMissingFiles` when the rdds 
created in Spark SQL.

### Why are the changes needed?
1. To make `HadoopRDD` and `NewHadoopRDD` consistent to other RDDs like 
`FileScanRDD` created by Spark SQL that take into account the SQL configs 
`spark.sql.files.ignoreCorruptFiles`/`ignoreMissingFiles`.
2. To improve user experience with Spark SQL, so, users can control 
ignoring of missing files without re-creating spark context.

### Does this PR introduce _any_ user-facing change?
Yes, `HadoopRDD`/`NewHadoopRDD` invoked by SQL code such hive table scans 
respect the SQL configs 
`spark.sql.files.ignoreCorruptFiles`/`ignoreMissingFiles` and don't respect the 
core configs `spark.files.ignoreCorruptFiles`/`ignoreMissingFiles`.

### How was this patch tested?
By running the affected tests:
```
$ build/sbt "test:testOnly *QueryPartitionSuite"
$ build/sbt "test:testOnly *FileSuite"
$ build/sbt "test:testOnly *FileBasedDataSourceSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43097 from MaxGekk/dynamic-ignoreMissingFiles.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/rdd/HadoopRDD.scala | 31 ++
 .../scala/org/apache/spark/rdd/NewHadoopRDD.scala  | 27 +++
 docs/sql-migration-guide.md|  1 +
 .../org/apache/spark/sql/hive/TableReader.scala|  9 ---
 .../spark/sql/hive/QueryPartitionSuite.scala   |  6 ++---
 5 files changed, 58 insertions(+), 16 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index cad107256c5..0b5f6a3d716 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -89,6 +89,8 @@ private[spark] class HadoopPartition(rddId: Int, override val 
index: Int, s: Inp
  * @param keyClass Class of the key associated with the inputFormatClass.
  * @param valueClass Class of the value associated with the inputFormatClass.
  * @param minPartitions Minimum number of HadoopRDD partitions (Hadoop Splits) 
to generate.
+ * @param ignoreCorruptFiles Whether to ignore corrupt files.
+ * @param ignoreMissingFiles Whether to ignore missing files.
  *
  * @note Instantiating this class directly is not recommended, please use
  * `org.apache.spark.SparkContext.hadoopRDD()`
@@ -101,13 +103,36 @@ class HadoopRDD[K, V](
 inputFormatClass: Class[_ <: InputFormat[K, V]],
 keyClass: Class[K],
 valueClass: Class[V],
-minPartitions: Int)
+minPartitions: Int,
+ignoreCorruptFiles: Boolean,
+ignoreMissingFiles: Boolean)
   extends RDD[(K, V)](sc, Nil) with Logging {
 
   if (initLocalJobConfFuncOpt.isDefined) {
 sparkContext.clean(initLocalJobConfFuncOpt.get)
   }
 
+  def this(
+  sc: SparkContext,
+  broadcastedConf: Broadcast[SerializableConfiguration],
+  initLocalJobConfFuncOpt: Option[JobConf => Unit],
+  inputFormatClass: Class[_ <: InputFormat[K, V]],
+  keyClass: Class[K],
+  valueClass: Class[V],
+  minPartitions: Int) = {
+this(
+  sc,
+  broadcastedConf,
+  initLocalJobConfFuncOpt,
+  inputFormatClass,
+  keyClass,
+  valueClass,
+  minPartitions,
+  ignoreCorruptFiles = sc.conf.get(IGNORE_CORRUPT_FILES),
+  ignoreMissingFiles = sc.conf.get(IGNORE_MISSING_FILES)
+)
+  }
+
   def this(
   sc: SparkContext,
   conf: JobConf,
@@ -135,10 +160,6 @@ class HadoopRDD[K, V](
 
   private val shouldCloneJobConf = 
sparkContext.conf.getBoolean("spark.hadoop.cloneConf", false)
 
-  private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES)
-
-  private val ignoreMissingFiles = sparkContext.conf.get(IGNORE_MISSING_FILES)
-
   private val ignoreEmptySplits = 
spa

[spark] branch master updated: [SPARK-43254][SQL] Assign a name to the error _LEGACY_ERROR_TEMP_2018

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

maxgekk 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 8b967e191b7 [SPARK-43254][SQL] Assign a name to the error 
_LEGACY_ERROR_TEMP_2018
8b967e191b7 is described below

commit 8b967e191b755d7f2830c15d382c83ce7aeb69c1
Author: dengziming 
AuthorDate: Thu Sep 21 10:22:37 2023 +0300

[SPARK-43254][SQL] Assign a name to the error _LEGACY_ERROR_TEMP_2018

### What changes were proposed in this pull request?
Assign the name `CLASS_UNSUPPORTED_BY_MAP_OBJECTS` to the legacy error 
class `_LEGACY_ERROR_TEMP_2018`.

### Why are the changes needed?
To assign proper name as a part of activity in SPARK-37935

### Does this PR introduce _any_ user-facing change?
Yes, the error message will include the error class name

### How was this patch tested?
Add a unit test to produce the error from user code.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #42939 from dengziming/SPARK-43254.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 10 +++---
 docs/sql-error-conditions.md   |  6 
 .../sql/catalyst/encoders/ExpressionEncoder.scala  |  2 +-
 .../spark/sql/errors/QueryExecutionErrors.scala|  2 +-
 .../expressions/ObjectExpressionsSuite.scala   | 11 +++---
 .../scala/org/apache/spark/sql/DatasetSuite.scala  | 40 --
 6 files changed, 57 insertions(+), 14 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index d92ccfce5c5..8942d3755e9 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -344,6 +344,11 @@
 ],
 "sqlState" : "22003"
   },
+  "CLASS_UNSUPPORTED_BY_MAP_OBJECTS" : {
+"message" : [
+  "`MapObjects` does not support the class  as resulting collection."
+]
+  },
   "CODEC_NOT_AVAILABLE" : {
 "message" : [
   "The codec  is not available. Consider to set the config 
 to ."
@@ -4944,11 +4949,6 @@
   "not resolved."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2018" : {
-"message" : [
-  "class `` is not supported by `MapObjects` as resulting collection."
-]
-  },
   "_LEGACY_ERROR_TEMP_2020" : {
 "message" : [
   "Couldn't find a valid constructor on ."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 1df00f72bc9..f6f94efc2b0 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -297,6 +297,12 @@ The value `` of the type `` cannot be 
cast to ``
 
 Fail to assign a value of `` type to the `` type 
column or variable `` due to an overflow. Use `try_cast` on the 
input value to tolerate overflow and return NULL instead.
 
+### CLASS_UNSUPPORTED_BY_MAP_OBJECTS
+
+SQLSTATE: none assigned
+
+`MapObjects` does not support the class `` as resulting collection.
+
 ### CODEC_NOT_AVAILABLE
 
 SQLSTATE: none assigned
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index ff72b5a0d96..74d7a5e7a67 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -170,7 +170,7 @@ object ExpressionEncoder {
* Function that deserializes an [[InternalRow]] into an object of type `T`. 
This class is not
* thread-safe.
*/
-  class Deserializer[T](private val expressions: Seq[Expression])
+  class Deserializer[T](val expressions: Seq[Expression])
 extends (InternalRow => T) with Serializable {
 @transient
 private[this] var constructProjection: Projection = _
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 e14fef1fad7..84472490128 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
@@ -422,7 +422,7 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
 
   def classUnsupportedByMapObjectsError(cls: Class[_]): SparkRuntimeException 
= {
 new SparkRuntimeException(
-  errorClass = "_LEGACY_ERROR_TEMP_2018",
+  errorClass = "CLASS_UNSUPPORTED_BY_MAP_OBJECTS",

[spark] branch master updated: [SPARK-45235][CONNECT][PYTHON] Support map and array parameters by `sql()`

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

maxgekk 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 a2bab5efc5b [SPARK-45235][CONNECT][PYTHON] Support map and array 
parameters by `sql()`
a2bab5efc5b is described below

commit a2bab5efc5b5f0e841e9b34ccbfd2cb99af5923e
Author: Max Gekk 
AuthorDate: Thu Sep 21 09:05:30 2023 +0300

[SPARK-45235][CONNECT][PYTHON] Support map and array parameters by `sql()`

### What changes were proposed in this pull request?
In the PR, I propose to change the Python connect client to support 
`Column` as a parameter of `sql()`.

### Why are the changes needed?
To achieve feature parity w/ regular PySpark which supports map and arrays 
as parameters of `sql()`, see https://github.com/apache/spark/pull/42996.

### Does this PR introduce _any_ user-facing change?
No. It fixes a bug.

### How was this patch tested?
By running the modified tests:
```
$ python/run-tests --parallelism=1 --testnames 
'pyspark.sql.tests.connect.test_connect_basic 
SparkConnectBasicTests.test_sql_with_named_args'
$ python/run-tests --parallelism=1 --testnames 
'pyspark.sql.tests.connect.test_connect_basic 
SparkConnectBasicTests.test_sql_with_pos_args'
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #43014 from MaxGekk/map-sql-parameterized-python-connect-2.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 python/pyspark/sql/connect/plan.py | 22 ++
 python/pyspark/sql/connect/session.py  |  2 +-
 .../sql/tests/connect/test_connect_basic.py| 12 
 3 files changed, 19 insertions(+), 17 deletions(-)

diff --git a/python/pyspark/sql/connect/plan.py 
b/python/pyspark/sql/connect/plan.py
index 3e8db2aae09..d069081e1af 100644
--- a/python/pyspark/sql/connect/plan.py
+++ b/python/pyspark/sql/connect/plan.py
@@ -1049,6 +1049,12 @@ class SQL(LogicalPlan):
 self._query = query
 self._args = args
 
+def _to_expr(self, session: "SparkConnectClient", v: Any) -> 
proto.Expression:
+if isinstance(v, Column):
+return v.to_plan(session)
+else:
+return LiteralExpression._from_value(v).to_plan(session)
+
 def plan(self, session: "SparkConnectClient") -> proto.Relation:
 plan = self._create_proto_relation()
 plan.sql.query = self._query
@@ -1056,14 +1062,10 @@ class SQL(LogicalPlan):
 if self._args is not None and len(self._args) > 0:
 if isinstance(self._args, Dict):
 for k, v in self._args.items():
-plan.sql.args[k].CopyFrom(
-
LiteralExpression._from_value(v).to_plan(session).literal
-)
+
plan.sql.named_arguments[k].CopyFrom(self._to_expr(session, v))
 else:
 for v in self._args:
-plan.sql.pos_args.append(
-
LiteralExpression._from_value(v).to_plan(session).literal
-)
+plan.sql.pos_arguments.append(self._to_expr(session, v))
 
 return plan
 
@@ -1073,14 +1075,10 @@ class SQL(LogicalPlan):
 if self._args is not None and len(self._args) > 0:
 if isinstance(self._args, Dict):
 for k, v in self._args.items():
-cmd.sql_command.args[k].CopyFrom(
-
LiteralExpression._from_value(v).to_plan(session).literal
-)
+
cmd.sql_command.named_arguments[k].CopyFrom(self._to_expr(session, v))
 else:
 for v in self._args:
-cmd.sql_command.pos_args.append(
-
LiteralExpression._from_value(v).to_plan(session).literal
-)
+
cmd.sql_command.pos_arguments.append(self._to_expr(session, v))
 
 return cmd
 
diff --git a/python/pyspark/sql/connect/session.py 
b/python/pyspark/sql/connect/session.py
index 7582fe86ff2..e5d1d95a699 100644
--- a/python/pyspark/sql/connect/session.py
+++ b/python/pyspark/sql/connect/session.py
@@ -557,7 +557,7 @@ class SparkSession:
 if "sql_command_result" in properties:
 return 
DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
 else:
-return DataFrame.withPlan(SQL(sqlQuery, args), self)
+return DataFrame.withPlan(cmd, self)
 
 sql.__doc__ = PySparkSession.sql.__doc__
 
diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py 
b/python/pyspark/sql/tests/connect/test_connect_basic.py
index 2b979570618..c5a127136d6 100644
--- a/python/pyspark/sql/tests/connect/test_con

[spark] branch master updated: [SPARK-45224][PYTHON] Add examples w/ map and array as parameters of `sql()`

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

maxgekk 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 c89221b02bb3 [SPARK-45224][PYTHON] Add examples w/ map and array as 
parameters of `sql()`
c89221b02bb3 is described below

commit c89221b02bb3000f707a31322e6d40b561e527bd
Author: Max Gekk 
AuthorDate: Wed Sep 20 11:09:01 2023 +0300

[SPARK-45224][PYTHON] Add examples w/ map and array as parameters of `sql()`

### What changes were proposed in this pull request?
In the PR, I propose to add a few more examples for the `sql()` method in 
PySpark API with array and map parameters.

### Why are the changes needed?
To inform users about recent changes introduced by #42752 and #42470, and 
check the changes work actually.

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

### How was this patch tested?
By running new examples:
```
$ python/run-tests --parallelism=1 --testnames 'pyspark.sql.session 
SparkSession.sql'
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42996 from MaxGekk/map-sql-parameterized-python-connect.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 python/pyspark/sql/session.py | 30 +-
 1 file changed, 17 insertions(+), 13 deletions(-)

diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index dc4f8f321a59..de2e8d0cda2a 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -1599,23 +1599,27 @@ class SparkSession(SparkConversionMixin):
 
 And substitude named parameters with the `:` prefix by SQL literals.
 
->>> spark.sql("SELECT * FROM {df} WHERE {df[B]} > :minB", {"minB" : 
5}, df=mydf).show()
-+---+---+
-|  A|  B|
-+---+---+
-|  3|  6|
-+---+---+
+>>> from pyspark.sql.functions import create_map
+>>> spark.sql(
+...   "SELECT *, element_at(:m, 'a') AS C FROM {df} WHERE {df[B]} > 
:minB",
+...   {"minB" : 5, "m" : create_map(lit('a'), lit(1))}, df=mydf).show()
++---+---+---+
+|  A|  B|  C|
++---+---+---+
+|  3|  6|  1|
++---+---+---+
 
 Or positional parameters marked by `?` in the SQL query by SQL 
literals.
 
+>>> from pyspark.sql.functions import array
 >>> spark.sql(
-...   "SELECT * FROM {df} WHERE {df[B]} > ? and ? < {df[A]}",
-...   args=[5, 2], df=mydf).show()
-+---+---+
-|  A|  B|
-+---+---+
-|  3|  6|
-+---+---+
+...   "SELECT *, element_at(?, 1) AS C FROM {df} WHERE {df[B]} > ? and 
? < {df[A]}",
+...   args=[array(lit(1), lit(2), lit(3)), 5, 2], df=mydf).show()
++---+---+---+
+|  A|  B|  C|
++---+---+---+
+|  3|  6|  1|
++---+---+---+
 """
 
 formatter = SQLStringFormatter(self)


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



[spark] branch master updated: [SPARK-45188][SQL][DOCS] Update error messages related to parameterized `sql()`

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

maxgekk 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 981312284f0 [SPARK-45188][SQL][DOCS] Update error messages related to 
parameterized `sql()`
981312284f0 is described below

commit 981312284f0776ca847c8d21411f74a72c639b22
Author: Max Gekk 
AuthorDate: Tue Sep 19 00:22:43 2023 +0300

[SPARK-45188][SQL][DOCS] Update error messages related to parameterized 
`sql()`

### What changes were proposed in this pull request?
In the PR, I propose to update some error formats and comments regarding 
`sql()` parameters - maps, arrays and struct might be used as `sql()` 
parameters. New behaviour has been added by 
https://github.com/apache/spark/pull/42752.

### Why are the changes needed?
To inform users about recent changes introduced by SPARK-45033.

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

### How was this patch tested?
By running the affected test suite:
```
$ build/sbt "core/testOnly *SparkThrowableSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42957 from MaxGekk/clean-ClientE2ETestSuite.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json  |  4 ++--
 .../scala/org/apache/spark/sql/SparkSession.scala| 11 +++
 docs/sql-error-conditions.md |  4 ++--
 python/pyspark/pandas/sql_formatter.py   |  3 ++-
 python/pyspark/sql/session.py|  3 ++-
 .../spark/sql/catalyst/analysis/parameters.scala | 14 +-
 .../scala/org/apache/spark/sql/SparkSession.scala| 20 ++--
 7 files changed, 34 insertions(+), 25 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 4740ed72f89..186e7b4640d 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1892,7 +1892,7 @@
   },
   "INVALID_SQL_ARG" : {
 "message" : [
-  "The argument  of `sql()` is invalid. Consider to replace it by a 
SQL literal."
+  "The argument  of `sql()` is invalid. Consider to replace it 
either by a SQL literal or by collection constructor functions such as `map()`, 
`array()`, `struct()`."
 ]
   },
   "INVALID_SQL_SYNTAX" : {
@@ -2768,7 +2768,7 @@
   },
   "UNBOUND_SQL_PARAMETER" : {
 "message" : [
-  "Found the unbound parameter: . Please, fix `args` and provide a 
mapping of the parameter to a SQL literal."
+  "Found the unbound parameter: . Please, fix `args` and provide a 
mapping of the parameter to either a SQL literal or collection constructor 
functions such as `map()`, `array()`, `struct()`."
 ],
 "sqlState" : "42P02"
   },
diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala
index 8788e34893e..5aa8c5a2bd5 100644
--- 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -235,8 +235,9 @@ class SparkSession private[sql] (
*   An array of Java/Scala objects that can be converted to SQL literal 
expressions. See https://spark.apache.org/docs/latest/sql-ref-datatypes.html;> 
Supported Data
*   Types for supported value types in Scala/Java. For example: 1, 
"Steven",
-   *   LocalDate.of(2023, 4, 2). A value can be also a `Column` of literal 
expression, in that
-   *   case it is taken as is.
+   *   LocalDate.of(2023, 4, 2). A value can be also a `Column` of a literal 
or collection
+   *   constructor functions such as `map()`, `array()`, `struct()`, in that 
case it is taken as
+   *   is.
*
* @since 3.5.0
*/
@@ -272,7 +273,8 @@ class SparkSession private[sql] (
*   expressions. See https://spark.apache.org/docs/latest/sql-ref-datatypes.html;>
*   Supported Data Types for supported value types in Scala/Java. For 
example, map keys:
*   "rank", "name", "birthdate"; map values: 1, "Steven", 
LocalDate.of(2023, 4, 2). Map value
-   *   can be also a `Column` of literal expression, in that case it is taken 
as is.
+   *   can be also a `Column` of a literal or collection constructor functions 
such as `map()`,
+   *   `array()`, `struct()`, in that case it is taken as is.
*
* @since 3.4.0
*/
@@ -292,7 +294,8 @@ class SparkSession private[sql] (
*   expression

[spark] branch master updated (8d363c6e2c8 -> 0dda75f824d)

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

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


from 8d363c6e2c8 [SPARK-45196][PYTHON][DOCS] Refine docstring of 
`array/array_contains/arrays_overlap`
 add 0dda75f824d [SPARK-45137][CONNECT] Support map/array parameters in 
parameterized `sql()`

No new revisions were added by this update.

Summary of changes:
 .../scala/org/apache/spark/sql/SparkSession.scala  |   6 +-
 .../org/apache/spark/sql/ClientE2ETestSuite.scala  |   7 +
 .../src/main/protobuf/spark/connect/commands.proto |  12 +-
 .../main/protobuf/spark/connect/relations.proto|  12 +-
 .../sql/connect/planner/SparkConnectPlanner.scala  |  26 +-
 python/pyspark/sql/connect/proto/commands_pb2.py   | 164 +++--
 python/pyspark/sql/connect/proto/commands_pb2.pyi  |  60 -
 python/pyspark/sql/connect/proto/relations_pb2.py  | 268 +++--
 python/pyspark/sql/connect/proto/relations_pb2.pyi |  60 -
 9 files changed, 396 insertions(+), 219 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-45034][SQL] Support deterministic mode function

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

maxgekk 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 f5365d0dc59 [SPARK-45034][SQL] Support deterministic mode function
f5365d0dc59 is described below

commit f5365d0dc590d4965a269da223dbd72fbb764595
Author: Peter Toth 
AuthorDate: Sun Sep 17 21:37:57 2023 +0300

[SPARK-45034][SQL] Support deterministic mode function

### What changes were proposed in this pull request?
This PR adds a new optional argument to the `mode` aggregate function to 
provide deterministic results. When multiple values have the same greatest 
frequency then the new boolean argument can be used to get the lowest or 
highest value instead of an arbitraty one.

### Why are the changes needed?
To make the function more user friendly.

### Does this PR introduce _any_ user-facing change?
Yes, it adds a new argument to the `mode` function.

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

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42755 from peter-toth/SPARK-45034-deterministic-mode-function.

Authored-by: Peter Toth 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/functions.scala |  14 ++-
 .../explain-results/function_mode.explain  |   2 +-
 .../query-tests/queries/function_mode.json |   4 +
 .../query-tests/queries/function_mode.proto.bin| Bin 173 -> 179 bytes
 python/pyspark/sql/connect/functions.py|   4 +-
 python/pyspark/sql/functions.py|  35 --
 .../sql/catalyst/expressions/aggregate/Mode.scala  |  76 ++--
 .../scala/org/apache/spark/sql/functions.scala |  16 ++-
 .../sql-functions/sql-expression-schema.md |   2 +-
 .../sql-tests/analyzer-results/group-by.sql.out| 120 ++-
 .../test/resources/sql-tests/inputs/group-by.sql   |  11 ++
 .../resources/sql-tests/results/group-by.sql.out   | 132 -
 .../apache/spark/sql/DatasetAggregatorSuite.scala  |  10 ++
 13 files changed, 397 insertions(+), 29 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
index b2102d4ba55..83f0ee64501 100644
--- 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
@@ -827,7 +827,19 @@ object functions {
* @group agg_funcs
* @since 3.4.0
*/
-  def mode(e: Column): Column = Column.fn("mode", e)
+  def mode(e: Column): Column = mode(e, deterministic = false)
+
+  /**
+   * Aggregate function: returns the most frequent value in a group.
+   *
+   * When multiple values have the same greatest frequency then either any of 
values is returned
+   * if deterministic is false or is not defined, or the lowest value is 
returned if deterministic
+   * is true.
+   *
+   * @group agg_funcs
+   * @since 4.0.0
+   */
+  def mode(e: Column, deterministic: Boolean): Column = Column.fn("mode", e, 
lit(deterministic))
 
   /**
* Aggregate function: returns the maximum value of the expression in a 
group.
diff --git 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_mode.explain
 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_mode.explain
index dfa2113a2c3..28bbb44b0fd 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_mode.explain
+++ 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_mode.explain
@@ -1,2 +1,2 @@
-Aggregate [mode(a#0, 0, 0) AS mode(a)#0]
+Aggregate [mode(a#0, 0, 0, false) AS mode(a, false)#0]
 +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git 
a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json
 
b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json
index 8e8183e9e08..5c26edee803 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json
+++ 
b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.json
@@ -18,6 +18,10 @@
   "unresolvedAttribute": {
 "unparsedIdentifier": "a"
   }
+}, {
+  "literal": {
+"boolean": false
+  }
 }]
   }
 }]
diff --git 
a/connector/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin
 
b/connector/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin
index dca0953a387..cc115e43172 100644
Binary files 
a/connector/connect/comm

[spark] branch branch-3.5 updated: [SPARK-45078][SQL] Fix `array_insert` ImplicitCastInputTypes not work

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 723a85eb2df [SPARK-45078][SQL] Fix `array_insert` 
ImplicitCastInputTypes not work
723a85eb2df is described below

commit 723a85eb2dffa69571cba841380eb759a9b89321
Author: Jia Fan 
AuthorDate: Sun Sep 17 11:16:24 2023 +0300

[SPARK-45078][SQL] Fix `array_insert` ImplicitCastInputTypes not work

### What changes were proposed in this pull request?
This PR fix call `array_insert` with different type between array and 
insert column, will throw exception. Sometimes it should be execute successed.
eg:
```sql
select array_insert(array(1), 2, cast(2 as tinyint))
```
The `ImplicitCastInputTypes` in `ArrayInsert` always return empty array at 
now. So that Spark can not convert `tinyint` to `int`.

### Why are the changes needed?
Fix error behavior in `array_insert`

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

### How was this patch tested?
Add new test.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #42951 from Hisoka-X/SPARK-45078_arrayinsert_type_mismatch.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
(cherry picked from commit e84c66db60c78476806161479344cd32a7606ab1)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/expressions/collectionOperations.scala | 1 -
 .../test/resources/sql-tests/analyzer-results/ansi/array.sql.out  | 7 +++
 .../src/test/resources/sql-tests/analyzer-results/array.sql.out   | 7 +++
 sql/core/src/test/resources/sql-tests/inputs/array.sql| 1 +
 sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out  | 8 
 sql/core/src/test/resources/sql-tests/results/array.sql.out   | 8 
 6 files changed, 31 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index fe9c4015c15..ade4a6c5be7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -4711,7 +4711,6 @@ case class ArrayInsert(
 }
   case (e1, e2, e3) => Seq.empty
 }
-Seq.empty
   }
 
   override def checkInputDataTypes(): TypeCheckResult = {
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out
index cd101c7a524..6fc30815793 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out
@@ -531,6 +531,13 @@ Project [array_insert(array(2, 3, cast(null as int), 4), 
-5, 1, false) AS array_
 +- OneRowRelation
 
 
+-- !query
+select array_insert(array(1), 2, cast(2 as tinyint))
+-- !query analysis
+Project [array_insert(array(1), 2, cast(cast(2 as tinyint) as int), false) AS 
array_insert(array(1), 2, CAST(2 AS TINYINT))#x]
++- OneRowRelation
+
+
 -- !query
 set spark.sql.legacy.negativeIndexInArrayInsert=true
 -- !query analysis
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out
index 8279fb3362e..e0585b77cb6 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out
@@ -531,6 +531,13 @@ Project [array_insert(array(2, 3, cast(null as int), 4), 
-5, 1, false) AS array_
 +- OneRowRelation
 
 
+-- !query
+select array_insert(array(1), 2, cast(2 as tinyint))
+-- !query analysis
+Project [array_insert(array(1), 2, cast(cast(2 as tinyint) as int), false) AS 
array_insert(array(1), 2, CAST(2 AS TINYINT))#x]
++- OneRowRelation
+
+
 -- !query
 set spark.sql.legacy.negativeIndexInArrayInsert=true
 -- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql 
b/sql/core/src/test/resources/sql-tests/inputs/array.sql
index 48edc6b4742..52a0906ea73 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/array.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql
@@ -141,6 +141,7 @@ select array_insert(array(1, 2, 3, NULL), cast(NULL as 
INT), 4);
 select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT));
 select array_insert(array(2, 3, NULL, 4), 5, 5);
 select array_insert(array(2, 3, NULL, 4), -5, 1);
+select array_insert(array(1), 2, cast(2 as tinyint));
 
 set spark.sql.legacy.negativeIndexInArrayInsert=true;
 select array_insert(array(1, 3, 4), -2, 2);
diff --

[spark] branch master updated: [SPARK-45078][SQL] Fix `array_insert` ImplicitCastInputTypes not work

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

maxgekk 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 e84c66db60c [SPARK-45078][SQL] Fix `array_insert` 
ImplicitCastInputTypes not work
e84c66db60c is described below

commit e84c66db60c78476806161479344cd32a7606ab1
Author: Jia Fan 
AuthorDate: Sun Sep 17 11:16:24 2023 +0300

[SPARK-45078][SQL] Fix `array_insert` ImplicitCastInputTypes not work

### What changes were proposed in this pull request?
This PR fix call `array_insert` with different type between array and 
insert column, will throw exception. Sometimes it should be execute successed.
eg:
```sql
select array_insert(array(1), 2, cast(2 as tinyint))
```
The `ImplicitCastInputTypes` in `ArrayInsert` always return empty array at 
now. So that Spark can not convert `tinyint` to `int`.

### Why are the changes needed?
Fix error behavior in `array_insert`

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

### How was this patch tested?
Add new test.

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #42951 from Hisoka-X/SPARK-45078_arrayinsert_type_mismatch.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/expressions/collectionOperations.scala | 1 -
 .../test/resources/sql-tests/analyzer-results/ansi/array.sql.out  | 7 +++
 .../src/test/resources/sql-tests/analyzer-results/array.sql.out   | 7 +++
 sql/core/src/test/resources/sql-tests/inputs/array.sql| 1 +
 sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out  | 8 
 sql/core/src/test/resources/sql-tests/results/array.sql.out   | 8 
 6 files changed, 31 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index 957aa1ab2d5..9c9127efb17 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -4749,7 +4749,6 @@ case class ArrayInsert(
 }
   case (e1, e2, e3) => Seq.empty
 }
-Seq.empty
   }
 
   override def checkInputDataTypes(): TypeCheckResult = {
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out
index cd101c7a524..6fc30815793 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out
@@ -531,6 +531,13 @@ Project [array_insert(array(2, 3, cast(null as int), 4), 
-5, 1, false) AS array_
 +- OneRowRelation
 
 
+-- !query
+select array_insert(array(1), 2, cast(2 as tinyint))
+-- !query analysis
+Project [array_insert(array(1), 2, cast(cast(2 as tinyint) as int), false) AS 
array_insert(array(1), 2, CAST(2 AS TINYINT))#x]
++- OneRowRelation
+
+
 -- !query
 set spark.sql.legacy.negativeIndexInArrayInsert=true
 -- !query analysis
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out
index 8279fb3362e..e0585b77cb6 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out
@@ -531,6 +531,13 @@ Project [array_insert(array(2, 3, cast(null as int), 4), 
-5, 1, false) AS array_
 +- OneRowRelation
 
 
+-- !query
+select array_insert(array(1), 2, cast(2 as tinyint))
+-- !query analysis
+Project [array_insert(array(1), 2, cast(cast(2 as tinyint) as int), false) AS 
array_insert(array(1), 2, CAST(2 AS TINYINT))#x]
++- OneRowRelation
+
+
 -- !query
 set spark.sql.legacy.negativeIndexInArrayInsert=true
 -- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql 
b/sql/core/src/test/resources/sql-tests/inputs/array.sql
index 48edc6b4742..52a0906ea73 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/array.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql
@@ -141,6 +141,7 @@ select array_insert(array(1, 2, 3, NULL), cast(NULL as 
INT), 4);
 select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT));
 select array_insert(array(2, 3, NULL, 4), 5, 5);
 select array_insert(array(2, 3, NULL, 4), -5, 1);
+select array_insert(array(1), 2, cast(2 as tinyint));
 
 set spark.sql.legacy.negativeIndexInArrayInsert=true;
 select array_insert(array(1, 3, 4), -2, 2);
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out 
b/sql/core/src/test/resources/sql-te

[spark] branch master updated (cd672b09ac6 -> 6653f94d489)

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

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


from cd672b09ac6 [SPARK-45162][SQL] Support maps and array parameters 
constructed via `call_function`
 add 6653f94d489 [SPARK-45156][SQL] Wrap `inputName` by backticks in the 
`NON_FOLDABLE_INPUT` error class

No new revisions were added by this update.

Summary of changes:
 .../expressions/CallMethodViaReflection.scala |  4 ++--
 .../aggregate/ApproxCountDistinctForIntervals.scala   |  2 +-
 .../expressions/aggregate/ApproximatePercentile.scala |  4 ++--
 .../expressions/aggregate/BloomFilterAggregate.scala  |  4 ++--
 .../expressions/aggregate/CountMinSketchAgg.scala |  6 +++---
 .../expressions/aggregate/HistogramNumeric.scala  |  2 +-
 .../catalyst/expressions/aggregate/percentiles.scala  |  2 +-
 .../sql/catalyst/expressions/csvExpressions.scala |  2 +-
 .../spark/sql/catalyst/expressions/generators.scala   |  2 +-
 .../sql/catalyst/expressions/jsonExpressions.scala|  2 +-
 .../sql/catalyst/expressions/maskExpressions.scala|  2 +-
 .../sql/catalyst/expressions/mathExpressions.scala|  2 +-
 .../sql/catalyst/expressions/regexpExpressions.scala  |  2 +-
 .../sql/catalyst/expressions/stringExpressions.scala  |  2 +-
 .../sql/catalyst/expressions/windowExpressions.scala  | 19 +++
 .../spark/sql/catalyst/expressions/xml/xpath.scala|  2 +-
 .../sql/catalyst/expressions/xmlExpressions.scala |  2 +-
 .../analysis/ExpressionTypeCheckingSuite.scala|  6 +++---
 .../expressions/CallMethodViaReflectionSuite.scala|  2 +-
 .../catalyst/expressions/RegexpExpressionsSuite.scala |  2 +-
 .../catalyst/expressions/StringExpressionsSuite.scala |  6 +++---
 .../ApproxCountDistinctForIntervalsSuite.scala|  2 +-
 .../aggregate/ApproximatePercentileSuite.scala|  4 ++--
 .../aggregate/CountMinSketchAggSuite.scala|  6 +++---
 .../expressions/aggregate/HistogramNumericSuite.scala |  2 +-
 .../expressions/aggregate/PercentileSuite.scala   |  2 +-
 .../expressions/xml/XPathExpressionSuite.scala|  2 +-
 .../analyzer-results/ansi/string-functions.sql.out|  2 +-
 .../sql-tests/analyzer-results/csv-functions.sql.out  |  2 +-
 .../sql-tests/analyzer-results/join-lateral.sql.out   |  2 +-
 .../sql-tests/analyzer-results/json-functions.sql.out |  2 +-
 .../sql-tests/analyzer-results/mask-functions.sql.out |  4 ++--
 .../sql-tests/analyzer-results/percentiles.sql.out|  2 +-
 .../analyzer-results/string-functions.sql.out |  2 +-
 .../sql-tests/results/ansi/string-functions.sql.out   |  2 +-
 .../resources/sql-tests/results/csv-functions.sql.out |  2 +-
 .../resources/sql-tests/results/join-lateral.sql.out  |  2 +-
 .../sql-tests/results/json-functions.sql.out  |  2 +-
 .../sql-tests/results/mask-functions.sql.out  |  4 ++--
 .../resources/sql-tests/results/percentiles.sql.out   |  2 +-
 .../sql-tests/results/string-functions.sql.out|  2 +-
 .../spark/sql/DataFrameWindowFunctionsSuite.scala |  2 +-
 .../org/apache/spark/sql/GeneratorFunctionSuite.scala |  2 +-
 43 files changed, 67 insertions(+), 64 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-45162][SQL] Support maps and array parameters constructed via `call_function`

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

maxgekk 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 cd672b09ac6 [SPARK-45162][SQL] Support maps and array parameters 
constructed via `call_function`
cd672b09ac6 is described below

commit cd672b09ac69724cd99dc12c9bb49dd117025be1
Author: Max Gekk 
AuthorDate: Thu Sep 14 11:31:56 2023 +0300

[SPARK-45162][SQL] Support maps and array parameters constructed via 
`call_function`

### What changes were proposed in this pull request?
In the PR, I propose to move the `BindParameters` rules from the 
`Substitution` to the `Resolution` batch, and change types of the `args` 
parameter of `NameParameterizedQuery` and `PosParameterizedQuery` to an 
`Iterable` to resolve argument expressions.

### Why are the changes needed?
After the PR, the parameterized `sql()` allows map/array/struct constructed 
by functions like `map()`, `array()`, and `struct()`, but the same functions 
invoked via `call_function` are not supported:
```scala
scala> sql("SELECT element_at(:mapParam, 'a')", Map("mapParam" -> 
call_function("map", lit("a"), lit(1
org.apache.spark.sql.catalyst.ExtendedAnalysisException: 
[UNBOUND_SQL_PARAMETER] Found the unbound parameter: mapParam. Please, fix 
`args` and provide a mapping of the parameter to a SQL literal.; line 1 pos 18;
```

### Does this PR introduce _any_ user-facing change?
No, should not since it fixes an issue. Only if user code depends on the 
error message.

After the changes:
```scala
scala> sql("SELECT element_at(:mapParam, 'a')", Map("mapParam" -> 
call_function("map", lit("a"), lit(1.show(false)
++
|element_at(map(a, 1), a)|
++
|1   |
++
```

### How was this patch tested?
By running new tests:
```
$ build/sbt "test:testOnly *ParametersSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42894 from MaxGekk/fix-parameterized-sql-unresolved.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../sql/connect/planner/SparkConnectPlanner.scala  |  2 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala |  2 +-
 .../spark/sql/catalyst/analysis/parameters.scala   | 28 +-
 .../sql/catalyst/analysis/AnalysisSuite.scala  |  4 ++--
 .../org/apache/spark/sql/ParametersSuite.scala | 19 ---
 5 files changed, 42 insertions(+), 13 deletions(-)

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 24dee006f0b..74a8ff290eb 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
@@ -269,7 +269,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) 
extends Logging {
 if (!args.isEmpty) {
   NameParameterizedQuery(parsedPlan, 
args.asScala.mapValues(transformLiteral).toMap)
 } else if (!posArgs.isEmpty) {
-  PosParameterizedQuery(parsedPlan, 
posArgs.asScala.map(transformLiteral).toArray)
+  PosParameterizedQuery(parsedPlan, 
posArgs.asScala.map(transformLiteral).toSeq)
 } else {
   parsedPlan
 }
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 e15b9730111..6491a4eea95 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
@@ -260,7 +260,6 @@ class Analyzer(override val catalogManager: CatalogManager) 
extends RuleExecutor
   // at the beginning of analysis.
   OptimizeUpdateFields,
   CTESubstitution,
-  BindParameters,
   WindowsSubstitution,
   EliminateUnions,
   SubstituteUnresolvedOrdinals),
@@ -322,6 +321,7 @@ class Analyzer(override val catalogManager: CatalogManager) 
extends RuleExecutor
   RewriteDeleteFromTable ::
   RewriteUpdateTable ::
   RewriteMergeIntoTable ::
+  BindParameters ::
   typeCoercionRules ++
   Seq(
 ResolveWithCTE,
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala
index 13404797490..a6072dcdd2c 100644

[spark] branch master updated: [SPARK-44911][SQL] Create hive table with invalid column should return error class

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

maxgekk 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 1e03db36a93 [SPARK-44911][SQL] Create hive table with invalid column 
should return error class
1e03db36a93 is described below

commit 1e03db36a939aea5b4d55059967ccde96cb29564
Author: ming95 <505306...@qq.com>
AuthorDate: Tue Sep 12 11:55:08 2023 +0300

[SPARK-44911][SQL] Create hive table with invalid column should return 
error class

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

create hive table with invalid column should return error class.

run sql
```
create table test stored as parquet as select id, date'2018-01-01' + 
make_dt_interval(0, id)  from range(0, 10)
```

before this issue , error would be :

```
org.apache.spark.sql.AnalysisException: Cannot create a table having a 
column whose name contains commas in Hive metastore. Table: 
`spark_catalog`.`default`.`test`; Column: DATE '2018-01-01' + 
make_dt_interval(0, id, 0, 0.00)
at 
org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$verifyDataSchema$4(HiveExternalCatalog.scala:175)
at 
org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$verifyDataSchema$4$adapted(HiveExternalCatalog.scala:171)
at scala.collection.Iterator.foreach(Iterator.scala:943)
```

after this issue
```
Exception in thread "main" org.apache.spark.sql.AnalysisException: 
[INVALID_HIVE_COLUMN_NAME] Cannot create the table 
`spark_catalog`.`default`.`parquet_ds1` having the column `DATE '2018-01-01' + 
make_dt_interval(0, id, 0, 0`.`00)` whose name contains invalid characters 
',' in Hive metastore.
at 
org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$verifyDataSchema$4(HiveExternalCatalog.scala:180)
at 
org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$verifyDataSchema$4$adapted(HiveExternalCatalog.scala:171)
at scala.collection.Iterator.foreach(Iterator.scala:943)
```

### Why are the changes needed?

as above

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

no

### How was this patch tested?

add UT

### Was this patch authored or co-authored using generative AI tooling?

no

Closes #42609 from ming95/SPARK-44911.

Authored-by: ming95 <505306...@qq.com>
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json|  2 +-
 docs/sql-error-conditions.md   |  2 +-
 .../spark/sql/hive/HiveExternalCatalog.scala   | 11 ---
 .../spark/sql/hive/execution/HiveDDLSuite.scala| 21 
 .../spark/sql/hive/execution/SQLQuerySuite.scala   | 23 +++---
 5 files changed, 47 insertions(+), 12 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 415bdbaf42a..4740ed72f89 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1587,7 +1587,7 @@
   },
   "INVALID_HIVE_COLUMN_NAME" : {
 "message" : [
-  "Cannot create the table  having the nested column 
 whose name contains invalid characters  in Hive 
metastore."
+  "Cannot create the table  having the column  
whose name contains invalid characters  in Hive metastore."
 ]
   },
   "INVALID_IDENTIFIER" : {
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 0d54938593c..444c2b7c0d1 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -971,7 +971,7 @@ For more details see 
[INVALID_HANDLE](sql-error-conditions-invalid-handle-error-
 
 SQLSTATE: none assigned
 
-Cannot create the table `` having the nested column `` 
whose name contains invalid characters `` in Hive metastore.
+Cannot create the table `` having the column `` whose 
name contains invalid characters `` in Hive metastore.
 
 ### INVALID_IDENTIFIER
 
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index e4325989b70..67292460bbc 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -42,7 +42,7 @@ import 
org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.types.DataTypeUtils
 import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, 
CharVarcharUtils}
-import org.apache.spark.sql.catalyst.util.TypeUti

[spark] branch master updated (6565ae47cae -> d8129f837c4)

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

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


from 6565ae47cae [SPARK-43252][SQL] Replace the error class 
`_LEGACY_ERROR_TEMP_2016` with an internal error
 add d8129f837c4 [SPARK-45085][SQL] Merge UNSUPPORTED_TEMP_VIEW_OPERATION 
into UNSUPPORTED_VIEW_OPERATION and refactor some logic

No new revisions were added by this update.

Summary of changes:
 R/pkg/tests/fulltests/test_sparkSQL.R  |  2 +-
 .../src/main/resources/error/error-classes.json| 17 --
 docs/sql-error-conditions.md   |  8 ---
 .../spark/sql/catalyst/analysis/Analyzer.scala | 19 +++
 .../sql/catalyst/analysis/v2ResolutionPlans.scala  |  4 +-
 .../spark/sql/errors/QueryCompilationErrors.scala  | 52 +++--
 .../analyzer-results/change-column.sql.out |  8 +--
 .../sql-tests/results/change-column.sql.out|  8 +--
 .../spark/sql/connector/DataSourceV2SQLSuite.scala |  4 +-
 .../apache/spark/sql/execution/SQLViewSuite.scala  | 66 +++---
 .../spark/sql/execution/SQLViewTestSuite.scala |  4 +-
 .../spark/sql/execution/command/DDLSuite.scala |  6 +-
 .../execution/command/TruncateTableSuiteBase.scala | 10 ++--
 .../execution/command/v1/ShowPartitionsSuite.scala | 10 ++--
 .../apache/spark/sql/internal/CatalogSuite.scala   |  4 +-
 15 files changed, 80 insertions(+), 142 deletions(-)


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



[spark] branch master updated (fa2bc21ba1e -> 6565ae47cae)

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

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


from fa2bc21ba1e [SPARK-45110][BUILD] Upgrade rocksdbjni to 8.5.3
 add 6565ae47cae [SPARK-43252][SQL] Replace the error class 
`_LEGACY_ERROR_TEMP_2016` with an internal error

No new revisions were added by this update.

Summary of changes:
 common/utils/src/main/resources/error/error-classes.json|  5 -
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala  |  6 ++
 .../sql/catalyst/expressions/codegen/CodeBlockSuite.scala   | 13 -
 3 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-43251][SQL] Replace the error class `_LEGACY_ERROR_TEMP_2015` with an internal error

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

maxgekk 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 c7ea3f7d53d [SPARK-43251][SQL] Replace the error class 
`_LEGACY_ERROR_TEMP_2015` with an internal error
c7ea3f7d53d is described below

commit c7ea3f7d53d5a7674f3da0db07018c1f0c43dbf6
Author: dengziming 
AuthorDate: Mon Sep 11 18:28:31 2023 +0300

[SPARK-43251][SQL] Replace the error class `_LEGACY_ERROR_TEMP_2015` with 
an internal error

### What changes were proposed in this pull request?
Replace the legacy error class `_LEGACY_ERROR_TEMP_2015` with an internal 
error as it is not triggered by the user space.

### Why are the changes needed?
As the error is not triggered by the user space, the legacy error class can 
be replaced by an internal error.

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

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

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42845 from dengziming/SPARK-43251.

Authored-by: dengziming 
Signed-off-by: Max Gekk 
---
 common/utils/src/main/resources/error/error-classes.json | 5 -
 .../scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala | 9 +++--
 2 files changed, 3 insertions(+), 11 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 2954d8b9338..282af8c199d 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -4944,11 +4944,6 @@
   "Negative values found in "
 ]
   },
-  "_LEGACY_ERROR_TEMP_2015" : {
-"message" : [
-  "Cannot generate  code for incomparable type: ."
-]
-  },
   "_LEGACY_ERROR_TEMP_2016" : {
 "message" : [
   "Can not interpolate  into code block."
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 2d655be0e70..417ba38c66f 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
@@ -405,12 +405,9 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
   }
 
   def cannotGenerateCodeForIncomparableTypeError(
-  codeType: String, dataType: DataType): SparkIllegalArgumentException = {
-new SparkIllegalArgumentException(
-  errorClass = "_LEGACY_ERROR_TEMP_2015",
-  messageParameters = Map(
-"codeType" -> codeType,
-"dataType" -> dataType.catalogString))
+  codeType: String, dataType: DataType): Throwable = {
+SparkException.internalError(
+  s"Cannot generate $codeType code for incomparable type: 
${toSQLType(dataType)}.")
   }
 
   def cannotInterpolateClassIntoCodeBlockError(arg: Any): 
SparkIllegalArgumentException = {


-
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-45100][SQL][3.3] Fix an internal error from `reflect()`on `NULL` class and method

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

maxgekk 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 a4d40e8a355 [SPARK-45100][SQL][3.3] Fix an internal error from 
`reflect()`on `NULL` class and method
a4d40e8a355 is described below

commit a4d40e8a355f451c6340dec0c90a332434433a75
Author: Max Gekk 
AuthorDate: Fri Sep 8 18:59:22 2023 +0300

[SPARK-45100][SQL][3.3] Fix an internal error from `reflect()`on `NULL` 
class and method

### What changes were proposed in this pull request?
In the PR, I propose to check that the `class` and `method` arguments are 
not a NULL in `CallMethodViaReflection`. And if they are, throw an 
`AnalysisException` with new error class `DATATYPE_MISMATCH.UNEXPECTED_NULL`.

This is a backport of https://github.com/apache/spark/pull/42849.

### Why are the changes needed?
To fix the issue demonstrated by the example:
```sql
$ spark-sql (default)> select reflect('java.util.UUID', CAST(NULL AS 
STRING));
[INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal 
error. You hit a bug in Spark or the Spark plugins you use. Please, report this 
bug to the corresponding communities or vendors, and provide the full stack 
trace.
```

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

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *.MiscFunctionsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Authored-by: Max Gekk 
(cherry picked from commit fd424caf6c46e7030ac2deb2afbe3f4a5fc1095c)

Closes #42856 from MaxGekk/fix-internal-error-in-reflect-3.3.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/expressions/CallMethodViaReflection.scala  | 2 ++
 .../src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala  | 8 
 2 files changed, 10 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
index 7cb830d1156..9764d9db7f0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
@@ -65,6 +65,8 @@ case class CallMethodViaReflection(children: Seq[Expression])
 } else if (!children.take(2).forall(e => e.dataType == StringType && 
e.foldable)) {
   // The first two arguments must be string type.
   TypeCheckFailure("first two arguments should be string literals")
+} else if (children.take(2).exists(_.eval() == null)) {
+  TypeCheckFailure("first two arguments must be non-NULL")
 } else if (!classExists) {
   TypeCheckFailure(s"class $className not found")
 } else if (children.slice(2, children.length)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
index 37ba52023dd..18262ccd407 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
@@ -34,6 +34,14 @@ class MiscFunctionsSuite extends QueryTest with 
SharedSparkSession {
 s"reflect('$className', 'method1', a, b)",
 s"java_method('$className', 'method1', a, b)"),
   Row("m1one", "m1one"))
+val e1 = intercept[AnalysisException] {
+  df.selectExpr("reflect(cast(null as string), 'fromString', a)")
+}
+assert(e1.getMessage.contains("first two arguments must be non-NULL"))
+val e2 = intercept[AnalysisException] {
+  df.selectExpr("reflect('java.util.UUID', cast(null as string), a)")
+}
+assert(e2.getMessage.contains("first two arguments must be non-NULL"))
   }
 
   test("version") {


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



[spark] branch branch-3.5 updated: [SPARK-45100][SQL] Fix an internal error from `reflect()`on `NULL` class and method

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 8f730e779ff [SPARK-45100][SQL] Fix an internal error from 
`reflect()`on `NULL` class and method
8f730e779ff is described below

commit 8f730e779ff64773beb20ad633151e866cfff7f2
Author: Max Gekk 
AuthorDate: Fri Sep 8 11:12:54 2023 +0300

[SPARK-45100][SQL] Fix an internal error from `reflect()`on `NULL` class 
and method

### What changes were proposed in this pull request?
In the PR, I propose to check that the `class` and `method` arguments are 
not a NULL in `CallMethodViaReflection`. And if they are, throw an 
`AnalysisException` with new error class `DATATYPE_MISMATCH.UNEXPECTED_NULL`.

### Why are the changes needed?
To fix the issue demonstrated by the example:
```sql
$ spark-sql (default)> select reflect('java.util.UUID', CAST(NULL AS 
STRING));
[INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal 
error. You hit a bug in Spark or the Spark plugins you use. Please, report this 
bug to the corresponding communities or vendors, and provide the full stack 
trace.
```

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

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *.MiscFunctionsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42849 from MaxGekk/fix-internal-error-in-reflect.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit fd424caf6c46e7030ac2deb2afbe3f4a5fc1095c)
Signed-off-by: Max Gekk 
---
 .../expressions/CallMethodViaReflection.scala|  8 
 .../org/apache/spark/sql/MiscFunctionsSuite.scala| 20 
 2 files changed, 28 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
index 52b057a3276..4511b5b548d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala
@@ -78,6 +78,10 @@ case class CallMethodViaReflection(children: Seq[Expression])
   "inputExpr" -> toSQLExpr(children.head)
 )
   )
+case (e, 0) if e.eval() == null =>
+  DataTypeMismatch(
+errorSubClass = "UNEXPECTED_NULL",
+messageParameters = Map("exprName" -> toSQLId("class")))
 case (e, 1) if !(e.dataType == StringType && e.foldable) =>
   DataTypeMismatch(
 errorSubClass = "NON_FOLDABLE_INPUT",
@@ -87,6 +91,10 @@ case class CallMethodViaReflection(children: Seq[Expression])
   "inputExpr" -> toSQLExpr(children(1))
 )
   )
+case (e, 1) if e.eval() == null =>
+  DataTypeMismatch(
+errorSubClass = "UNEXPECTED_NULL",
+messageParameters = Map("exprName" -> toSQLId("method")))
 case (e, idx) if idx > 1 && 
!CallMethodViaReflection.typeMapping.contains(e.dataType) =>
   DataTypeMismatch(
 errorSubClass = "UNEXPECTED_INPUT_TYPE",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
index 074556fa2f9..b890ae73fb6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
@@ -232,6 +232,26 @@ class MiscFunctionsSuite extends QueryTest with 
SharedSparkSession {
   Seq(Row("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2")))
 checkAnswer(df.select(reflect(lit("java.util.UUID"), lit("fromString"), 
col("a"))),
   Seq(Row("a5cf6c42-0c85-418f-af6c-3e4e5b1328f2")))
+
+checkError(
+  exception = intercept[AnalysisException] {
+df.selectExpr("reflect(cast(null as string), 'fromString', a)")
+  },
+  errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL",
+  parameters = Map(
+"exprName" -> "`class`",
+"sqlExpr" -> "\"reflect(CAST(NULL AS STRING), fromString, a)\""),
+  context = ExpectedContext("", "", 0, 45, "reflect(cast(null as string), 
'fromString', a)"))
+checkError(
+  exception = intercept[AnalysisException] {
+  

[spark] branch master updated (aaf413ce351 -> fd424caf6c4)

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

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


from aaf413ce351 [SPARK-44508][PYTHON][DOCS] Add user guide for Python 
user-defined table functions
 add fd424caf6c4 [SPARK-45100][SQL] Fix an internal error from 
`reflect()`on `NULL` class and method

No new revisions were added by this update.

Summary of changes:
 .../expressions/CallMethodViaReflection.scala|  8 
 .../org/apache/spark/sql/MiscFunctionsSuite.scala| 20 
 2 files changed, 28 insertions(+)


-
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-45079][SQL][3.3] Fix an internal error from `percentile_approx()` on `NULL` accuracy

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

maxgekk 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 5250ed65cf2 [SPARK-45079][SQL][3.3] Fix an internal error from 
`percentile_approx()` on `NULL` accuracy
5250ed65cf2 is described below

commit 5250ed65cf2c70e4b456c96c1006b854f56ef1f2
Author: Max Gekk 
AuthorDate: Wed Sep 6 18:56:14 2023 +0300

[SPARK-45079][SQL][3.3] Fix an internal error from `percentile_approx()` on 
`NULL` accuracy

### What changes were proposed in this pull request?
In the PR, I propose to check the `accuracy` argument is not a NULL in 
`ApproximatePercentile`. And if it is, throw an `AnalysisException` with new 
error class `DATATYPE_MISMATCH.UNEXPECTED_NULL`.

This is a backport of https://github.com/apache/spark/pull/42817.

### Why are the changes needed?
To fix the issue demonstrated by the example:
```sql
$ spark-sql (default)> SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 
NULL) FROM VALUES (0), (1), (2), (10) AS tab(col);
[INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal 
error. You hit a bug in Spark or the Spark plugins you use. Please, report this 
bug to the corresponding communities or vendors, and provide the full stack 
trace.
```

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

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *.ApproximatePercentileQuerySuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Authored-by: Max Gekk 
(cherry picked from commit 24b29adcf53616067a9fa2ca201e3f4d2f54436b)

Closes #42835 from MaxGekk/fix-internal-error-in-percentile_approx-3.3.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../expressions/aggregate/ApproximatePercentile.scala |  5 -
 .../spark/sql/ApproximatePercentileQuerySuite.scala   | 19 +++
 2 files changed, 23 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index d8eccc075a2..b816e4a9719 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -95,7 +95,8 @@ case class ApproximatePercentile(
   }
 
   // Mark as lazy so that accuracyExpression is not evaluated during tree 
transformation.
-  private lazy val accuracy: Long = 
accuracyExpression.eval().asInstanceOf[Number].longValue
+  private lazy val accuracyNum = accuracyExpression.eval().asInstanceOf[Number]
+  private lazy val accuracy: Long = accuracyNum.longValue
 
   override def inputTypes: Seq[AbstractDataType] = {
 // Support NumericType, DateType, TimestampType and TimestampNTZType since 
their internal types
@@ -120,6 +121,8 @@ case class ApproximatePercentile(
   defaultCheck
 } else if (!percentageExpression.foldable || !accuracyExpression.foldable) 
{
   TypeCheckFailure(s"The accuracy or percentage provided must be a 
constant literal")
+} else if (accuracyNum == null) {
+  TypeCheckFailure("Accuracy value must not be null")
 } else if (accuracy <= 0 || accuracy > Int.MaxValue) {
   TypeCheckFailure(s"The accuracy provided must be a literal between (0, 
${Int.MaxValue}]" +
 s" (current value = $accuracy)")
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 9237c9e9486..3fd1592a107 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -337,4 +337,23 @@ class ApproximatePercentileQuerySuite extends QueryTest 
with SharedSparkSession
   Row(Period.ofMonths(200).normalized(), null, 
Duration.ofSeconds(200L)))
 }
   }
+
+  test("SPARK-45079: NULL arguments of percentile_approx") {
+val e1 = intercept[AnalysisException] {
+  sql(
+"""
+  |SELECT percentile_approx(col, array(0.5, 0.4, 0.1), NULL)
+  |FROM VALUES (0), (1), (2), (10) AS tab(col);
+  |""".stripMargin).collect()
+}
+assert(e1.getMessage.contains("Accuracy value must not be null"))
+val e2 = intercept[AnalysisException] {
+  sql(
+"""
+  |SELECT percentile_approx(col, NULL

[spark] branch branch-3.4 updated: [SPARK-45079][SQL] Fix an internal error from `percentile_approx()`on `NULL` accuracy

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

maxgekk 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 f0b421553bc [SPARK-45079][SQL] Fix an internal error from 
`percentile_approx()`on `NULL` accuracy
f0b421553bc is described below

commit f0b421553bc1850cc3e8ed5d564da8f6425cd244
Author: Max Gekk 
AuthorDate: Wed Sep 6 10:32:37 2023 +0300

[SPARK-45079][SQL] Fix an internal error from `percentile_approx()`on 
`NULL` accuracy

### What changes were proposed in this pull request?
In the PR, I propose to check the `accuracy` argument is not a NULL in 
`ApproximatePercentile`. And if it is, throw an `AnalysisException` with new 
error class `DATATYPE_MISMATCH.UNEXPECTED_NULL`.

### Why are the changes needed?
To fix the issue demonstrated by the example:
```sql
$ spark-sql (default)> SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 
NULL) FROM VALUES (0), (1), (2), (10) AS tab(col);
[INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal 
error. You hit a bug in Spark or the Spark plugins you use. Please, report this 
bug to the corresponding communities or vendors, and provide the full stack 
trace.
```

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

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *.ApproximatePercentileQuerySuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42817 from MaxGekk/fix-internal-error-in-percentile_approx.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit 24b29adcf53616067a9fa2ca201e3f4d2f54436b)
Signed-off-by: Max Gekk 
---
 .../aggregate/ApproximatePercentile.scala  |  7 -
 .../sql/ApproximatePercentileQuerySuite.scala  | 31 ++
 2 files changed, 37 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index 1499f358ac4..ebf1085c0c1 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -96,7 +96,8 @@ case class ApproximatePercentile(
   }
 
   // Mark as lazy so that accuracyExpression is not evaluated during tree 
transformation.
-  private lazy val accuracy: Long = 
accuracyExpression.eval().asInstanceOf[Number].longValue
+  private lazy val accuracyNum = accuracyExpression.eval().asInstanceOf[Number]
+  private lazy val accuracy: Long = accuracyNum.longValue
 
   override def inputTypes: Seq[AbstractDataType] = {
 // Support NumericType, DateType, TimestampType and TimestampNTZType since 
their internal types
@@ -137,6 +138,10 @@ case class ApproximatePercentile(
   "inputExpr" -> toSQLExpr(accuracyExpression)
 )
   )
+} else if (accuracyNum == null) {
+  DataTypeMismatch(
+errorSubClass = "UNEXPECTED_NULL",
+messageParameters = Map("exprName" -> "accuracy"))
 } else if (accuracy <= 0 || accuracy > Int.MaxValue) {
   DataTypeMismatch(
 errorSubClass = "VALUE_OUT_OF_RANGE",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 9237c9e9486..8598e92f029 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -337,4 +337,35 @@ class ApproximatePercentileQuerySuite extends QueryTest 
with SharedSparkSession
   Row(Period.ofMonths(200).normalized(), null, 
Duration.ofSeconds(200L)))
 }
   }
+
+  test("SPARK-45079: NULL arguments of percentile_approx") {
+checkError(
+  exception = intercept[AnalysisException] {
+sql(
+  """
+|SELECT percentile_approx(col, array(0.5, 0.4, 0.1), NULL)
+|FROM VALUES (0), (1), (2), (10) AS tab(col);
+|""".stripMargin).collect()
+  },
+  errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL",
+  parameters = Map(
+"exprName" -> "accuracy",
+"sqlExpr" -> "\"percentile_approx(col, array(0.5, 0.4, 0.1), NULL)\""),
+  context = ExpectedContext(
+"", "", 8, 57, "percentile_approx(col, array(0.5

[spark] branch branch-3.5 updated: [SPARK-45079][SQL] Fix an internal error from `percentile_approx()`on `NULL` accuracy

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 9b750e93035 [SPARK-45079][SQL] Fix an internal error from 
`percentile_approx()`on `NULL` accuracy
9b750e93035 is described below

commit 9b750e930357eae092420f09ca9366e49dc589e2
Author: Max Gekk 
AuthorDate: Wed Sep 6 10:32:37 2023 +0300

[SPARK-45079][SQL] Fix an internal error from `percentile_approx()`on 
`NULL` accuracy

### What changes were proposed in this pull request?
In the PR, I propose to check the `accuracy` argument is not a NULL in 
`ApproximatePercentile`. And if it is, throw an `AnalysisException` with new 
error class `DATATYPE_MISMATCH.UNEXPECTED_NULL`.

### Why are the changes needed?
To fix the issue demonstrated by the example:
```sql
$ spark-sql (default)> SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 
NULL) FROM VALUES (0), (1), (2), (10) AS tab(col);
[INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal 
error. You hit a bug in Spark or the Spark plugins you use. Please, report this 
bug to the corresponding communities or vendors, and provide the full stack 
trace.
```

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

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *.ApproximatePercentileQuerySuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42817 from MaxGekk/fix-internal-error-in-percentile_approx.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit 24b29adcf53616067a9fa2ca201e3f4d2f54436b)
Signed-off-by: Max Gekk 
---
 .../aggregate/ApproximatePercentile.scala  |  7 -
 .../sql/ApproximatePercentileQuerySuite.scala  | 31 ++
 2 files changed, 37 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index 3c3afc1c7e7..5b44c3fa31b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -97,7 +97,8 @@ case class ApproximatePercentile(
   }
 
   // Mark as lazy so that accuracyExpression is not evaluated during tree 
transformation.
-  private lazy val accuracy: Long = 
accuracyExpression.eval().asInstanceOf[Number].longValue
+  private lazy val accuracyNum = accuracyExpression.eval().asInstanceOf[Number]
+  private lazy val accuracy: Long = accuracyNum.longValue
 
   override def inputTypes: Seq[AbstractDataType] = {
 // Support NumericType, DateType, TimestampType and TimestampNTZType since 
their internal types
@@ -138,6 +139,10 @@ case class ApproximatePercentile(
   "inputExpr" -> toSQLExpr(accuracyExpression)
 )
   )
+} else if (accuracyNum == null) {
+  DataTypeMismatch(
+errorSubClass = "UNEXPECTED_NULL",
+messageParameters = Map("exprName" -> "accuracy"))
 } else if (accuracy <= 0 || accuracy > Int.MaxValue) {
   DataTypeMismatch(
 errorSubClass = "VALUE_OUT_OF_RANGE",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 18e8dd6249b..273e8e08fd7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -339,4 +339,35 @@ class ApproximatePercentileQuerySuite extends QueryTest 
with SharedSparkSession
   Row(Period.ofMonths(200).normalized(), null, 
Duration.ofSeconds(200L)))
 }
   }
+
+  test("SPARK-45079: NULL arguments of percentile_approx") {
+checkError(
+  exception = intercept[AnalysisException] {
+sql(
+  """
+|SELECT percentile_approx(col, array(0.5, 0.4, 0.1), NULL)
+|FROM VALUES (0), (1), (2), (10) AS tab(col);
+|""".stripMargin).collect()
+  },
+  errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL",
+  parameters = Map(
+"exprName" -> "accuracy",
+"sqlExpr" -> "\"percentile_approx(col, array(0.5, 0.4, 0.1), NULL)\""),
+  context = ExpectedContext(
+"", "", 8, 57, "percentile_approx(col, array(0.5

[spark] branch master updated: [SPARK-45079][SQL] Fix an internal error from `percentile_approx()`on `NULL` accuracy

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

maxgekk 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 24b29adcf53 [SPARK-45079][SQL] Fix an internal error from 
`percentile_approx()`on `NULL` accuracy
24b29adcf53 is described below

commit 24b29adcf53616067a9fa2ca201e3f4d2f54436b
Author: Max Gekk 
AuthorDate: Wed Sep 6 10:32:37 2023 +0300

[SPARK-45079][SQL] Fix an internal error from `percentile_approx()`on 
`NULL` accuracy

### What changes were proposed in this pull request?
In the PR, I propose to check the `accuracy` argument is not a NULL in 
`ApproximatePercentile`. And if it is, throw an `AnalysisException` with new 
error class `DATATYPE_MISMATCH.UNEXPECTED_NULL`.

### Why are the changes needed?
To fix the issue demonstrated by the example:
```sql
$ spark-sql (default)> SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 
NULL) FROM VALUES (0), (1), (2), (10) AS tab(col);
[INTERNAL_ERROR] The Spark SQL phase analysis failed with an internal 
error. You hit a bug in Spark or the Spark plugins you use. Please, report this 
bug to the corresponding communities or vendors, and provide the full stack 
trace.
```

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

### How was this patch tested?
By running new test:
```
$ build/sbt "test:testOnly *.ApproximatePercentileQuerySuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42817 from MaxGekk/fix-internal-error-in-percentile_approx.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../aggregate/ApproximatePercentile.scala  |  7 -
 .../sql/ApproximatePercentileQuerySuite.scala  | 31 ++
 2 files changed, 37 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index 3c3afc1c7e7..5b44c3fa31b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -97,7 +97,8 @@ case class ApproximatePercentile(
   }
 
   // Mark as lazy so that accuracyExpression is not evaluated during tree 
transformation.
-  private lazy val accuracy: Long = 
accuracyExpression.eval().asInstanceOf[Number].longValue
+  private lazy val accuracyNum = accuracyExpression.eval().asInstanceOf[Number]
+  private lazy val accuracy: Long = accuracyNum.longValue
 
   override def inputTypes: Seq[AbstractDataType] = {
 // Support NumericType, DateType, TimestampType and TimestampNTZType since 
their internal types
@@ -138,6 +139,10 @@ case class ApproximatePercentile(
   "inputExpr" -> toSQLExpr(accuracyExpression)
 )
   )
+} else if (accuracyNum == null) {
+  DataTypeMismatch(
+errorSubClass = "UNEXPECTED_NULL",
+messageParameters = Map("exprName" -> "accuracy"))
 } else if (accuracy <= 0 || accuracy > Int.MaxValue) {
   DataTypeMismatch(
 errorSubClass = "VALUE_OUT_OF_RANGE",
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 18e8dd6249b..273e8e08fd7 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -339,4 +339,35 @@ class ApproximatePercentileQuerySuite extends QueryTest 
with SharedSparkSession
   Row(Period.ofMonths(200).normalized(), null, 
Duration.ofSeconds(200L)))
 }
   }
+
+  test("SPARK-45079: NULL arguments of percentile_approx") {
+checkError(
+  exception = intercept[AnalysisException] {
+sql(
+  """
+|SELECT percentile_approx(col, array(0.5, 0.4, 0.1), NULL)
+|FROM VALUES (0), (1), (2), (10) AS tab(col);
+|""".stripMargin).collect()
+  },
+  errorClass = "DATATYPE_MISMATCH.UNEXPECTED_NULL",
+  parameters = Map(
+"exprName" -> "accuracy",
+"sqlExpr" -> "\"percentile_approx(col, array(0.5, 0.4, 0.1), NULL)\""),
+  context = ExpectedContext(
+"", "", 8, 57, "percentile_approx(col, array(0.5, 0.4, 0.1), NULL)"))
+checkError(
+  exception = intercept[AnalysisException] {

[spark] branch master updated: [SPARK-45070][SQL][DOCS] Describe the binary and datetime formats of `to_char`/`to_varchar`

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

maxgekk 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 637f16e7ff8 [SPARK-45070][SQL][DOCS] Describe the binary and datetime 
formats of `to_char`/`to_varchar`
637f16e7ff8 is described below

commit 637f16e7ff88c2aef0e7f29163e13138ff472c1d
Author: Max Gekk 
AuthorDate: Wed Sep 6 08:25:41 2023 +0300

[SPARK-45070][SQL][DOCS] Describe the binary and datetime formats of 
`to_char`/`to_varchar`

### What changes were proposed in this pull request?
In the PR, I propose to document the recent changes related to the `format` 
of the `to_char`/`to_varchar` functions:
1. binary formats added by https://github.com/apache/spark/pull/42632
2. datetime formats introduced by https://github.com/apache/spark/pull/42534

### Why are the changes needed?
To inform users about recent changes.

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

### How was this patch tested?
By CI.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42801 from MaxGekk/doc-to_char-api.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../main/scala/org/apache/spark/sql/functions.scala| 18 --
 python/pyspark/sql/functions.py| 12 
 .../main/scala/org/apache/spark/sql/functions.scala| 18 ++
 3 files changed, 46 insertions(+), 2 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
index 527848e95e6..54bf0106956 100644
--- 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala
@@ -4280,6 +4280,7 @@ object functions {
*/
   def to_binary(e: Column): Column = Column.fn("to_binary", e)
 
+  // scalastyle:off line.size.limit
   /**
* Convert `e` to a string based on the `format`. Throws an exception if the 
conversion fails.
*
@@ -4300,13 +4301,20 @@ object functions {
*   (optional, only allowed once at the beginning or end of the format 
string). Note that 'S'
*   prints '+' for positive values but 'MI' prints a space. 'PR': 
Only allowed at the
*   end of the format string; specifies that the result string will be 
wrapped by angle
-   *   brackets if the input value is negative. 
+   *   brackets if the input value is negative.  If `e` is a 
datetime, `format` shall be
+   *   a valid datetime pattern, see https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html;>Datetime
+   *   Patterns. If `e` is a binary, it is converted to a string in one of 
the formats: 
+   *   'base64': a base 64 string. 'hex': a string in the 
hexadecimal format.
+   *   'utf-8': the input binary is decoded to UTF-8 string. 
*
* @group string_funcs
* @since 3.5.0
*/
+  // scalastyle:on line.size.limit
   def to_char(e: Column, format: Column): Column = Column.fn("to_char", e, 
format)
 
+  // scalastyle:off line.size.limit
   /**
* Convert `e` to a string based on the `format`. Throws an exception if the 
conversion fails.
*
@@ -4327,11 +4335,17 @@ object functions {
*   (optional, only allowed once at the beginning or end of the format 
string). Note that 'S'
*   prints '+' for positive values but 'MI' prints a space. 'PR': 
Only allowed at the
*   end of the format string; specifies that the result string will be 
wrapped by angle
-   *   brackets if the input value is negative. 
+   *   brackets if the input value is negative.  If `e` is a 
datetime, `format` shall be
+   *   a valid datetime pattern, see https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html;>Datetime
+   *   Patterns. If `e` is a binary, it is converted to a string in one of 
the formats: 
+   *   'base64': a base 64 string. 'hex': a string in the 
hexadecimal format.
+   *   'utf-8': the input binary is decoded to UTF-8 string. 
*
* @group string_funcs
* @since 3.5.0
*/
+  // scalastyle:on line.size.limit
   def to_varchar(e: Column, format: Column): Column = Column.fn("to_varchar", 
e, format)
 
   /**
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 56b436421af..de91cced206 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -10902,6 +10902,12 @@ def to_char(col: "ColumnOrName", format: 
"ColumnOrName") -> Column:
 values but 'MI' prints a space.
 'PR': Only allowed at the end of the format string; specifies that the 
result string
 will be wrapped by angle brackets if the input value is negative.
+   

[spark] branch master updated (b0b7835bee2 -> 416207659aa)

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

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


from b0b7835bee2 [SPARK-45059][CONNECT][PYTHON] Add `try_reflect` functions 
to Scala and Python
 add 416207659aa [SPARK-45033][SQL] Support maps by parameterized `sql()`

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/parameters.scala   | 15 --
 .../org/apache/spark/sql/ParametersSuite.scala | 62 +-
 2 files changed, 72 insertions(+), 5 deletions(-)


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



[spark] branch master updated (f2a6c97d718 -> d03ebced0ef)

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

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


from f2a6c97d718 [SPARK-44876][PYTHON][FOLLOWUP] Fix Arrow-optimized Python 
UDF to delay wrapping the function with fail_on_stopiteration
 add d03ebced0ef [SPARK-45060][SQL] Fix an internal error from 
`to_char()`on `NULL` format

No new revisions were added by this update.

Summary of changes:
 common/utils/src/main/resources/error/error-classes.json   |  5 +
 ...error-conditions-invalid-parameter-value-error-class.md |  4 
 .../sql/catalyst/expressions/numberFormatExpressions.scala |  8 ++--
 .../apache/spark/sql/errors/QueryCompilationErrors.scala   |  8 
 .../scala/org/apache/spark/sql/StringFunctionsSuite.scala  | 14 ++
 5 files changed, 37 insertions(+), 2 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-44987][SQL] Assign a name to the error class `_LEGACY_ERROR_TEMP_1100`

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

maxgekk 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 e72ce91250a [SPARK-44987][SQL] Assign a name to the error class 
`_LEGACY_ERROR_TEMP_1100`
e72ce91250a is described below

commit e72ce91250a9a2c40fd5ed55a50dbc46e4e7e46d
Author: Max Gekk 
AuthorDate: Thu Aug 31 22:50:21 2023 +0300

[SPARK-44987][SQL] Assign a name to the error class 
`_LEGACY_ERROR_TEMP_1100`

### What changes were proposed in this pull request?
In the PR, I propose to assign the name `NON_FOLDABLE_ARGUMENT` to the 
legacy error class `_LEGACY_ERROR_TEMP_1100`, and improve the error message 
format: make it less restrictive.

### Why are the changes needed?
1. To don't confuse users by slightly restrictive error message about 
literals.
2. To assign proper name as a part of activity in SPARK-37935

### Does this PR introduce _any_ user-facing change?
No. Only if user's code depends on error class name and message parameters.

### How was this patch tested?
By running the modified and affected tests:
```
$ build/sbt "test:testOnly *.StringFunctionsSuite"
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite"
$ build/sbt "core/testOnly *SparkThrowableSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42737 from MaxGekk/assign-name-_LEGACY_ERROR_TEMP_1100.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json| 11 ---
 docs/sql-error-conditions.md   |  6 
 .../catalyst/expressions/datetimeExpressions.scala |  2 +-
 .../sql/catalyst/expressions/mathExpressions.scala |  4 +--
 .../expressions/numberFormatExpressions.scala  |  2 +-
 .../spark/sql/errors/QueryCompilationErrors.scala  | 14 +
 .../ceil-floor-with-scale-param.sql.out| 36 --
 .../sql-tests/analyzer-results/extract.sql.out | 18 ++-
 .../results/ceil-floor-with-scale-param.sql.out| 36 --
 .../resources/sql-tests/results/extract.sql.out| 18 ++-
 .../apache/spark/sql/StringFunctionsSuite.scala|  8 ++---
 11 files changed, 88 insertions(+), 67 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 3b537cc3d9f..af78dd2f9f8 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -2215,6 +2215,12 @@
 ],
 "sqlState" : "42607"
   },
+  "NON_FOLDABLE_ARGUMENT" : {
+"message" : [
+  "The function  requires the parameter  to be a 
foldable expression of the type , but the actual argument is a 
non-foldable."
+],
+"sqlState" : "22024"
+  },
   "NON_LAST_MATCHED_CLAUSE_OMIT_CONDITION" : {
 "message" : [
   "When there are more than one MATCHED clauses in a MERGE statement, only 
the last MATCHED clause can omit the condition."
@@ -4029,11 +4035,6 @@
   "() doesn't support the  mode. Acceptable modes are 
 and ."
 ]
   },
-  "_LEGACY_ERROR_TEMP_1100" : {
-"message" : [
-  "The '' parameter of function '' needs to be a 
 literal."
-]
-  },
   "_LEGACY_ERROR_TEMP_1103" : {
 "message" : [
   "Unsupported component type  in arrays."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index 89c27f72ea0..33072f6c440 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -1305,6 +1305,12 @@ Cannot call function `` because named 
argument references are not
 
 It is not allowed to use an aggregate function in the argument of another 
aggregate function. Please use the inner aggregate function in a sub-query.
 
+### NON_FOLDABLE_ARGUMENT
+
+[SQLSTATE: 22024](sql-error-conditions-sqlstates.html#class-22-data-exception)
+
+The function `` requires the parameter `` to be a 
foldable expression of the type ``, but the actual argument is a 
non-foldable.
+
 ### NON_LAST_MATCHED_CLAUSE_OMIT_CONDITION
 
 [SQLSTATE: 
42613](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 51ddf2b85f8..30a6bec1868 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/

[spark] branch branch-3.5 updated: [SPARK-43438][SQL] Error on missing input columns in `INSERT`

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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 24bd29cc56a [SPARK-43438][SQL] Error on missing input columns in 
`INSERT`
24bd29cc56a is described below

commit 24bd29cc56a7e12a45d713b5ca0bf2205b80a8f6
Author: Max Gekk 
AuthorDate: Tue Aug 29 23:04:44 2023 +0300

[SPARK-43438][SQL] Error on missing input columns in `INSERT`

### What changes were proposed in this pull request?
In the PR, I propose to raise an error when an user uses V1 `INSERT` 
without a list of columns, and the number of inserting columns doesn't match to 
the number of actual table columns.

At the moment Spark inserts data successfully in such case after the PR 
https://github.com/apache/spark/pull/41262 which changed the behaviour of Spark 
3.4.x.

### Why are the changes needed?
1. To conform the SQL standard which requires the number of columns must be 
the same:
![Screenshot 2023-08-07 at 11 01 27 
AM](https://github.com/apache/spark/assets/1580697/c55badec-5716-490f-a83a-0bb6b22c84c7)

Apparently, the insertion below must not succeed:
```sql
spark-sql (default)> CREATE TABLE tabtest(c1 INT, c2 INT);
spark-sql (default)> INSERT INTO tabtest SELECT 1;
```

2. To have the same behaviour as **Spark 3.4**:
```sql
spark-sql (default)> INSERT INTO tabtest SELECT 1;
`spark_catalog`.`default`.`tabtest` requires that the data to be inserted 
have the same number of columns as the target table: target table has 2 
column(s) but the inserted data has 1 column(s), including 0 partition 
column(s) having constant value(s).
```

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

After the changes:
```sql
spark-sql (default)> INSERT INTO tabtest SELECT 1;
[INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS] Cannot write to 
`spark_catalog`.`default`.`tabtest`, the reason is not enough data columns:
Table columns: `c1`, `c2`.
Data columns: `1`.
```

### How was this patch tested?
By running the modified tests:
```
$ build/sbt "test:testOnly *InsertSuite"
$ build/sbt "test:testOnly *ResolveDefaultColumnsSuite"
$ build/sbt -Phive "test:testOnly *HiveQuerySuite"
    ```

Closes #42393 from MaxGekk/fix-num-cols-insert.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit a7eef2116919bd0c1a1b52adaf49de903e8c9c46)
Signed-off-by: Max Gekk 
---
 .../catalyst/analysis/TableOutputResolver.scala| 15 +-
 .../spark/sql/execution/datasources/rules.scala|  6 ++-
 .../spark/sql/ResolveDefaultColumnsSuite.scala | 59 +-
 .../org/apache/spark/sql/sources/InsertSuite.scala | 18 ---
 .../org/apache/spark/sql/hive/InsertSuite.scala|  2 +-
 .../spark/sql/hive/execution/HiveQuerySuite.scala  |  6 +--
 6 files changed, 69 insertions(+), 37 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
index 894cd0b3991..6671836b351 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
@@ -65,22 +65,11 @@ object TableOutputResolver {
 errors += _,
 fillDefaultValue = supportColDefaultValue)
 } else {
-  // If the target table needs more columns than the input query, fill 
them with
-  // the columns' default values, if the `supportColDefaultValue` 
parameter is true.
-  val fillDefaultValue = supportColDefaultValue && actualExpectedCols.size 
> query.output.size
-  val queryOutputCols = if (fillDefaultValue) {
-query.output ++ actualExpectedCols.drop(query.output.size).flatMap { 
expectedCol =>
-  getDefaultValueExprOrNullLit(expectedCol, 
conf.useNullsForMissingDefaultColumnValues)
-}
-  } else {
-query.output
-  }
-  if (actualExpectedCols.size > queryOutputCols.size) {
+  if (actualExpectedCols.size > query.output.size) {
 throw QueryCompilationErrors.cannotWriteNotEnoughColumnsToTableError(
   tableName, actualExpectedCols.map(_.name), query)
   }
-
-  resolveColumnsByPosition(tableName, queryOutputCols, actualExpectedCols, 
conf, errors += _)
+  resolveColumnsByPosition(tableName, query.output, actualExpectedCols, 
conf, errors += _)
 }
 
 if (errors.nonEmpty) {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution

[spark] branch master updated (8505084bc26 -> a7eef211691)

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

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


from 8505084bc26 [SPARK-45003][PYTHON][DOCS] Refine docstring of `asc/desc`
 add a7eef211691 [SPARK-43438][SQL] Error on missing input columns in 
`INSERT`

No new revisions were added by this update.

Summary of changes:
 .../catalyst/analysis/TableOutputResolver.scala| 15 +-
 .../spark/sql/execution/datasources/rules.scala|  6 ++-
 .../spark/sql/ResolveDefaultColumnsSuite.scala | 59 +-
 .../org/apache/spark/sql/sources/InsertSuite.scala | 18 ---
 .../org/apache/spark/sql/hive/InsertSuite.scala|  2 +-
 .../spark/sql/hive/execution/HiveQuerySuite.scala  |  6 +--
 6 files changed, 69 insertions(+), 37 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-44868][SQL][FOLLOWUP] Invoke the `to_varchar` function in Scala API

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

maxgekk 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 49da438ece8 [SPARK-44868][SQL][FOLLOWUP] Invoke the `to_varchar` 
function in Scala API
49da438ece8 is described below

commit 49da438ece84391db22f9c56e747d555d9b01969
Author: Max Gekk 
AuthorDate: Mon Aug 28 20:57:27 2023 +0300

[SPARK-44868][SQL][FOLLOWUP] Invoke the `to_varchar` function in Scala API

### What changes were proposed in this pull request?
In the PR, I propose to invoke the `to_varchar` function instead of 
`to_char` in `to_varchar` of Scala/Java API.

### Why are the changes needed?
1. To show correct function name in error messages and in `explain`.
2. To be consistent to other API: PySpark and the previous Spark SQL 
version 3.5.0.

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

### How was this patch tested?
By running the modified test:
```
$ build/sbt "test:testOnly *.StringFunctionsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #42703 from MaxGekk/fix-to_varchar-call.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 sql/core/src/main/scala/org/apache/spark/sql/functions.scala  | 2 +-
 .../src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala| 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index f6699b66af9..6b474c84cdb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -4431,7 +4431,7 @@ object functions {
* @group string_funcs
* @since 3.5.0
*/
-  def to_varchar(e: Column, format: Column): Column = to_char(e, format)
+  def to_varchar(e: Column, format: Column): Column = 
call_function("to_varchar", e, format)
 
   /**
* Convert string 'e' to a number based on the string format 'format'.
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
index 12881f4a22a..03b9053c71a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
@@ -878,7 +878,7 @@ class StringFunctionsSuite extends QueryTest with 
SharedSparkSession {
 errorClass = "_LEGACY_ERROR_TEMP_1100",
 parameters = Map(
   "argName" -> "format",
-  "funcName" -> "to_char",
+  "funcName" -> funcName,
   "requiredType" -> "string"))
   checkError(
 exception = intercept[AnalysisException] {
@@ -887,7 +887,7 @@ class StringFunctionsSuite extends QueryTest with 
SharedSparkSession {
 errorClass = "INVALID_PARAMETER_VALUE.BINARY_FORMAT",
 parameters = Map(
   "parameter" -> "`format`",
-  "functionName" -> "`to_char`",
+  "functionName" -> s"`$funcName`",
   "invalidFormat" -> "'invalid_format'"))
   checkError(
 exception = intercept[AnalysisException] {


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



[spark] branch master updated: [SPARK-44983][SQL] Convert binary to string by `to_char` for the formats: `hex`, `base64`, `utf-8`

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

maxgekk 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 4946d025b62 [SPARK-44983][SQL] Convert binary to string by `to_char` 
for the formats: `hex`, `base64`, `utf-8`
4946d025b62 is described below

commit 4946d025b6200ad90dfdfbb1f24526016f810523
Author: Max Gekk 
AuthorDate: Mon Aug 28 16:55:35 2023 +0300

[SPARK-44983][SQL] Convert binary to string by `to_char` for the formats: 
`hex`, `base64`, `utf-8`

### What changes were proposed in this pull request?
In the PR, I propose to re-use the `Hex`, `Base64` and `Decode` expressions 
in the `ToCharacter` (the `to_char`/`to_varchar` functions) when the `format` 
parameter is one of `hex`, `base64` and `utf-8`.

### Why are the changes needed?
To make the migration to Spark SQL easier from the systems like:
- Snowflake: https://docs.snowflake.com/en/sql-reference/functions/to_char
- SAP SQL Anywhere: 
https://help.sap.com/docs/SAP_SQL_Anywhere/93079d4ba8e44920ae63ffb4def91f5b/81fe51196ce21014b9c6cf43b298.html
- Oracle: 
https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/TO_CHAR-number.html#GUID-00DA076D-2468-41AB-A3AC-CC78DBA0D9CB
- Vertica: 
https://www.vertica.com/docs/9.3.x/HTML/Content/Authoring/SQLReferenceManual/Functions/Formatting/TO_CHAR.htm

### Does this PR introduce _any_ user-facing change?
No. This PR extends existing API. It might be considered as an user-facing 
change only if user's code depends on errors in the case of wrong formats.

### How was this patch tested?
By running new examples:
```
$ build/sbt "sql/test:testOnly 
org.apache.spark.sql.expressions.ExpressionInfoSuite"
```
and new tests:
```
$ build/sbt "test:testOnly *.StringFunctionsSuite"
```

### Was this patch authored or co-authored using generative AI tooling?
No.

    Closes #42632 from MaxGekk/to_char-binary-2.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../src/main/resources/error/error-classes.json|  5 ++
 ...nditions-invalid-parameter-value-error-class.md |  4 ++
 .../expressions/numberFormatExpressions.scala  | 28 +++--
 .../spark/sql/errors/QueryCompilationErrors.scala  |  9 +++
 .../apache/spark/sql/StringFunctionsSuite.scala| 69 +++---
 5 files changed, 89 insertions(+), 26 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 632c449b992..53c596c00fc 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -1788,6 +1788,11 @@
   "expects a binary value with 16, 24 or 32 bytes, but got 
 bytes."
 ]
   },
+  "BINARY_FORMAT" : {
+"message" : [
+  "expects one of binary formats 'base64', 'hex', 'utf-8', but got 
."
+]
+  },
   "DATETIME_UNIT" : {
 "message" : [
   "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, 
DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the 
string literal ."
diff --git a/docs/sql-error-conditions-invalid-parameter-value-error-class.md 
b/docs/sql-error-conditions-invalid-parameter-value-error-class.md
index 370e6da3362..96829e564aa 100644
--- a/docs/sql-error-conditions-invalid-parameter-value-error-class.md
+++ b/docs/sql-error-conditions-invalid-parameter-value-error-class.md
@@ -37,6 +37,10 @@ supports 16-byte CBC IVs and 12-byte GCM IVs, but got 
`` bytes for
 
 expects a binary value with 16, 24 or 32 bytes, but got `` bytes.
 
+## BINARY_FORMAT
+
+expects one of binary formats 'base64', 'hex', 'utf-8', but got 
``.
+
 ## DATETIME_UNIT
 
 expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, 
DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string 
literal ``.
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
index 3a424ac21c5..7875ed8fe20 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala
@@ -26,7 +26,7 @@ import 
org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe
 import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
 import org.apache.spark.sql.catalyst.util.ToNumberParser
 import org.apache.spark.sql.errors.QueryCompilationErrors
-import org.apache.spark.sql.t

[spark] branch master updated: [SPARK-44975][SQL] Remove BinaryArithmetic useless override resolved

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

maxgekk 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 04339e30dbd [SPARK-44975][SQL] Remove BinaryArithmetic useless 
override resolved
04339e30dbd is described below

commit 04339e30dbdda2805edbac7e1e3cd8dfb5c3c608
Author: Jia Fan 
AuthorDate: Sat Aug 26 21:11:20 2023 +0300

[SPARK-44975][SQL] Remove BinaryArithmetic useless override resolved

### What changes were proposed in this pull request?
Remove `BinaryArithmetic` useless override resolved, it is exactly the same 
as the abstract class `Expression`

### Why are the changes needed?
remove useless logic

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

### How was this patch tested?
exist test

### Was this patch authored or co-authored using generative AI tooling?

Closes #42689 from Hisoka-X/SPARK-44975_remove_resolved_override.

Authored-by: Jia Fan 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala| 2 --
 1 file changed, 2 deletions(-)

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 31d4d71cd40..2d9bccc0854 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
@@ -264,8 +264,6 @@ abstract class BinaryArithmetic extends BinaryOperator
 
   final override val nodePatterns: Seq[TreePattern] = Seq(BINARY_ARITHMETIC)
 
-  override lazy val resolved: Boolean = childrenResolved && 
checkInputDataTypes().isSuccess
-
   override def initQueryContext(): Option[SQLQueryContext] = {
 if (failOnError) {
   Some(origin.context)


-
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 (352810b2b45 -> aa6f6f74dc9)

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

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


from 352810b2b45 [SPARK-44920][CORE] Use await() instead of 
awaitUninterruptibly() in TransportClientFactory.createClient()
 add aa6f6f74dc9 [SPARK-44871][SQL][3.3] Fix percentile_disc behaviour

No new revisions were added by this update.

Summary of changes:
 .../expressions/aggregate/percentiles.scala|  39 +--
 .../org/apache/spark/sql/internal/SQLConf.scala|  10 ++
 .../resources/sql-tests/inputs/percentiles.sql |  74 +
 .../sql-tests/results/percentiles.sql.out  | 118 +
 4 files changed, 234 insertions(+), 7 deletions(-)
 create mode 100644 sql/core/src/test/resources/sql-tests/inputs/percentiles.sql
 create mode 100644 
sql/core/src/test/resources/sql-tests/results/percentiles.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-44840][SQL] Make `array_insert()` 1-based for negative indexes

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

maxgekk 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 ce50a563d31 [SPARK-44840][SQL] Make `array_insert()` 1-based for 
negative indexes
ce50a563d31 is described below

commit ce50a563d311ccfe36d1fcc4f0743e4e4d7d8116
Author: Max Gekk 
AuthorDate: Tue Aug 22 21:04:32 2023 +0300

[SPARK-44840][SQL] Make `array_insert()` 1-based for negative indexes

### What changes were proposed in this pull request?
In the PR, I propose to make the `array_insert` function 1-based for 
negative indexes. So, the maximum index is -1 should point out to the last 
element, and the function should insert new element at the end of the given 
array for the index -1.

The old behaviour can be restored via the SQL config 
`spark.sql.legacy.negativeIndexInArrayInsert`.

### Why are the changes needed?
1.  To match the behaviour of functions such as `substr()` and 
`element_at()`.
```sql
spark-sql (default)> select element_at(array('a', 'b'), -1), substr('ab', 
-1);
b   b
```
2. To fix an inconsistency in `array_insert` in which positive indexes are 
1-based, but negative indexes are 0-based.

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

Before:
```sql
spark-sql (default)> select array_insert(array('a', 'b'), -1, 'c');
["a","c","b"]
```

After:
```sql
spark-sql (default)> select array_insert(array('a', 'b'), -1, 'c');
["a","b","c"]
```

### How was this patch tested?
By running the modified test suite:
```
$ build/sbt "test:testOnly *CollectionExpressionsSuite"
$ build/sbt "test:testOnly *DataFrameFunctionsSuite"
$ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite"
```

Closes #42564 from MaxGekk/fix-array_insert.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../explain-results/function_array_insert.explain  |  2 +-
 .../explain-results/function_array_prepend.explain |  2 +-
 docs/sql-migration-guide.md|  1 +
 python/pyspark/sql/functions.py|  2 +-
 .../expressions/collectionOperations.scala | 37 ++--
 .../org/apache/spark/sql/internal/SQLConf.scala| 16 +++
 .../expressions/CollectionExpressionsSuite.scala   | 50 +++---
 .../scala/org/apache/spark/sql/functions.scala |  2 +-
 .../sql-tests/analyzer-results/ansi/array.sql.out  | 44 +++
 .../sql-tests/analyzer-results/array.sql.out   | 44 +++
 .../src/test/resources/sql-tests/inputs/array.sql  |  5 +++
 .../resources/sql-tests/results/ansi/array.sql.out | 34 ++-
 .../test/resources/sql-tests/results/array.sql.out | 34 ++-
 .../apache/spark/sql/DataFrameFunctionsSuite.scala |  6 ++-
 14 files changed, 218 insertions(+), 61 deletions(-)

diff --git 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_insert.explain
 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_insert.explain
index edcd790596b..f5096a363a3 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_insert.explain
+++ 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_insert.explain
@@ -1,2 +1,2 @@
-Project [array_insert(e#0, 0, 1) AS array_insert(e, 0, 1)#0]
+Project [array_insert(e#0, 0, 1, false) AS array_insert(e, 0, 1)#0]
 +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_prepend.explain
 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_prepend.explain
index 4c3e7c85d64..1b20682b09d 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_prepend.explain
+++ 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_array_prepend.explain
@@ -1,2 +1,2 @@
-Project [array_insert(e#0, 1, 1) AS array_prepend(e, 1)#0]
+Project [array_insert(e#0, 1, 1, false) AS array_prepend(e, 1)#0]
 +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index c71b16cd8d6..5fc323ec1b0 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -29,6 +29,7 @@ license: |
 - Since Spark 3.5, Row's json and prettyJson methods are moved to `ToJsonUtil`.
 - Since Spark 3.5, the `plan` field is moved from `AnalysisException` to 
`EnhancedAnalysisException`.
 - Since Spark 3.5, `spark.sql.optimizer.c

[spark] branch branch-3.4 updated: [SPARK-44871][SQL][3.4] Fix percentile_disc behaviour

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

maxgekk 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 0060279f733 [SPARK-44871][SQL][3.4] Fix percentile_disc behaviour
0060279f733 is described below

commit 0060279f733989b03aca2bbb0624dfc0c3193aae
Author: Peter Toth 
AuthorDate: Tue Aug 22 19:27:15 2023 +0300

[SPARK-44871][SQL][3.4] Fix percentile_disc behaviour

### What changes were proposed in this pull request?
This PR fixes `percentile_disc()` function as currently it returns 
inforrect results in some cases. E.g.:
```
SELECT
  percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0,
  percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1,
  percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2,
  percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3,
  percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4,
  percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5,
  percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6,
  percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7,
  percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8,
  percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9,
  percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10
FROM VALUES (0), (1), (2), (3), (4) AS v(a)
```
currently returns:
```
+---+---+---+---+---+---+---+---+---+---+---+
| p0| p1| p2| p3| p4| p5| p6| p7| p8| p9|p10|
+---+---+---+---+---+---+---+---+---+---+---+
|0.0|0.0|0.0|1.0|1.0|2.0|2.0|2.0|3.0|3.0|4.0|
+---+---+---+---+---+---+---+---+---+---+---+
```
but after this PR it returns the correct:
```
+---+---+---+---+---+---+---+---+---+---+---+
| p0| p1| p2| p3| p4| p5| p6| p7| p8| p9|p10|
+---+---+---+---+---+---+---+---+---+---+---+
|0.0|0.0|0.0|1.0|1.0|2.0|2.0|3.0|3.0|4.0|4.0|
+---+---+---+---+---+---+---+---+---+---+---+
```

### Why are the changes needed?
Bugfix.

### Does this PR introduce _any_ user-facing change?
Yes, fixes a correctness bug, but the old behaviour can be restored with 
`spark.sql.legacy.percentileDiscCalculation=true`.

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

Closes #42610 from peter-toth/SPARK-44871-fix-percentile-disc-behaviour-3.4.

Authored-by: Peter Toth 
Signed-off-by: Max Gekk 
---
 .../expressions/aggregate/percentiles.scala|  39 +--
 .../org/apache/spark/sql/internal/SQLConf.scala|  10 ++
 .../resources/sql-tests/inputs/percentiles.sql |  77 +-
 .../sql-tests/results/percentiles.sql.out  | 116 +
 4 files changed, 234 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
index 8447a5f9b51..da04c5a1c8a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Cast._
 import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike, UnaryLike}
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.types.TypeCollection.NumericAndAnsiInterval
 import org.apache.spark.util.collection.OpenHashMap
@@ -168,11 +169,8 @@ abstract class PercentileBase
 val accumulatedCounts = sortedCounts.scanLeft((sortedCounts.head._1, 0L)) {
   case ((key1, count1), (key2, count2)) => (key2, count1 + count2)
 }.tail
-val maxPosition = accumulatedCounts.last._2 - 1
 
-percentages.map { percentile =>
-  getPercentile(accumulatedCounts, maxPosition * percentile)
-}
+percentages.map(getPercentile(accumulatedCounts, _))
   }
 
   private def generateOutput(percentiles: Seq[Double]): Any = {
@@ -195,8 +193,11 @@ abstract class PercentileBase
* This function has been based upon similar function from HIVE
* `org.apache.hadoop.hive.ql.udf.UDAFPercentile.getPercentile()`.
*/
-  private def getPercentile(
-  accumulatedCounts: Seq[(AnyRef, Long)], position: Double): Double = {
+  protected def getPercentile(
+  accumulatedCounts: Seq[(AnyRef, Long)],
+  percentile: Double): Double = {
+val position = (accumulatedCounts.last._2 - 1) * percentile
+
 // We may need to do linear interpolation to get the exact percentile
 val lower = position.floor.toLong
 val higher = position.ceil.toLong
@@ -219,6 +220,7 @@ abstract class PercentileBase
 }
 
 if (discrete) {
+ 

<    1   2   3   4   5   6   7   8   9   10   >