[spark] branch master updated: [SPARK-43597][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_0017
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 c6443faea45 [SPARK-43597][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_0017 c6443faea45 is described below commit c6443faea45d5accdbb01f3901ce12d3abce4a20 Author: panbingkun AuthorDate: Mon May 22 22:41:03 2023 +0300 [SPARK-43597][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_0017 ### What changes were proposed in this pull request? The pr aims to assign a name to the error class _LEGACY_ERROR_TEMP_0017. ### Why are the changes needed? The changes improve the error framework. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Update existed UT. Pass GA. Closes #41241 from panbingkun/LEGACY_ERROR_TEMP_0017. Lead-authored-by: panbingkun Co-authored-by: panbingkun <84731...@qq.com> Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json| 10 +- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../apache/spark/sql/errors/QueryParsingErrors.scala| 7 +-- .../sql/catalyst/parser/ExpressionParserSuite.scala | 17 - .../spark/sql/errors/QueryParsingErrorsSuite.scala | 11 +++ 5 files changed, 30 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8b8de042ccf..b5b33758341 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -916,6 +916,11 @@ ], "sqlState" : "42K05" }, + "INVALID_ESC" : { +"message" : [ + "Found an invalid escape string: . The escape string must contain only one character." +] + }, "INVALID_EXECUTOR_MEMORY" : { "message" : [ "Executor memory must be at least . Please increase executor memory using the --executor-memory option or \"\" in Spark configuration." @@ -2202,11 +2207,6 @@ " is not a valid byte length literal, expected syntax: DIGIT+ ('B' | 'K' | 'M' | 'G')." ] }, - "_LEGACY_ERROR_TEMP_0017" : { -"message" : [ - "Invalid escape string. Escape string must contain only one character." -] - }, "_LEGACY_ERROR_TEMP_0018" : { "message" : [ "Function trim doesn't support with type . Please use BOTH, LEADING or TRAILING as trim type." 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 4761836cbad..8eb88f93219 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1865,7 +1865,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val escapeChar = Option(ctx.escapeChar) .map(stringLitCtx => string(visitStringLit(stringLitCtx))).map { str => if (str.length != 1) { -throw QueryParsingErrors.invalidEscapeStringError(ctx) +throw QueryParsingErrors.invalidEscapeStringError(str, ctx) } str.charAt(0) }.getOrElse('\\') diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index 7f4000ec9fa..4b6c3645916 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -188,8 +188,11 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { ctx) } - def invalidEscapeStringError(ctx: PredicateContext): Throwable = { -new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0017", ctx) + def invalidEscapeStringError(invalidEscape: String, ctx: PredicateContext): Throwable = { +new ParseException( + errorClass = "INVALID_ESC", + messageParameters = Map("invalidEscape" -> toSQLValue(invalidEscape, StringType)), + ctx) } def trimOptionUnsupportedError(trimOption: Int, ctx: TrimContext): Throwable = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index c011f49b7aa..5e70402f2e7 100644 --- a/sql/catalyst/sr
[spark] branch master updated: [SPARK-43598][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2400
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 09b37114c9c [SPARK-43598][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2400 09b37114c9c is described below commit 09b37114c9c110f5e24b71f55ff40b26392a7d41 Author: Jiaan Geng AuthorDate: Mon May 22 09:47:35 2023 +0300 [SPARK-43598][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2400 ### What changes were proposed in this pull request? The pr aims to assign a name to the error class _LEGACY_ERROR_TEMP_2400. ### Why are the changes needed? Improve the error framework. ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Exists test cases. Closes #41242 from beliefer/SPARK-43598. Authored-by: Jiaan Geng Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 10 +- .../org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- .../test/resources/sql-tests/analyzer-results/limit.sql.out| 4 ++-- .../sql-tests/analyzer-results/postgreSQL/limit.sql.out| 8 sql/core/src/test/resources/sql-tests/results/limit.sql.out| 4 ++-- .../test/resources/sql-tests/results/postgreSQL/limit.sql.out | 8 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index b130f6f6c93..8b8de042ccf 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1213,6 +1213,11 @@ } } }, + "LIMIT_LIKE_EXPRESSION_IS_UNFOLDABLE" : { +"message" : [ + "The expression must evaluate to a constant value, but got ." +] + }, "LOCATION_ALREADY_EXISTS" : { "message" : [ "Cannot name the managed table as , as its associated location already exists. Please pick a different table name, or remove the existing location first." @@ -5216,11 +5221,6 @@ "failed to evaluate expression : " ] }, - "_LEGACY_ERROR_TEMP_2400" : { -"message" : [ - "The expression must evaluate to a constant value, but got ." -] - }, "_LEGACY_ERROR_TEMP_2401" : { "message" : [ "The expression must be integer type, but got ." 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 b67b4ee9912..3240f9bee56 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 @@ -85,10 +85,10 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB private def checkLimitLikeClause(name: String, limitExpr: Expression): Unit = { limitExpr match { case e if !e.foldable => limitExpr.failAnalysis( -errorClass = "_LEGACY_ERROR_TEMP_2400", +errorClass = "LIMIT_LIKE_EXPRESSION_IS_UNFOLDABLE", messageParameters = Map( "name" -> name, - "limitExpr" -> limitExpr.sql)) + "limitExpr" -> toSQLExpr(limitExpr))) case e if e.dataType != IntegerType => limitExpr.failAnalysis( errorClass = "_LEGACY_ERROR_TEMP_2401", messageParameters = Map( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out index c38e4696605..3b2ddb5dae1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/limit.sql.out @@ -124,9 +124,9 @@ SELECT * FROM testdata LIMIT key > 3 -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2400", + "errorClass" : "LIMIT_LIKE_EXPRESSION_IS_UNFOLDABLE", "messageParameters" : { -"limitExpr" : "(spark_catalog.default.testdata.key > 3)", +"limitExpr" : "\"(key > 3)\"", "name" : "limit" }, "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/limit.sql.out index 8964e7f2340..7ef2912cef2 100644 --- a/sql/core/src/test/resources/sql-tests/analyz
[spark] branch master updated: [SPARK-43542][SS] Define a new error class and apply for the case where streaming query fails due to concurrent run of streaming query with same checkpoint
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 37b9c532d69 [SPARK-43542][SS] Define a new error class and apply for the case where streaming query fails due to concurrent run of streaming query with same checkpoint 37b9c532d69 is described below commit 37b9c532d698a35d2f577a8fd85ba31b4529f5ea Author: Eric Marnadi AuthorDate: Sat May 20 10:33:12 2023 +0300 [SPARK-43542][SS] Define a new error class and apply for the case where streaming query fails due to concurrent run of streaming query with same checkpoint ### What changes were proposed in this pull request? We are migrating to a new error framework in order to surface errors in a friendlier way to customers. This PR defines a new error class specifically for when there are concurrent updates to the log for the same batch ID ### Why are the changes needed? This gives more information to customers, and allows them to filter in a better way ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? There is an existing test to check the error message upon this condition. Because we are only changing the error type, and not the error message, this test is sufficient. Closes #41205 from ericm-db/SC-130782. Authored-by: Eric Marnadi Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 7 +++ .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 7 +++ .../spark/sql/execution/streaming/AsyncCommitLog.scala | 5 ++--- .../spark/sql/execution/streaming/AsyncOffsetSeqLog.scala | 5 ++--- .../AsyncProgressTrackingMicroBatchExecution.scala | 5 ++--- .../sql/execution/streaming/MicroBatchExecution.scala | 14 -- 6 files changed, 28 insertions(+), 15 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 3c7c29f7532..b130f6f6c93 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -212,6 +212,13 @@ "Another instance of this query was just started by a concurrent session." ] }, + "CONCURRENT_STREAM_LOG_UPDATE" : { +"message" : [ + "Concurrent update to the log. Multiple streaming jobs detected for .", + "Please make sure only one streaming job runs on a specific checkpoint location at a time." +], +"sqlState" : "4" + }, "CONNECT" : { "message" : [ "Generic Spark Connect error." 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 99f7489e8bc..67c5fa54732 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 @@ -1409,6 +1409,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map.empty[String, String]) } + def concurrentStreamLogUpdate(batchId: Long): Throwable = { +new SparkException( + errorClass = "CONCURRENT_STREAM_LOG_UPDATE", + messageParameters = Map("batchId" -> batchId.toString), + cause = null) + } + def cannotParseJsonArraysAsStructsError(): SparkRuntimeException = { new SparkRuntimeException( errorClass = "_LEGACY_ERROR_TEMP_2132", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala index e9ad8bed27c..495f2f7ac0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/AsyncCommitLog.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{CompletableFuture, ConcurrentLinkedDeque, ThreadPoo import scala.collection.JavaConverters._ import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.errors.QueryExecutionErrors /** * Implementation of CommitLog to perform asynchronous writes to storage @@ -54,9 +55,7 @@ class AsyncCommitLog(sparkSession: SparkSession, path: String, executorService: if (ret) { batchId } else { -throw new IllegalStateException( - s"Concurrent update to the log. Multiple streaming jobs detected for $batchId" -) +throw QueryExecutionErrors.concurrentStreamLogUpdate(batchId) } }) diff --g
[spark] branch master updated (4382a458d82 -> a06f6ffcc1b)
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 4382a458d82 [SPARK-43584][BUILD] Update `sbt-assembly`, `sbt-revolver`, `sbt-mima-plugin` plugins add a06f6ffcc1b [SPARK-43539][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_0003 No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 10 +- .../scala/org/apache/spark/sql/errors/QueryParsingErrors.scala | 4 ++-- .../org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala | 6 +++--- 3 files changed, 10 insertions(+), 10 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-43345][SPARK-43346][SQL] Rename the error classes _LEGACY_ERROR_TEMP_[0041|1206]
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 df7e2d07f04 [SPARK-43345][SPARK-43346][SQL] Rename the error classes _LEGACY_ERROR_TEMP_[0041|1206] df7e2d07f04 is described below commit df7e2d07f04c7e16af04c1292227a90516083de1 Author: Terry Kim AuthorDate: Fri May 19 10:49:29 2023 +0300 [SPARK-43345][SPARK-43346][SQL] Rename the error classes _LEGACY_ERROR_TEMP_[0041|1206] ### What changes were proposed in this pull request? This PR proposes to assign the proper names to the following `_LEGACY_ERROR_TEMP*` error classes: * `_LEGACY_ERROR_TEMP_0041` -> `DUPLICATE_CLAUSES` * `_LEGACY_ERROR_TEMP_1206` -> `COLUMN_NOT_DEFINED_IN_TABLE` ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? By running modified test suties. Closes #41020 from imback82/error_messages. Authored-by: Terry Kim Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 20 ++-- .../spark/sql/errors/QueryCompilationErrors.scala | 8 ++--- .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 36 +++--- .../command/CreateNamespaceParserSuite.scala | 8 ++--- .../sql/execution/command/DDLParserSuite.scala | 4 +-- .../spark/sql/execution/command/DDLSuite.scala | 16 +- .../execution/command/PlanResolutionSuite.scala| 14 - .../spark/sql/hive/execution/HiveDDLSuite.scala| 2 +- 9 files changed, 55 insertions(+), 55 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index e7203c0292b..dcfe8165735 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -186,6 +186,11 @@ ], "sqlState" : "42711" }, + "COLUMN_NOT_DEFINED_IN_TABLE" : { +"message" : [ + " column is not defined in table , defined table columns are: ." +] + }, "COLUMN_NOT_FOUND" : { "message" : [ "The column cannot be found. Verify the spelling and correctness of the column name according to the SQL config ." @@ -556,6 +561,11 @@ ], "sqlState" : "23505" }, + "DUPLICATE_CLAUSES" : { +"message" : [ + "Found duplicate clauses: . Please, remove one of them." +] + }, "DUPLICATE_KEY" : { "message" : [ "Found duplicate keys ." @@ -2265,11 +2275,6 @@ "Unsupported SQL statement." ] }, - "_LEGACY_ERROR_TEMP_0041" : { -"message" : [ - "Found duplicate clauses: ." -] - }, "_LEGACY_ERROR_TEMP_0043" : { "message" : [ "Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`." @@ -3198,11 +3203,6 @@ "Expected only partition pruning predicates: ." ] }, - "_LEGACY_ERROR_TEMP_1206" : { -"message" : [ - " column is not defined in table , defined table columns are: ." -] - }, "_LEGACY_ERROR_TEMP_1207" : { "message" : [ "The duration and time inputs to window must be an integer, long or string literal." 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 ad0a17ef4f4..bd987dab5bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2132,12 +2132,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def columnNotDefinedInTableError( colType: String, colName: String, tableName: String, tableCols: Seq[String]): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1206", + errorClass = "COLUMN_NOT_DEFINED_IN_TABLE", messageParameters = Map( "colType" -> colType, -"colName" -> colName, -"tableName" -> tableName, -"tableCols" -> tableCols.mkString(", "))) +"colName" -> toSQLId(colName), +"tableName" -> toSQLId(tab
[spark] branch master updated (d7a8b852eaa -> 09d5742a867)
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 d7a8b852eaa [SPARK-43569][SQL] Remove workaround for HADOOP-14067 add 09d5742a867 [SPARK-43541][SQL] Propagate all `Project` tags in resolving of expressions and missing columns No new revisions were added by this update. Summary of changes: .../catalyst/analysis/ColumnResolutionHelper.scala | 4 ++- .../sql-tests/analyzer-results/using-join.sql.out | 30 ++ .../test/resources/sql-tests/inputs/using-join.sql | 7 + .../resources/sql-tests/results/using-join.sql.out | 13 ++ 4 files changed, 53 insertions(+), 1 deletion(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-38469][CORE] Use error class in org.apache.spark.network
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 6f377efd3f3 [SPARK-38469][CORE] Use error class in org.apache.spark.network 6f377efd3f3 is described below commit 6f377efd3f3b8db1909349a7c134929a2ec0bf60 Author: Bo Zhang AuthorDate: Tue May 16 19:13:20 2023 +0300 [SPARK-38469][CORE] Use error class in org.apache.spark.network ### What changes were proposed in this pull request? This PR aims to change exceptions created in package org.apache.spark.netrowk to use error class. This also adds an error class INTERNAL_ERROR_NETWORK and uses that for the internal errors in the package. ### Why are the changes needed? This is to move exceptions created in package org.apache.spark.network to error class. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. Closes #41140 from bozhang2820/spark-38469. Lead-authored-by: Bo Zhang Co-authored-by: Bo Zhang Signed-off-by: Max Gekk --- .../main/scala/org/apache/spark/SparkException.scala | 3 ++- core/src/main/resources/error/error-classes.json | 6 ++ .../spark/network/netty/NettyBlockRpcServer.scala | 19 --- .../network/netty/NettyBlockTransferService.scala | 2 +- 4 files changed, 21 insertions(+), 9 deletions(-) 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 4abf0fdf498..feb7bf5b66e 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala @@ -118,7 +118,8 @@ private[spark] case class SparkUserAppException(exitCode: Int) * Exception thrown when the relative executor to access is dead. */ private[spark] case class ExecutorDeadException(message: String) - extends SparkException(message) + extends SparkException(errorClass = "INTERNAL_ERROR_NETWORK", +messageParameters = Map("message" -> message), cause = null) /** * Exception thrown when Spark returns different result after upgrading to a new version. diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index edc5a5a66e5..24f972a5006 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -830,6 +830,12 @@ ], "sqlState" : "XX000" }, + "INTERNAL_ERROR_NETWORK" : { +"message" : [ + "" +], +"sqlState" : "XX000" + }, "INTERVAL_ARITHMETIC_OVERFLOW" : { "message" : [ "." diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index f2a1fe49fcf..16ad848a326 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import scala.collection.JavaConverters._ import scala.reflect.ClassTag +import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.network.BlockDataManager import org.apache.spark.network.buffer.NioManagedBuffer @@ -93,8 +94,8 @@ class NettyBlockRpcServer( } else { val startAndEndId = fetchShuffleBlocks.reduceIds(index) if (startAndEndId.length != 2) { - throw new IllegalStateException(s"Invalid shuffle fetch request when batch mode " + -s"is enabled: $fetchShuffleBlocks") + throw SparkException.internalError("Invalid shuffle fetch request when batch mode " + +s"is enabled: $fetchShuffleBlocks", category = "NETWORK") } Array(blockManager.getLocalBlockData( ShuffleBlockBatchId( @@ -125,8 +126,10 @@ class NettyBlockRpcServer( if (blockStored) { responseContext.onSuccess(ByteBuffer.allocate(0)) } else { - val exception = new Exception(s"Upload block for $blockId failed. This mostly happens " + -s"when there is not sufficient space available to store the block.") + val exception = SparkException.internalError( +s"Upload block for $blockId failed. This mostly happens " + +"when there is not sufficient space available to store the block.", +category = "NETWORK") responseContex
[spark] branch master updated: [SPARK-39281][SQL] Speed up Timestamp type inference with legacy format in JSON/CSV data source
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 3192bbd2958 [SPARK-39281][SQL] Speed up Timestamp type inference with legacy format in JSON/CSV data source 3192bbd2958 is described below commit 3192bbd29585607d43d0819c6c2d3ac00180261a Author: Jia Fan AuthorDate: Tue May 16 15:59:01 2023 +0300 [SPARK-39281][SQL] Speed up Timestamp type inference with legacy format in JSON/CSV data source ### What changes were proposed in this pull request? Follow up https://github.com/apache/spark/pull/36562 , performance improvement when Timestamp type inference with legacy format. In the current implementation of CSV/JSON data source, the schema inference with legacy format relies on methods that will throw exceptions if the fields can't convert as some data types . Throwing and catching exceptions can be slow. We can improve it by creating methods that return optional results instead. The optimization of DefaultTimestampFormatter has been implemented in https://github.com/apache/spark/pull/36562 , this PR adds the optimization of legacy format. The basic logic is to prevent the formatter from throwing exceptions, and then use catch to determine whether the parsing is successful. ### Why are the changes needed? Performance improvement when Timestamp type inference with legacy format. When use JSON datasource, the speed up `67%`. CSV datasource speed also up, but not obvious. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add new test Closes #41091 from Hisoka-X/SPARK-39281_legacy_format. Lead-authored-by: Jia Fan Co-authored-by: Hisoka Signed-off-by: Max Gekk --- .../sql/catalyst/util/TimestampFormatter.scala | 22 .../catalyst/util/TimestampFormatterSuite.scala| 19 sql/core/benchmarks/CSVBenchmark-jdk11-results.txt | 82 +++--- sql/core/benchmarks/CSVBenchmark-jdk17-results.txt | 82 +++--- sql/core/benchmarks/CSVBenchmark-results.txt | 82 +++--- .../benchmarks/JsonBenchmark-jdk11-results.txt | 98 - .../benchmarks/JsonBenchmark-jdk17-results.txt | 122 ++--- sql/core/benchmarks/JsonBenchmark-results.txt | 122 ++--- 8 files changed, 335 insertions(+), 294 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 2a8283bde1d..aab90ec3844 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -407,6 +407,19 @@ class LegacyFastTimestampFormatter( if (!fastDateFormat.parse(s, new ParsePosition(0), cal)) { throw new IllegalArgumentException(s"'$s' is an invalid timestamp") } +extractMicros(cal) + } + + override def parseOptional(s: String): Option[Long] = { +cal.clear() // Clear the calendar because it can be re-used many times +if (fastDateFormat.parse(s, new ParsePosition(0), cal)) { + Some(extractMicros(cal)) +} else { + None +} + } + + private def extractMicros(cal: MicrosCalendar): Long = { val micros = cal.getMicros() cal.set(Calendar.MILLISECOND, 0) val julianMicros = Math.addExact(millisToMicros(cal.getTimeInMillis), micros) @@ -451,6 +464,15 @@ class LegacySimpleTimestampFormatter( fromJavaTimestamp(new Timestamp(sdf.parse(s).getTime)) } + override def parseOptional(s: String): Option[Long] = { +val date = sdf.parse(s, new ParsePosition(0)) +if (date == null) { + None +} else { + Some(fromJavaTimestamp(new Timestamp(date.getTime))) +} + } + override def format(us: Long): String = { sdf.format(toJavaTimestamp(us)) } 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 10553d421ea..8f6099e96ef 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 @@ -489,4 +489,23 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { assert(formatter.parseWithoutTimeZoneOptional("2012-00-65 23:59:59.9990", false) .isEmpty) } + + test("SPARK-39281: support returning optional parse results in the legacy formatter") { +val fastFormatter = new LegacyFastTimestam
[spark] branch master updated: [SPARK-43518][SQL] Convert `_LEGACY_ERROR_TEMP_2029` to INTERNAL_ERROR
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 dbadb5f275c [SPARK-43518][SQL] Convert `_LEGACY_ERROR_TEMP_2029` to INTERNAL_ERROR dbadb5f275c is described below commit dbadb5f275cf0519b8b1ed78decfe4ce83934825 Author: panbingkun AuthorDate: Tue May 16 12:47:15 2023 +0300 [SPARK-43518][SQL] Convert `_LEGACY_ERROR_TEMP_2029` to INTERNAL_ERROR ### What changes were proposed in this pull request? The pr aims to convert _LEGACY_ERROR_TEMP_2029 to INTERNAL_ERROR. ### Why are the changes needed? 1. I found that it can only be triggered it with the parameter value: UP,DOWN,HALF_DOWN,UNNECESSARY, but from a user's perspective, it is impossible (the internal code limits its value to only: HALF_UP,HALF_EVEN,CEILING,FLOOR), so we should convert it to an internal error. 2. The changes improve the error framework. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? 1. Update existed UT. 2. Pass GA. Closes #41179 from panbingkun/SPARK-43518. Authored-by: panbingkun Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../org/apache/spark/sql/errors/QueryExecutionErrors.scala| 6 ++ .../test/scala/org/apache/spark/sql/types/DecimalSuite.scala | 11 +++ 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index fa838a6da76..edc5a5a66e5 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3873,11 +3873,6 @@ "This line should be unreachable." ] }, - "_LEGACY_ERROR_TEMP_2029" : { -"message" : [ - "Not supported rounding mode: ." -] - }, "_LEGACY_ERROR_TEMP_2030" : { "message" : [ "Can not handle nested schema yet... plan ." 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 1f3ee517dd2..52e8c7df91e 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 @@ -514,10 +514,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("err" -> err)) } - def unsupportedRoundingMode(roundMode: BigDecimal.RoundingMode.Value): SparkRuntimeException = { -new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2029", - messageParameters = Map("roundMode" -> roundMode.toString())) + def unsupportedRoundingMode(roundMode: BigDecimal.RoundingMode.Value): SparkException = { +SparkException.internalError(s"Not supported rounding mode: ${roundMode.toString}.") } def resolveCannotHandleNestedSchema(plan: LogicalPlan): SparkRuntimeException = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index 465c25118fa..ab3f831fbcb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -303,6 +303,17 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper } } + test("Not supported rounding mode: HALF_DOWN") { +val d = Decimal(1L, 100, 80) +checkError( + exception = intercept[SparkException] { +d.toPrecision(5, 50, BigDecimal.RoundingMode.HALF_DOWN) + }, + errorClass = "INTERNAL_ERROR", + parameters = Map("message" -> "Not supported rounding mode: HALF_DOWN.") +) + } + test("SPARK-20341: support BigInt's value does not fit in long value range") { val bigInt = scala.math.BigInt("9223372036854775808") val decimal = Decimal.apply(bigInt) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43485][SQL] Fix the error message for the `unit` argument of the datetime add/diff functions
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 e06275c6f14 [SPARK-43485][SQL] Fix the error message for the `unit` argument of the datetime add/diff functions e06275c6f14 is described below commit e06275c6f14e88ba583ffb3aac1159718a8cae83 Author: Max Gekk AuthorDate: Mon May 15 13:55:18 2023 +0300 [SPARK-43485][SQL] Fix the error message for the `unit` argument of the datetime add/diff functions ### What changes were proposed in this pull request? In the PR, I propose to extend the grammar rule of the `DATEADD`/`TIMESTAMPADD` and `DATEDIFF`/`TIMESTAMPDIFF`, and catch wrong type of the first argument `unit` when an user pass a string instead of an identifier like `YEAR`, ..., `MICROSECOND`. In that case, Spark raised an error of new error class `INVALID_PARAMETER_VALUE.DATETIME_UNIT`. ### Why are the changes needed? To make the error message clear for the case when a literal string instead of an identifier is passed to the datetime `ADD`/`DIFF` functions: ```sql spark-sql (default)> select dateadd('MONTH', 1, date'2023-05-11'); [WRONG_NUM_ARGS.WITHOUT_SUGGESTION] The `dateadd` requires 2 parameters but the actual number is 3. Please, refer to 'https://spark.apache.org/docs/latest/sql-ref-functions.html' for a fix.; line 1 pos 7 ``` ### Does this PR introduce _any_ user-facing change? Yes, it changes the error class. After the changes: ```sql spark-sql (default)> select dateadd('MONTH', 1, date'2023-05-11'); [INVALID_PARAMETER_VALUE.DATETIME_UNIT] The value of parameter(s) `unit` in `dateadd` is invalid: expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal 'MONTH'.(line 1, pos 7) == SQL == select dateadd('MONTH', 1, date'2023-05-11') ---^^^ ``` ### How was this patch tested? By running the existing test suites: ``` $ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite" ``` Closes #41143 from MaxGekk/dateadd-unit-error. Authored-by: Max Gekk Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 + .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 4 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 18 +- .../spark/sql/errors/QueryParsingErrors.scala | 14 ++ .../sql-tests/analyzer-results/ansi/date.sql.out | 88 ++ .../analyzer-results/ansi/timestamp.sql.out| 88 ++ .../sql-tests/analyzer-results/date.sql.out| 88 ++ .../analyzer-results/datetime-legacy.sql.out | 176 +++ .../sql-tests/analyzer-results/timestamp.sql.out | 88 ++ .../timestampNTZ/timestamp-ansi.sql.out| 88 ++ .../timestampNTZ/timestamp.sql.out | 88 ++ .../src/test/resources/sql-tests/inputs/date.sql | 6 + .../test/resources/sql-tests/inputs/timestamp.sql | 6 + .../resources/sql-tests/results/ansi/date.sql.out | 96 +++ .../sql-tests/results/ansi/timestamp.sql.out | 96 +++ .../test/resources/sql-tests/results/date.sql.out | 96 +++ .../sql-tests/results/datetime-legacy.sql.out | 192 + .../resources/sql-tests/results/timestamp.sql.out | 96 +++ .../results/timestampNTZ/timestamp-ansi.sql.out| 96 +++ .../results/timestampNTZ/timestamp.sql.out | 96 +++ 20 files changed, 1521 insertions(+), 4 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index dde165e5fa9..fa838a6da76 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1051,6 +1051,11 @@ "expects a binary value with 16, 24 or 32 bytes, but got bytes." ] }, + "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 ." +] + }, "PATTERN" : { "message" : [ "." diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 2bc79430343..591b0839ac7 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/par
[spark] branch master updated (cfd09a2a9e1 -> b1c0d599ba3)
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 cfd09a2a9e1 [SPARK-43272][CORE] Directly call `createFile` instead of reflection add b1c0d599ba3 [SPARK-39280][SQL] Speed up Timestamp type inference with user-provided format in JSON/CSV data source No new revisions were added by this update. Summary of changes: .../sql/catalyst/util/TimestampFormatter.scala | 66 -- .../catalyst/util/TimestampFormatterSuite.scala| 17 +++ sql/core/benchmarks/CSVBenchmark-jdk11-results.txt | 82 ++-- sql/core/benchmarks/CSVBenchmark-jdk17-results.txt | 70 +- sql/core/benchmarks/CSVBenchmark-results.txt | 94 ++--- .../benchmarks/JsonBenchmark-jdk11-results.txt | 146 ++--- .../benchmarks/JsonBenchmark-jdk17-results.txt | 146 ++--- sql/core/benchmarks/JsonBenchmark-results.txt | 122 - 8 files changed, 399 insertions(+), 344 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-43386][SQL] Improve list of suggested column/attributes in `UNRESOLVED_COLUMN.WITH_SUGGESTION` error class
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 1782d0ed9d3 [SPARK-43386][SQL] Improve list of suggested column/attributes in `UNRESOLVED_COLUMN.WITH_SUGGESTION` error class 1782d0ed9d3 is described below commit 1782d0ed9d3c82caee8e57b94229184e308d8b84 Author: Vitalii Li AuthorDate: Fri May 12 08:38:49 2023 +0300 [SPARK-43386][SQL] Improve list of suggested column/attributes in `UNRESOLVED_COLUMN.WITH_SUGGESTION` error class ### What changes were proposed in this pull request? In this change we determine whether unresolved identifier is multipart or not and remap list of suggested columns to fit the same pattern. - Main change is in `StringUtils.scala`. The rest is caused by method rename and test fixes. ### Why are the changes needed? When determining a list of suggested columns/attributes for `UNRESOLVED_COLUMN.WITH_SUGGESTION` we sort by Levenshtein distance between unresolved identifier and list of fully qualified column names from target table. In case of a single-part identifier this might lead to poor experience especially for short(ish) identifiers, e.g. `a` and table with columns `m` and `aa` in default spark catalog => suggested list will be (`spark_catalog.default.m`, `spark_catalog.default.aa`) ### Does this PR introduce _any_ user-facing change? No, since we don't document internals of how we generate suggestion list for this error. ### How was this patch tested? Existing tests. Closes #41038 from vitaliili-db/missing_col. Authored-by: Vitalii Li Signed-off-by: Max Gekk --- .../spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../spark/sql/catalyst/util/StringUtils.scala | 33 -- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../columnresolution-negative.sql.out | 2 +- .../sql-tests/analyzer-results/cte.sql.out | 2 +- .../analyzer-results/group-by-all-duckdb.sql.out | 2 +- .../analyzer-results/group-by-all.sql.out | 2 +- .../sql-tests/analyzer-results/group-by.sql.out| 4 +-- .../analyzer-results/order-by-all.sql.out | 2 +- .../sql-tests/analyzer-results/pivot.sql.out | 4 +-- .../analyzer-results/postgreSQL/join.sql.out | 2 +- .../analyzer-results/postgreSQL/union.sql.out | 2 +- .../analyzer-results/query_regex_column.sql.out| 14 - .../negative-cases/invalid-correlation.sql.out | 2 +- .../analyzer-results/table-aliases.sql.out | 2 +- .../udf/postgreSQL/udf-join.sql.out| 2 +- .../analyzer-results/udf/udf-group-by.sql.out | 2 +- .../analyzer-results/udf/udf-pivot.sql.out | 4 +-- .../results/columnresolution-negative.sql.out | 2 +- .../test/resources/sql-tests/results/cte.sql.out | 2 +- .../sql-tests/results/group-by-all-duckdb.sql.out | 2 +- .../sql-tests/results/group-by-all.sql.out | 2 +- .../resources/sql-tests/results/group-by.sql.out | 4 +-- .../sql-tests/results/order-by-all.sql.out | 2 +- .../test/resources/sql-tests/results/pivot.sql.out | 4 +-- .../sql-tests/results/postgreSQL/join.sql.out | 2 +- .../sql-tests/results/postgreSQL/union.sql.out | 2 +- .../sql-tests/results/query_regex_column.sql.out | 14 - .../negative-cases/invalid-correlation.sql.out | 2 +- .../sql-tests/results/table-aliases.sql.out| 2 +- .../sql-tests/results/udaf/udaf-group-by.sql.out | 4 +-- .../results/udf/postgreSQL/udf-join.sql.out| 2 +- .../sql-tests/results/udf/udf-group-by.sql.out | 2 +- .../sql-tests/results/udf/udf-pivot.sql.out| 4 +-- .../scala/org/apache/spark/sql/SubquerySuite.scala | 2 +- .../spark/sql/connector/DataSourceV2SQLSuite.scala | 8 ++ .../sql/errors/QueryCompilationErrorsSuite.scala | 6 ++-- .../apache/spark/sql/execution/SQLViewSuite.scala | 4 +-- .../execution/command/v2/DescribeTableSuite.scala | 4 +-- .../org/apache/spark/sql/sources/InsertSuite.scala | 3 +- .../apache/spark/sql/hive/HiveParquetSuite.scala | 4 +-- 43 files changed, 98 insertions(+), 75 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 c3480c35680..dbc9da1ea22 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 @@ -3366,7 +3366,7 @@ class Analyzer(override val catalogMana
[spark] branch master updated (bd669a927f0 -> 46251f00b85)
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 bd669a927f0 [SPARK-43443][SQL] Add benchmark for Timestamp type inference when use invalid value add 46251f00b85 [SPARK-38467][CORE] Use error classes in org.apache.spark.memory No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 18 ++ .../org/apache/spark/memory/StorageMemoryPool.scala | 3 ++- .../apache/spark/memory/UnifiedMemoryManager.scala | 20 +--- 3 files changed, 33 insertions(+), 8 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43443][SQL] Add benchmark for Timestamp type inference when use invalid value
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 bd669a927f0 [SPARK-43443][SQL] Add benchmark for Timestamp type inference when use invalid value bd669a927f0 is described below commit bd669a927f09f271afd6a1058493c23d8a0e3c04 Author: Hisoka AuthorDate: Thu May 11 13:35:25 2023 +0300 [SPARK-43443][SQL] Add benchmark for Timestamp type inference when use invalid value ### What changes were proposed in this pull request? When we try to speed up Timestamp type inference with format (PR: #36562 #41078 #41091). There is no way to judge whether the change has improved the speed for Timestamp type inference. So we need a benchmark to measure whether our optimization of Timestamp type inference is useful, we have valid Timestamp value benchmark at now, but don't have invalid Timestamp value benchmark when use Timestamp type inference. ### Why are the changes needed? Add new benchmark for Timestamp type inference when use invalid value, to make sure our speed up PR work normally. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? benchmarks already are test code. Closes #41131 from Hisoka-X/add_banchmarks. Authored-by: Hisoka Signed-off-by: Max Gekk --- sql/core/benchmarks/CSVBenchmark-jdk11-results.txt | 95 ++--- sql/core/benchmarks/CSVBenchmark-jdk17-results.txt | 95 ++--- sql/core/benchmarks/CSVBenchmark-results.txt | 95 ++--- .../benchmarks/JsonBenchmark-jdk11-results.txt | 123 - .../benchmarks/JsonBenchmark-jdk17-results.txt | 123 - sql/core/benchmarks/JsonBenchmark-results.txt | 147 +++-- .../execution/datasources/csv/CSVBenchmark.scala | 30 + .../execution/datasources/json/JsonBenchmark.scala | 26 8 files changed, 404 insertions(+), 330 deletions(-) diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index ca33c059b3a..0185251877e 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -2,66 +2,69 @@ Benchmark to measure CSV read/write performance -OpenJDK 64-Bit Server VM 11.0.18+10 on Linux 5.15.0-1031-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.18+10 on Linux 5.15.0-1036-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz Parsing quoted values:Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative -One quoted string 36620 36718 168 0.0 732395.8 1.0X +One quoted string 30782 30948 229 0.0 615635.9 1.0X -OpenJDK 64-Bit Server VM 11.0.18+10 on Linux 5.15.0-1031-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.18+10 on Linux 5.15.0-1036-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms)Rate(M/s) Per Row(ns) Relative -Select 1000 columns 86305 86907 1033 0.0 86305.2 1.0X -Select 100 columns38778 38792 15 0.0 38778.3 2.2X -Select one column 31901 31913 12 0.0 31901.0 2.7X -count()6971 7033 61 0.16970.9 12.4X -Select 100 columns, one bad input field 51175 51195 26 0.0 51174.8 1.7X -Select 100 columns, corrupt record field 56219 56283 60 0.0 56219.3 1.5X +Select 1000 columns 74038 74677 1024 0.0 74038.3 1.0X +Select 100 columns33611 33625 12 0.0 33611.1 2.2X +Select one column 29350 29428 73 0.0 29349.7 2.5X +count()4909 4934 26 0.24908.8 15.1X
[spark] branch master updated: [SPARK-42585][CONNECT][FOLLOWUP] Store cached local relations as proto
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 5750bdc553b [SPARK-42585][CONNECT][FOLLOWUP] Store cached local relations as proto 5750bdc553b is described below commit 5750bdc553be6f78ceb81a16d124ecc11481db8d Author: Max Gekk AuthorDate: Wed May 10 14:29:17 2023 +0300 [SPARK-42585][CONNECT][FOLLOWUP] Store cached local relations as proto ### What changes were proposed in this pull request? In the PR, I propose to store the cached local relations in the proto format, the same as `LocalRelation`. Also I reverted `transformLocalRelation()` to the state before the commit https://github.com/apache/spark/commit/0d7618a2ca847cf9577659e50409dd5a383d66d3. ### Why are the changes needed? To explicitly specify the format of cached local relations in the proto API. ### Does this PR introduce _any_ user-facing change? Yes but the feature of cached local relations haven't been released yet. ### How was this patch tested? By running the existing tests: ``` $ build/sbt "test:testOnly *.ArtifactManagerSuite" $ build/sbt "test:testOnly *.ClientE2ETestSuite" $ build/sbt "test:testOnly *.ArtifactStatusesHandlerSuite" ``` Closes #41107 from MaxGekk/cached-blob-in-proto. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../scala/org/apache/spark/sql/SparkSession.scala | 2 +- .../sql/connect/client/SparkConnectClient.scala| 18 ++- .../main/protobuf/spark/connect/relations.proto| 2 +- .../sql/connect/planner/SparkConnectPlanner.scala | 133 ++--- python/pyspark/sql/connect/proto/relations_pb2.pyi | 2 +- 5 files changed, 71 insertions(+), 86 deletions(-) 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 4e5474a33b7..7395bb5f16c 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 @@ -131,7 +131,7 @@ class SparkSession private[sql] ( .setSchema(encoder.schema.json) .setData(arrowData) } else { - val hash = client.cacheLocalRelation(arrowDataSize, arrowData, encoder.schema.json) + val hash = client.cacheLocalRelation(arrowData, encoder.schema.json) builder.getCachedLocalRelationBuilder .setUserId(client.userId) .setSessionId(client.sessionId) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index a5aabe62ae4..0c3b1cae091 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -18,8 +18,6 @@ package org.apache.spark.sql.connect.client import java.net.URI -import java.nio.ByteBuffer -import java.nio.charset.StandardCharsets import java.util.UUID import java.util.concurrent.Executor @@ -237,14 +235,14 @@ private[sql] class SparkConnectClient( /** * Cache the given local relation at the server, and return its key in the remote cache. */ - def cacheLocalRelation(size: Int, data: ByteString, schema: String): String = { -val schemaBytes = schema.getBytes(StandardCharsets.UTF_8) -val locRelData = data.toByteArray -val locRel = ByteBuffer.allocate(4 + locRelData.length + schemaBytes.length) -locRel.putInt(size) -locRel.put(locRelData) -locRel.put(schemaBytes) -artifactManager.cacheArtifact(locRel.array()) + def cacheLocalRelation(data: ByteString, schema: String): String = { +val localRelation = proto.Relation + .newBuilder() + .getLocalRelationBuilder + .setSchema(schema) + .setData(data) + .build() +artifactManager.cacheArtifact(localRelation.toByteArray) } } diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index 984b7d3166c..68133f509f3 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -390,7 +390,7 @@ message CachedLocalRelation { // (Required) An identifier of the Spark SQL session in which the user created the local relation. string sessionId = 2; - // (Required) A sha-256 hash of the serialized local relation. +
[spark] branch master updated: [SPARK-37942][CORE][SQL] Migrate error classes
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 b913e015e76 [SPARK-37942][CORE][SQL] Migrate error classes b913e015e76 is described below commit b913e015e76b60753d227a25ff6c4d49d77ed4ae Author: narek_karapetian AuthorDate: Wed May 10 13:59:59 2023 +0300 [SPARK-37942][CORE][SQL] Migrate error classes ### What changes were proposed in this pull request? Rename error classes: 1. QueryCompilationErrors.cannotReadCorruptedTablePropertyError `_LEGACY_ERROR_TEMP_1091` -> `INSUFFICIENT_TABLE_PROPERTY` (with subclasses) + add test case + refactor CatalogTable.readLargeTableProp function 2. QueryCompilationErrors.notSupportedInJDBCCatalog `_LEGACY_ERROR_TEMP_1119` -> `NOT_SUPPORTED_IN_JDBC_CATALOG` (with subclasses) + add test case 3. QueryCompilationErrors.cannotSetJDBCNamespaceWithPropertyError + add test case 4. QueryCompilationErrors.alterTableSerDePropertiesNotSupportedForV2TablesError `_LEGACY_ERROR_TEMP_1124` -> `NOT_SUPPORTED_COMMAND_FOR_V2_TABLE` + add test case + updated test case in `AlterTableSetSerdeSuite` test class 5. QueryCompilationErrors.unsetNonExistentPropertyError rename to QueryCompilationErrors.unsetNonExistentPropertiesError + change error message and code to include all the nonexistent keys + add test case 6. QueryCompilationErrors.cannotUnsetJDBCNamespaceWithPropertyError `_LEGACY_ERROR_TEMP_1119` -> `NOT_SUPPORTED_IN_JDBC_CATALOG` + not able to reproduce this error class from tests ### Why are the changes needed? We should assign proper name to LEGACY_ERROR_TEMP* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By existing unit tests and an additional test cases were added. Closes #41018 from NarekDW/SPARK-37942-migrate-errors. Lead-authored-by: narek_karapetian Co-authored-by: Narek Karapetian Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 66 +-- .../spark/sql/catalyst/catalog/interface.scala | 27 .../spark/sql/errors/QueryCompilationErrors.scala | 59 ++--- .../apache/spark/sql/execution/command/ddl.scala | 9 +-- .../datasources/v2/jdbc/JDBCTableCatalog.scala | 2 +- .../spark/sql/connector/DataSourceV2SQLSuite.scala | 24 ++- .../sql/errors/QueryCompilationErrorsSuite.scala | 75 ++ .../command/v2/AlterTableSetSerdeSuite.scala | 13 ++-- .../spark/sql/hive/MetastoreDataSourcesSuite.scala | 51 --- .../spark/sql/hive/execution/HiveDDLSuite.scala| 8 ++- 10 files changed, 252 insertions(+), 82 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index dc97a735b39..269e2e4cb7b 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -789,6 +789,23 @@ ], "sqlState" : "21S01" }, + "INSUFFICIENT_TABLE_PROPERTY" : { +"message" : [ + "Can't find table property:" +], +"subClass" : { + "MISSING_KEY" : { +"message" : [ + "." +] + }, + "MISSING_KEY_PART" : { +"message" : [ + ", parts are expected." +] + } +} + }, "INTERNAL_ERROR" : { "message" : [ "" @@ -1244,6 +1261,30 @@ }, "sqlState" : "42000" }, + "NOT_SUPPORTED_COMMAND_FOR_V2_TABLE" : { +"message" : [ + " is not supported for v2 tables." +], +"sqlState" : "46110" + }, + "NOT_SUPPORTED_IN_JDBC_CATALOG" : { +"message" : [ + "Not supported command in JDBC catalog:" +], +"subClass" : { + "COMMAND" : { +"message" : [ + "" +] + }, + "COMMAND_WITH_PROPERTY" : { +"message" : [ + " with property ." +] + } +}, +"sqlState" : "46110" + }, "NO_HANDLER_FOR_UDAF" : { "message" : [ "No handler for UDAF ''. Use sparkSession.udf.register(...) instead." @@ -1670,6 +1711,11 @@ ], "sqlState" : "42703" }, + "UNSET_NONEXISTENT_PROPERTIES" : { +"message" : [ + "Attempted to unset non-existent properties [] in table ." +] + }, "UNSUPPORTED_ARROWTYPE" : {
[spark] branch master updated: [SPARK-43286][SQL] Updates aes_encrypt CBC mode to generate random IVs
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 2643223e21b [SPARK-43286][SQL] Updates aes_encrypt CBC mode to generate random IVs 2643223e21b is described below commit 2643223e21b6e80ea150b41a99c040ef7eebd51a Author: Steve Weis AuthorDate: Tue May 9 11:56:28 2023 +0300 [SPARK-43286][SQL] Updates aes_encrypt CBC mode to generate random IVs ### What changes were proposed in this pull request? The current implementation of AES-CBC mode called via `aes_encrypt` and `aes_decrypt` uses a key derivation function (KDF) based on OpenSSL's [EVP_BytesToKey](https://www.openssl.org/docs/man3.0/man3/EVP_BytesToKey.html). This is intended for generating keys based on passwords and OpenSSL's documents discourage its use: "Newer applications should use a more modern algorithm". `aes_encrypt` and `aes_decrypt` should use the key directly in CBC mode, as it does for both GCM and ECB mode. The output should then be the initialization vector (IV) prepended to the ciphertext – as is done with GCM mode: `[16-byte randomly generated IV | AES-CBC encrypted ciphertext]` ### Why are the changes needed? We want to have the ciphertext output similar across different modes. OpenSSL's EVP_BytesToKey is effectively deprecated and their own documentation says not to use it. Instead, CBC mode will generate a random vector. ### Does this PR introduce _any_ user-facing change? AES-CBC output generated by the previous format will be incompatible with this change. That change was recently landed and we want to land this before CBC mode is used in practice. ### How was this patch tested? A new unit test in `DataFrameFunctionsSuite` was added to test both GCM and CBC modes. Also, a new standalone unit test suite was added in `ExpressionImplUtilsSuite` to test all the modes and various key lengths. ``` build/sbt "sql/test:testOnly org.apache.spark.sql.DataFrameFunctionsSuite" build/sbt "sql/test:testOnly org.apache.spark.sql.catalyst.expressions.ExpressionImplUtilsSuite" ``` CBC values can be verified with `openssl enc` using the following command: ``` echo -n "[INPUT]" | openssl enc -a -e -aes-256-cbc -iv [HEX IV] -K [HEX KEY] echo -n "Spark" | openssl enc -a -e -aes-256-cbc -iv f8c832cc9c61bac6151960a58e4edf86 -K 6162636465666768696a6b6c6d6e6f7031323334353637384142434445464748 ``` Closes #40969 from sweisdb/SPARK-43286. Authored-by: Steve Weis Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../catalyst/expressions/ExpressionImplUtils.java | 187 ++--- .../spark/sql/catalyst/expressions/misc.scala | 4 +- .../spark/sql/errors/QueryExecutionErrors.scala| 9 - .../apache/spark/sql/DataFrameFunctionsSuite.scala | 36 +++- .../expressions/ExpressionImplUtilsSuite.scala | 113 + .../sql/errors/QueryExecutionErrorsSuite.scala | 19 --- 7 files changed, 232 insertions(+), 141 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8b0d98c7e3d..dc97a735b39 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1016,11 +1016,6 @@ "expects a binary value with 16, 24 or 32 bytes, but got bytes." ] }, - "AES_SALTED_MAGIC" : { -"message" : [ - "Initial bytes from input do not match 'Salted__' (0x53616C7465645F5F)." -] - }, "PATTERN" : { "message" : [ "." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java index 680ad11ad73..6843a348006 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java @@ -26,27 +26,54 @@ import javax.crypto.spec.IvParameterSpec; import javax.crypto.spec.SecretKeySpec; import java.nio.ByteBuffer; import java.security.GeneralSecurityException; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; import java.security.SecureRandom; -import java.util.Arrays; +import java.security.spec.AlgorithmParameterSpec; -import static java.nio.charset.StandardCharsets.US_ASCII; /** - * An utility class for constructing expressions. + * A utility class for constructing expressions. */ public cla
[spark] branch master updated: [SPARK-38462][CORE] Add error class INTERNAL_ERROR_EXECUTOR
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 566980fba1c [SPARK-38462][CORE] Add error class INTERNAL_ERROR_EXECUTOR 566980fba1c is described below commit 566980fba1c36fc41ffef9be41382c6bbeec9f55 Author: Bo Zhang AuthorDate: Sat May 6 14:14:13 2023 +0300 [SPARK-38462][CORE] Add error class INTERNAL_ERROR_EXECUTOR ### What changes were proposed in this pull request? This change adds a error class INTERNAL_ERROR_EXECUTOR and change exceptions created in the executor package to use that error class. ### Why are the changes needed? This is to move exceptions created in package org.apache.spark.executor to error class. ### Does this PR introduce _any_ user-facing change? Yes. The exceptions created in package org.apache.spark.executor will change to be of error class INTERNAL_ERROR_EXECUTOR. ### How was this patch tested? Existing unit tests. Closes #41048 from bozhang2820/spark-38462. Authored-by: Bo Zhang Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 6 ++ core/src/main/scala/org/apache/spark/executor/Executor.scala | 8 .../scala/org/apache/spark/executor/ProcfsMetricsGetter.scala | 3 ++- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 88740ca66ff..8b0d98c7e3d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -801,6 +801,12 @@ ], "sqlState" : "XX000" }, + "INTERNAL_ERROR_EXECUTOR" : { +"message" : [ + "" +], +"sqlState" : "XX000" + }, "INTERVAL_ARITHMETIC_OVERFLOW" : { "message" : [ "." diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4ea11d753e1..5d623b22abd 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -561,7 +561,7 @@ private[spark] class Executor( if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, $taskName" if (conf.get(UNSAFE_EXCEPTION_ON_MEMORY_LEAK)) { - throw new SparkException(errMsg) + throw SparkException.internalError(errMsg, category = "EXECUTOR") } else { logWarning(errMsg) } @@ -572,7 +572,7 @@ private[spark] class Executor( s"${releasedLocks.size} block locks were not released by $taskName\n" + releasedLocks.mkString("[", ", ", "]") if (conf.get(STORAGE_EXCEPTION_PIN_LEAK)) { - throw new SparkException(errMsg) + throw SparkException.internalError(errMsg, category = "EXECUTOR") } else { logInfo(errMsg) } @@ -934,9 +934,9 @@ private[spark] class Executor( } else { // In non-local-mode, the exception thrown here will bubble up to the uncaught exception // handler and cause the executor JVM to exit. -throw new SparkException( +throw SparkException.internalError( s"Killing executor JVM because killed task $taskId could not be stopped within " + -s"$killTimeoutMs ms.") +s"$killTimeoutMs ms.", category = "EXECUTOR") } } } finally { diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index 2c9ccbc5d86..5448d7da6d6 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -148,7 +148,8 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L if (exitCode != 0 && exitCode > 2) { val cmd = builder.command().toArray.mkString(" ") logWarning(s"Process $cmd exited with code $exitCode and stderr: $errorString") -throw new SparkException(s"Process $cmd exited with code $exitCode") +throw SparkException.internalError(msg = s"Process $cmd exited with code $exitCode", + category = "EXECUTOR") } childPidsInInt } catch { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43387][SQL] Provide a human readable error code for _LEGACY_ERROR_TEMP_1168
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 6c74e355715 [SPARK-43387][SQL] Provide a human readable error code for _LEGACY_ERROR_TEMP_1168 6c74e355715 is described below commit 6c74e355715882e69c6140af3c52010c8e8258d4 Author: Eric Ogren AuthorDate: Sat May 6 12:07:34 2023 +0300 [SPARK-43387][SQL] Provide a human readable error code for _LEGACY_ERROR_TEMP_1168 ### What changes were proposed in this pull request? Change the error code of `_LEGACY_ERROR_TEMP_1168` -> `INSERT_COLUMN_ARITY_MISMATCH` ### Why are the changes needed? Cleaning up legacy error codes ### Does this PR introduce _any_ user-facing change? Yes - changes error code for legacy error. ### How was this patch tested? existing tests suffice Closes #41068 from eogren/SPARK-43387. Authored-by: Eric Ogren Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 2 +- .../sql-tests/analyzer-results/postgreSQL/numeric.sql.out | 3 ++- .../resources/sql-tests/results/postgreSQL/numeric.sql.out| 3 ++- .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 3 ++- 5 files changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index abb843d519f..88740ca66ff 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -783,6 +783,12 @@ ], "sqlState" : "42704" }, + "INSERT_COLUMN_ARITY_MISMATCH" : { +"message" : [ + " requires that the data to be inserted have the same number of columns as the target table: target table has column(s) but the inserted data has column(s), including partition column(s) having constant value(s)." +], +"sqlState" : "21S01" + }, "INTERNAL_ERROR" : { "message" : [ "" @@ -2969,11 +2975,6 @@ "Bucket sorting column '' should not be part of partition columns ''." ] }, - "_LEGACY_ERROR_TEMP_1168" : { -"message" : [ - " requires that the data to be inserted have the same number of columns as the target table: target table has column(s) but the inserted data has column(s), including partition column(s) having constant value(s)." -] - }, "_LEGACY_ERROR_TEMP_1169" : { "message" : [ "Requested partitioning does not match the table :", 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 e1a09a4f843..e332e95c50d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1717,7 +1717,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def mismatchedInsertedDataColumnNumberError( tableName: String, insert: InsertIntoStatement, staticPartCols: Set[String]): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1168", + errorClass = "INSERT_COLUMN_ARITY_MISMATCH", messageParameters = Map( "tableName" -> tableName, "targetColumns" -> insert.table.output.size.toString, diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out index 7c4ffa76710..d32e2abe156 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out @@ -3841,7 +3841,8 @@ INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_1168", + "errorClass" : "INSERT_COLUMN_ARITY_MISMATCH", + "sqlState" : "21S01", "messageParameters" : { "insertedColumns" : "5", "staticPartCols" : "0", diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 8e63191a803..db81160bf03 100644 --- a/sql/core/src/test/resources/sql-
[spark] branch master updated: [SPARK-42842][SQL] Merge the error class _LEGACY_ERROR_TEMP_2006 into REGEX_GROUP_INDEX
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 59d14d73521 [SPARK-42842][SQL] Merge the error class _LEGACY_ERROR_TEMP_2006 into REGEX_GROUP_INDEX 59d14d73521 is described below commit 59d14d7352109b6b14ae435073979437c857c99f Author: Liang Yan AuthorDate: Sat May 6 09:10:53 2023 +0300 [SPARK-42842][SQL] Merge the error class _LEGACY_ERROR_TEMP_2006 into REGEX_GROUP_INDEX ### What changes were proposed in this pull request? Merge the error class _LEGACY_ERROR_TEMP_2006 into REGEX_GROUP_INDEX. ### Why are the changes needed? Fix jira issue [SPARK-42842](https://issues.apache.org/jira/browse/SPARK-42842). The original name just a number, update it to an informal name. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Current tests covered it. Closes #41059 from liang3zy22/spark42842. Authored-by: Liang Yan Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 --- .../catalyst/expressions/regexpExpressions.scala | 6 +-- .../spark/sql/errors/QueryExecutionErrors.scala| 8 +--- .../expressions/RegexpExpressionsSuite.scala | 49 ++ .../sql-tests/results/regexp-functions.sql.out | 44 +++ 5 files changed, 79 insertions(+), 33 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 4eea5f9684e..abb843d519f 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3752,11 +3752,6 @@ "Type does not support ordered operations." ] }, - "_LEGACY_ERROR_TEMP_2006" : { -"message" : [ - "The specified group index cannot be less than zero." -] - }, "_LEGACY_ERROR_TEMP_2011" : { "message" : [ "Unexpected data type ." 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 2025a554998..7f37f02f5a9 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 @@ -747,10 +747,8 @@ object RegExpReplace { object RegExpExtractBase { def checkGroupIndex(prettyName: String, groupCount: Int, groupIndex: Int): Unit = { -if (groupIndex < 0) { - throw QueryExecutionErrors.regexGroupIndexLessThanZeroError -} else if (groupCount < groupIndex) { - throw QueryExecutionErrors.regexGroupIndexExceedGroupCountError( +if (groupIndex < 0 || groupCount < groupIndex) { + throw QueryExecutionErrors.invalidRegexGroupIndexError( prettyName, groupCount, groupIndex) } } 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 111f0391a72..bc51727f8fb 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 @@ -357,13 +357,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("dataType" -> dataType)) } - def regexGroupIndexLessThanZeroError(): SparkIllegalArgumentException = { -new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_2006", - messageParameters = Map.empty) - } - - def regexGroupIndexExceedGroupCountError( + def invalidRegexGroupIndexError( funcName: String, groupCount: Int, groupIndex: Int): RuntimeException = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index ace0c7959a1..c69944b2161 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -421,11 +421,26 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { "groupIndex" -> "1" ) ) -checkExceptionInExpression[IllegalArgumentException]( - expr, row11, "The specified group index cannot be less than zero") -checkExceptionInExp
[spark] branch master updated: [SPARK-42843][SQL][FOLLOWUP] Remove the old error class _LEGACY_ERROR_TEMP_2007
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 97e7d497476 [SPARK-42843][SQL][FOLLOWUP] Remove the old error class _LEGACY_ERROR_TEMP_2007 97e7d497476 is described below commit 97e7d4974763b9851dc7a5050115668fcbe86b74 Author: Liang Yan AuthorDate: Thu May 4 16:30:25 2023 +0300 [SPARK-42843][SQL][FOLLOWUP] Remove the old error class _LEGACY_ERROR_TEMP_2007 ### What changes were proposed in this pull request? Remove the old error class _LEGACY_ERROR_TEMP_2007 since new one is already created. ### Why are the changes needed? To fix JIRA issue 42843 completely. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Current tests covered it. Closes #41045 from liang3zy22/spark42843. Authored-by: Liang Yan Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - 1 file changed, 5 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 1d5ee78aa8c..753908932c8 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3752,11 +3752,6 @@ "The specified group index cannot be less than zero." ] }, - "_LEGACY_ERROR_TEMP_2007" : { -"message" : [ - "Regex group count is , but the specified group index is ." -] - }, "_LEGACY_ERROR_TEMP_2011" : { "message" : [ "Unexpected data type ." - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42843][SQL] Update the error class _LEGACY_ERROR_TEMP_2007 to REGEX_GROUP_INDEX
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 e43db6cc887 [SPARK-42843][SQL] Update the error class _LEGACY_ERROR_TEMP_2007 to REGEX_GROUP_INDEX e43db6cc887 is described below commit e43db6cc88795547242d005259fdb9b6d992a774 Author: Liang Yan AuthorDate: Thu May 4 09:54:12 2023 +0300 [SPARK-42843][SQL] Update the error class _LEGACY_ERROR_TEMP_2007 to REGEX_GROUP_INDEX ### What changes were proposed in this pull request? Update the error class _LEGACY_ERROR_TEMP_2007 to REGEX_GROUP_INDEX. ### Why are the changes needed? Fix jira issue [SPARK-42843](https://issues.apache.org/jira/browse/SPARK-42843). The original name just a number, update it to an informal name. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Current tests already covered it. Closes #40955 from liang3zy22/spark42843. Authored-by: Liang Yan Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 ++ .../catalyst/expressions/regexpExpressions.scala | 15 +++-- .../spark/sql/errors/QueryExecutionErrors.scala| 10 ++- .../expressions/RegexpExpressionsSuite.scala | 72 ++ .../sql-tests/results/regexp-functions.sql.out | 72 ++ 5 files changed, 129 insertions(+), 45 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7718b8b9127..1d5ee78aa8c 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1009,6 +1009,11 @@ "." ] }, + "REGEX_GROUP_INDEX" : { +"message" : [ + "Expects group index between 0 and , but got ." +] + }, "ZERO_INDEX" : { "message" : [ "expects %1$, %2$ and so on, but got %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 29510bc3852..2025a554998 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 @@ -746,12 +746,12 @@ object RegExpReplace { } object RegExpExtractBase { - def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = { + def checkGroupIndex(prettyName: String, groupCount: Int, groupIndex: Int): Unit = { if (groupIndex < 0) { throw QueryExecutionErrors.regexGroupIndexLessThanZeroError } else if (groupCount < groupIndex) { throw QueryExecutionErrors.regexGroupIndexExceedGroupCountError( -groupCount, groupIndex) +prettyName, groupCount, groupIndex) } } } @@ -857,7 +857,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio if (m.find) { val mr: MatchResult = m.toMatchResult val index = r.asInstanceOf[Int] - RegExpExtractBase.checkGroupIndex(mr.groupCount, index) + RegExpExtractBase.checkGroupIndex(prettyName, mr.groupCount, index) val group = mr.group(index) if (group == null) { // Pattern matched, but it's an optional group UTF8String.EMPTY_UTF8 @@ -887,7 +887,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio ${initLastMatcherCode(ctx, subject, regexp, matcher)} if ($matcher.find()) { java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); -$classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx); +$classNameRegExpExtractBase.checkGroupIndex("$prettyName", $matchResult.groupCount(), $idx); if ($matchResult.group($idx) == null) { ${ev.value} = UTF8String.EMPTY_UTF8; } else { @@ -950,7 +950,7 @@ case class RegExpExtractAll(subject: Expression, regexp: Expression, idx: Expres while(m.find) { val mr: MatchResult = m.toMatchResult val index = r.asInstanceOf[Int] - RegExpExtractBase.checkGroupIndex(mr.groupCount, index) + RegExpExtractBase.checkGroupIndex(prettyName, mr.groupCount, index) val group = mr.group(index) if (group == null) { // Pattern matched, but it's an optional group matchResults += UTF8String.EMPTY_UTF8 @@ -982,7 +982,10 @@ case class RegExpExtractAll(subject: Expression, regexp: Expression, idx: Expres | java.util.ArrayList $matchResults = new java.util.ArrayList(); | while ($matcher.find
[spark] branch master updated (39813c9dcf5 -> 49fc992f02a)
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 39813c9dcf5 [SPARK-43352][K8S][TEST] Inline `DepsTestsSuite#setPythonSparkConfProperties` add 49fc992f02a [SPARK-43250][SQL] Replace the error class `_LEGACY_ERROR_TEMP_2014` with an internal error No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 5 - .../scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala | 8 2 files changed, 4 insertions(+), 9 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (c609ae7638d -> 00108470daa)
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 c609ae7638d [MINOR][CONNECT][DOC] Add information on how to regenerate proto for python client add 00108470daa [SPARK-42845][SQL] Update the error class _LEGACY_ERROR_TEMP_2010 to InternalError No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 5 - .../spark/sql/catalyst/expressions/windowExpressions.scala | 2 +- .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 7 +++ .../spark/sql/errors/QueryExecutionErrorsSuite.scala | 14 +- 4 files changed, 17 insertions(+), 11 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (429176b419c -> c609ae7638d)
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 429176b419c [SPARK-16484][SQL] Use 8-bit registers for representing DataSketches add c609ae7638d [MINOR][CONNECT][DOC] Add information on how to regenerate proto for python client No new revisions were added by this update. Summary of changes: connector/connect/README.md | 22 ++ 1 file changed, 22 insertions(+) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43257][SQL] Replace the error class _LEGACY_ERROR_TEMP_2022 by an internal error
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 4a238cd9d8e [SPARK-43257][SQL] Replace the error class _LEGACY_ERROR_TEMP_2022 by an internal error 4a238cd9d8e is described below commit 4a238cd9d8e80eed06732fc52b1456cb5ece6652 Author: 庞贝 AuthorDate: Thu Apr 27 15:01:37 2023 +0300 [SPARK-43257][SQL] Replace the error class _LEGACY_ERROR_TEMP_2022 by an internal error ### What changes were proposed in this pull request? Change the error class_LEGACY_ERROR_TEMP_2022 to internal error as it cannot be accessed. ### Why are the changes needed? Fix jira issue [SPARK-43257](https://issues.apache.org/jira/browse/SPARK-43257). `ResolveNaturalAndUsingJoin` only handles `UsingJoin` and `NaturalJoin`, and all join types supported by `UsingJoin` and `NaturalJoin` are supported by natural join, so `commonNaturalJoinProcessing` will never go to the default branch and throw `unsupportedNaturalJoinTypeError`. I think this error class could be replaced with internal error. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Already exist tests. Closes #40957 from JinHelin404/spark-43257. Authored-by: 庞贝 Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala | 7 +++ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 370508b70a8..38502171a0a 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3796,11 +3796,6 @@ "Couldn't find a primary constructor on ." ] }, - "_LEGACY_ERROR_TEMP_2022" : { -"message" : [ - "Unsupported natural join type ." -] - }, "_LEGACY_ERROR_TEMP_2023" : { "message" : [ "Unresolved encoder expected, but was found." 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 6c4066e638c..e77f575b89e 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 @@ -469,10 +469,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("cls" -> cls.toString())) } - def unsupportedNaturalJoinTypeError(joinType: JoinType): SparkRuntimeException = { -new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2022", - messageParameters = Map("joinType" -> joinType.toString())) + def unsupportedNaturalJoinTypeError(joinType: JoinType): SparkException = { +SparkException.internalError( + s"Unsupported natural join type ${joinType.toString}") } def notExpectedUnresolvedEncoderError(attr: AttributeReference): SparkRuntimeException = { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42317][SQL] Assign name to _LEGACY_ERROR_TEMP_2247: CANNOT_MERGE_SCHEMAS
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 69946bb5c70 [SPARK-42317][SQL] Assign name to _LEGACY_ERROR_TEMP_2247: CANNOT_MERGE_SCHEMAS 69946bb5c70 is described below commit 69946bb5c707657bf0840b21356fbe95b8524ab9 Author: Koray Beyaz AuthorDate: Mon Apr 24 11:30:11 2023 +0300 [SPARK-42317][SQL] Assign name to _LEGACY_ERROR_TEMP_2247: CANNOT_MERGE_SCHEMAS ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2247 as "CANNOT_MERGE_SCHEMAS". Also proposes to display both left and right schemas in the exception so that one can compare them. Please let me know if you prefer the old error message with a single schema. This is the stack trace after the changes: ``` scala> spark.read.option("mergeSchema", "true").parquet(path) org.apache.spark.SparkException: [CANNOT_MERGE_SCHEMAS] Failed merging schemas: Initial schema: "STRUCT" Schema that cannot be merged with the initial schema: "STRUCT". at org.apache.spark.sql.errors.QueryExecutionErrors$.failedMergingSchemaError(QueryExecutionErrors.scala:2355) at org.apache.spark.sql.execution.datasources.SchemaMergeUtils$.$anonfun$mergeSchemasInParallel$5(SchemaMergeUtils.scala:104) at org.apache.spark.sql.execution.datasources.SchemaMergeUtils$.$anonfun$mergeSchemasInParallel$5$adapted(SchemaMergeUtils.scala:100) at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198) at org.apache.spark.sql.execution.datasources.SchemaMergeUtils$.mergeSchemasInParallel(SchemaMergeUtils.scala:100) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$.mergeSchemasInParallel(ParquetFileFormat.scala:496) at org.apache.spark.sql.execution.datasources.parquet.ParquetUtils$.inferSchema(ParquetUtils.scala:132) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.inferSchema(ParquetFileFormat.scala:78) at org.apache.spark.sql.execution.datasources.DataSource.$anonfun$getOrInferFileFormatSchema$11(DataSource.scala:208) at scala.Option.orElse(Option.scala:447) at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:205) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:407) at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:229) at org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:211) at scala.Option.getOrElse(Option.scala:189) at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:211) at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:563) at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:548) ... 49 elided Caused by: org.apache.spark.SparkException: [CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE] Failed to merge incompatible data types "BIGINT" and "INT". at org.apache.spark.sql.errors.QueryExecutionErrors$.cannotMergeIncompatibleDataTypesError(QueryExecutionErrors.scala:1326) at org.apache.spark.sql.types.StructType$.$anonfun$merge$3(StructType.scala:610) at scala.Option.map(Option.scala:230) at org.apache.spark.sql.types.StructType$.$anonfun$merge$2(StructType.scala:602) at org.apache.spark.sql.types.StructType$.$anonfun$merge$2$adapted(StructType.scala:599) at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36) at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198) at org.apache.spark.sql.types.StructType$.$anonfun$merge$1(StructType.scala:599) at org.apache.spark.sql.types.StructType$.mergeInternal(StructType.scala:647) at org.apache.spark.sql.types.StructType$.merge(StructType.scala:593) at org.apache.spark.sql.types.StructType.merge(StructType.scala:498) at org.apache.spark.sql.execution.datasources.SchemaMergeUtils$.$anonfun$mergeSchemasInParallel$5(SchemaMergeUtils.scala:102) ... 67 more ``` ### Why are the changes needed? We should assign proper name to LEGACY_ERROR_TEMP* ### Does this PR introduce _any_ user-facing change? Yes, the users will see an improved error message. ### How was this patch tested? Changed an existing test case to test the new error class with `checkError` utility.
[spark] branch master updated: [SPARK-43038][SQL] Support the CBC mode by `aes_encrypt()`/`aes_decrypt()`
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 dabd771c37b [SPARK-43038][SQL] Support the CBC mode by `aes_encrypt()`/`aes_decrypt()` dabd771c37b is described below commit dabd771c37be9cbd773b5223d8c78226ece84f8a Author: Max Gekk AuthorDate: Wed Apr 12 16:02:29 2023 +0300 [SPARK-43038][SQL] Support the CBC mode by `aes_encrypt()`/`aes_decrypt()` ### What changes were proposed in this pull request? In the PR, I propose new AES mode for the `aes_encrypt()`/`aes_decrypt()` functions - `CBC` ([Cipher Block Chaining](https://www.ibm.com/docs/en/linux-on-systems?topic=operation-cipher-block-chaining-cbc-mode)) with the padding `PKCS7(5)`. The `aes_encrypt()` function returns a binary value which consists of the following fields: 1. The salt magic prefix `Salted__` with the length of 8 bytes. 2. A salt generated per every `aes_encrypt()` call using `java.security.SecureRandom`. Its length is 8 bytes. 3. The encrypted input. The encrypt function derives the secret key and initialization vector (16 bytes) from the salt and user's key using the same algorithm as OpenSSL's `EVP_BytesToKey()` (versions >= 1.1.0c). The `aes_decrypt()` functions assumes that its input has the fields as showed above. For example: ```sql spark-sql> SELECT base64(aes_encrypt('Apache Spark', '', 'CBC', 'PKCS')); U2FsdGVkX1/ERGxwEOTDpDD4bQvDtQaNe+gXGudCcUk= spark-sql> SELECT aes_decrypt(unbase64('U2FsdGVkX1/ERGxwEOTDpDD4bQvDtQaNe+gXGudCcUk='), '', 'CBC', 'PKCS'); Apache Spark ``` ### Why are the changes needed? To achieve feature parity with other systems/frameworks, and make the migration process from them to Spark SQL easier. For example, the `CBC` mode is supported by: - BigQuery: https://cloud.google.com/bigquery/docs/reference/standard-sql/aead-encryption-concepts#block_cipher_modes - Snowflake: https://docs.snowflake.com/en/sql-reference/functions/encrypt.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running new checks: ``` $ build/sbt "sql/testOnly *QueryExecutionErrorsSuite" $ build/sbt "sql/test:testOnly org.apache.spark.sql.expressions.ExpressionInfoSuite" $ build/sbt "test:testOnly org.apache.spark.sql.MiscFunctionsSuite" $ build/sbt "core/testOnly *SparkThrowableSuite" ``` and checked compatibility with LibreSSL/OpenSSL: ``` $ openssl version LibreSSL 3.3.6 $ echo -n 'Apache Spark' | openssl enc -e -aes-128-cbc -pass pass: -a U2FsdGVkX1+5GyAmmG7wDWWDBAuUuxjMy++cMFytpls= ``` ```sql spark-sql (default)> SELECT aes_decrypt(unbase64('U2FsdGVkX1+5GyAmmG7wDWWDBAuUuxjMy++cMFytpls='), '', 'CBC'); Apache Spark ``` decrypt Spark's output by OpenSSL: ```sql spark-sql (default)> SELECT base64(aes_encrypt('Apache Spark', 'abcdefghijklmnop12345678ABCDEFGH', 'CBC', 'PKCS')); U2FsdGVkX1+maU2vmxrulgxXuQSyZ3ODnlHKqnt2fDA= ``` ``` $ echo 'U2FsdGVkX1+maU2vmxrulgxXuQSyZ3ODnlHKqnt2fDA=' | openssl aes-256-cbc -a -d -pass pass:abcdefghijklmnop12345678ABCDEFGH Apache Spark ``` Closes #40704 from MaxGekk/aes-cbc. Authored-by: Max Gekk Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 ++ .../catalyst/expressions/ExpressionImplUtils.java | 72 ++ .../spark/sql/catalyst/expressions/misc.scala | 16 +++-- .../spark/sql/errors/QueryExecutionErrors.scala| 9 +++ .../org/apache/spark/sql/MiscFunctionsSuite.scala | 33 +- .../sql/errors/QueryExecutionErrorsSuite.scala | 31 -- 6 files changed, 141 insertions(+), 25 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ae73071a120..1edf625fdc3 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -978,6 +978,11 @@ "expects a binary value with 16, 24 or 32 bytes, but got bytes." ] }, + "AES_SALTED_MAGIC" : { +"message" : [ + "Initial bytes from input do not match 'Salted__' (0x53616C7465645F5F)." +] + }, "PATTERN" : { "message" : [ "." diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java index a6e482db57b..680ad11ad73 100644 --- a/sq
[spark] branch master updated: [SPARK-43028][SQL] Add error class SQL_CONF_NOT_FOUND
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 d4134a83088 [SPARK-43028][SQL] Add error class SQL_CONF_NOT_FOUND d4134a83088 is described below commit d4134a8308889c3b8f87c93a23f91aa070e97b21 Author: allisonwang-db AuthorDate: Tue Apr 11 13:43:16 2023 +0300 [SPARK-43028][SQL] Add error class SQL_CONF_NOT_FOUND ### What changes were proposed in this pull request? This PR adds a new error class `SQL_CONF_NOT_FOUND`. ### Why are the changes needed? To make the error message more user-friendly when getting a non-existing SQL config. For example: ``` spark.conf.get("some.conf") ``` Before this PR, it will throw this error: ``` java.util.NoSuchElementException: some.conf ``` After this PR: ``` [SQL_CONF_NOT_FOUND] The SQL config "some.conf" cannot be found. Please verify that the config exists. ``` ### Does this PR introduce _any_ user-facing change? Yes. The error message will be changed. ### How was this patch tested? Added a new UT. Closes #40660 from allisonwang-db/SPARK-43028-conf-error. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- R/pkg/R/SQLContext.R | 2 +- core/src/main/resources/error/error-classes.json | 5 + .../main/scala/org/apache/spark/SparkException.scala | 19 +++ .../spark/sql/errors/QueryExecutionErrors.scala | 6 -- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- .../org/apache/spark/sql/internal/SQLConfSuite.scala | 9 - 6 files changed, 39 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 5adebade8b7..5f500c29958 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -111,7 +111,7 @@ sparkR.conf <- function(key, defaultValue) { tryCatch(callJMethod(conf, "get", key), error = function(e) { estr <- as.character(e) -if (any(grepl("java.util.NoSuchElementException", estr, fixed = TRUE))) { +if (any(grepl("SQL_CONF_NOT_FOUND", estr, fixed = TRUE))) { stop("Config '", key, "' is not set") } else { stop("Unknown error: ", estr) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 54cffa498cc..ae73071a120 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1425,6 +1425,11 @@ "sortBy must be used together with bucketBy." ] }, + "SQL_CONF_NOT_FOUND" : { +"message" : [ + "The SQL config cannot be found. Please verify that the config exists." +] + }, "STAR_GROUP_BY_POS" : { "message" : [ "Star (*) is not allowed in a select list when GROUP BY an ordinal position is used." diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index 2f05b2ad6a7..4e48e9c8d41 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -293,6 +293,25 @@ private[spark] class SparkRuntimeException( override def getQueryContext: Array[QueryContext] = context } +/** + * No such element exception thrown from Spark with an error class. + */ +private[spark] class SparkNoSuchElementException( +errorClass: String, +messageParameters: Map[String, String], +context: Array[QueryContext] = Array.empty, +summary: String = "") +extends NoSuchElementException( + SparkThrowableHelper.getMessage(errorClass, messageParameters, summary)) +with SparkThrowable { + + override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava + + override def getErrorClass: String = errorClass + + override def getQueryContext: Array[QueryContext] = context +} + /** * Security exception thrown from Spark with an error class. */ 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 2fccca839d5..d07dcec3693 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 @@ -2139,8 +2139,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { cause = null) } - def noSuchEleme
[spark] branch master updated: [SPARK-42840][SQL] Change `_LEGACY_ERROR_TEMP_2004` error to internal error
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 56043c24a55 [SPARK-42840][SQL] Change `_LEGACY_ERROR_TEMP_2004` error to internal error 56043c24a55 is described below commit 56043c24a55663e692c3671ba263e642aebc1a50 Author: Leibniz.Hu AuthorDate: Thu Apr 6 15:53:08 2023 +0300 [SPARK-42840][SQL] Change `_LEGACY_ERROR_TEMP_2004` error to internal error ### What changes were proposed in this pull request? In the PR, I propose to change the `_LEGACY_ERROR_TEMP_2004` error to an internal error. Also this PR improves the error message. ### Why are the changes needed? `_LEGACY_ERROR_TEMP_2004` cannot be triggered from user code (for instance, some SQL query), more detail: 1. `_LEGACY_ERROR_TEMP_2004` error is thrown in `Literal.default` for `CharType`, `VarcharType` and other user-implemented DataType. 2. `Literal.default` is called in these cases below: 1. in `org.apache.spark.sql.catalyst.expressions.aggregate.Average`, for getting initial value; but in this case, DataType would only be DecimalType / YearMonthIntervalType / DayTimeIntervalType / DoubleType; these types are supported by `Literal.default`. 2. in `org.apache.spark.sql.catalyst.expressions.aggregate.Sum`, is same as `Average` 3. in `org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys` and `org.apache.spark.sql.catalyst.plans.logical.AsOfJoin`; In real scene, they are use for DateFrame api, but `CharType` and `VarcharType` cannot be as a part of DataFrame's schema( they would be converted to StringType); and user-defined DataType do not have a matched Encoder in `org.apache.spark.sql.catalyst.encoders.RowEncoder#encoderForDataType` so, it should be an internal error, but not `SparkRuntimeException`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites: ```bash build/sbt "sql/testOnly org.apache.spark.sql.catalyst.expressions.LiteralExpressionSuite" ``` Closes #40634 from Leibnizhu/SPARK-42840. Lead-authored-by: Leibniz.Hu Co-authored-by: Leibniz.Hu Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 6 ++ .../sql/catalyst/expressions/LiteralExpressionSuite.scala | 14 +- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 79c6f3e6d82..7014ce36b1d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3706,11 +3706,6 @@ "Unsuccessful try to zip maps with unique keys due to exceeding the array size limit ." ] }, - "_LEGACY_ERROR_TEMP_2004" : { -"message" : [ - "no default for type ." -] - }, "_LEGACY_ERROR_TEMP_2005" : { "message" : [ "Type does not support ordered operations." 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 7ae9d55f96c..7ec5f588754 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 @@ -339,10 +339,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "type" -> toSQLType(dataType))) } - def noDefaultForDataTypeError(dataType: DataType): SparkRuntimeException = { -new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2004", - messageParameters = Map("dataType" -> dataType.toString())) + def noDefaultForDataTypeError(dataType: DataType): SparkException = { +SparkException.internalError(s"No default value for type: ${toSQLType(dataType)}.") } def orderedOperationUnsupportedByDataTypeError( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 80e7a3206aa..90438d81661 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -24,12 +24,13 @@ import java.util.TimeZone import scala.reflect.runtime.universe.TypeTag -import or
[spark] branch master updated: [SPARK-42844][SQL] Update the error class `_LEGACY_ERROR_TEMP_2008` to `INVALID_URL`
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 f541301b768 [SPARK-42844][SQL] Update the error class `_LEGACY_ERROR_TEMP_2008` to `INVALID_URL` f541301b768 is described below commit f541301b7680d96611796d92943d4ec72c71ec0d Author: Liang Yan AuthorDate: Thu Apr 6 10:10:34 2023 +0300 [SPARK-42844][SQL] Update the error class `_LEGACY_ERROR_TEMP_2008` to `INVALID_URL` ### What changes were proposed in this pull request? Update the error_class _LEGACY_ERROR_TEMP_2008 to INVALID_URL. ### Why are the changes needed? Fix jira issue SPARK-42844. The original name just a number, update it to a informal name. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add a test case in UrlFunctionsSuite to catch the error using sql command. Closes #40657 from liang3zy22/spark42844. Authored-by: Liang Yan Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 10 +- .../apache/spark/sql/errors/QueryExecutionErrors.scala| 2 +- .../sql/catalyst/expressions/StringExpressionsSuite.scala | 10 ++ .../scala/org/apache/spark/sql/UrlFunctionsSuite.scala| 15 +++ 4 files changed, 23 insertions(+), 14 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index d43996f6a49..79c6f3e6d82 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1068,6 +1068,11 @@ ], "sqlState" : "42604" }, + "INVALID_URL" : { +"message" : [ + "The url is invalid: . If necessary set to \"false\" to bypass this error." +] + }, "INVALID_WHERE_CONDITION" : { "message" : [ "The WHERE condition contains invalid expressions: .", @@ -3721,11 +3726,6 @@ "Regex group count is , but the specified group index is ." ] }, - "_LEGACY_ERROR_TEMP_2008" : { -"message" : [ - "Find an invalid url string . If necessary set to false to bypass this error." -] - }, "_LEGACY_ERROR_TEMP_2010" : { "message" : [ "Window Functions do not support merging." 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 ad0796e4eb1..7ae9d55f96c 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 @@ -369,7 +369,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def invalidUrlError(url: UTF8String, e: URISyntaxException): SparkIllegalArgumentException = { new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_2008", + errorClass = "INVALID_URL", messageParameters = Map( "url" -> url.toString, "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 017f4483e88..9c97508b7a8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -1832,16 +1832,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-33468: ParseUrl in ANSI mode should fail if input string is not a valid url") { -withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { - val msg = intercept[IllegalArgumentException] { -evaluateWithoutCodegen( - ParseUrl(Seq("https://a.b.c/index.php?params1=a|b=x", "HOST"))) - }.getMessage - assert(msg.contains("Find an invalid url string")) -} +val url = "https://a.b.c/index.php?params1=a|b=x" withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation( -ParseUrl(Seq("https://a.b.c/index.php?params1=a|b=x", "HOST")), null) +ParseUrl(Seq(url, "HOST")), null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UrlFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UrlFunct
[spark] branch master updated: [SPARK-42316][SQL] Assign name to _LEGACY_ERROR_TEMP_2044
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 45a51d6311a [SPARK-42316][SQL] Assign name to _LEGACY_ERROR_TEMP_2044 45a51d6311a is described below commit 45a51d6311a041eb7480199a6da19f67d56367b7 Author: Hisoka AuthorDate: Wed Apr 5 10:57:58 2023 +0300 [SPARK-42316][SQL] Assign name to _LEGACY_ERROR_TEMP_2044 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2044, "BINARY_ARITHMETIC_OVERFLOW". ### Why are the changes needed? Assign proper name to LEGACY_ERROR_TEMP ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ./build/sbt "testOnly org.apache.spark.sql.errors.QueryExecutionErrorsSuite" Closes #40609 from Hisoka-X/legacy_error_2044_. Authored-by: Hisoka Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../apache/spark/sql/errors/QueryExecutionErrors.scala| 6 +++--- .../spark/sql/errors/QueryExecutionErrorsSuite.scala | 15 +++ 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index d330ea09f30..d43996f6a49 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -35,6 +35,12 @@ ], "sqlState" : "22003" }, + "BINARY_ARITHMETIC_OVERFLOW" : { +"message" : [ + " caused overflow." +], +"sqlState" : "22003" + }, "CANNOT_CAST_DATATYPE" : { "message" : [ "Cannot cast to ." @@ -3880,11 +3886,6 @@ "- caused overflow." ] }, - "_LEGACY_ERROR_TEMP_2044" : { -"message" : [ - " caused overflow." -] - }, "_LEGACY_ERROR_TEMP_2045" : { "message" : [ "Unsupported table change: " 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 7ec9f41af36..ad0796e4eb1 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 @@ -654,11 +654,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def binaryArithmeticCauseOverflowError( eval1: Short, symbol: String, eval2: Short): SparkArithmeticException = { new SparkArithmeticException( - errorClass = "_LEGACY_ERROR_TEMP_2044", + errorClass = "BINARY_ARITHMETIC_OVERFLOW", messageParameters = Map( -"sqlValue1" -> toSQLValue(eval1, ShortType), +"value1" -> toSQLValue(eval1, ShortType), "symbol" -> symbol, -"sqlValue2" -> toSQLValue(eval2, ShortType)), +"value2" -> toSQLValue(eval2, ShortType)), context = Array.empty, summary = "") } 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 5f94840a16a..6cdbbc4c1a3 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 @@ -625,6 +625,21 @@ class QueryExecutionErrorsSuite } } + test("BINARY_ARITHMETIC_OVERFLOW: byte plus byte result overflow") { +withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + checkError( +exception = intercept[SparkArithmeticException] { + sql(s"select 127Y + 5Y").collect() +}, +errorClass = "BINARY_ARITHMETIC_OVERFLOW", +parameters = Map( + "value1" -> "127S", + "symbol" -> "+", + "value2" -> "5S"), +sqlState = "22003") +} + } + test("UNSUPPORTED_DATATYPE: invalid StructType raw format") { checkError( exception = intercept[SparkIllegalArgumentException] { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-43011][SQL] `array_insert` should fail with 0 index
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 9b1f2db48fa [SPARK-43011][SQL] `array_insert` should fail with 0 index 9b1f2db48fa is described below commit 9b1f2db48facb2b79b54396675748c1c70f01a85 Author: Ruifeng Zheng AuthorDate: Tue Apr 4 10:22:16 2023 +0300 [SPARK-43011][SQL] `array_insert` should fail with 0 index ### What changes were proposed in this pull request? Make `array_insert` fail when input index `pos` is zero. ### Why are the changes needed? see https://github.com/apache/spark/pull/40563#discussion_r1155673089 ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? updated UT Closes #40641 from zhengruifeng/sql_array_insert_fails_zero. Authored-by: Ruifeng Zheng Signed-off-by: Max Gekk (cherry picked from commit 3e9574c54f149b13ca768c0930c634eb67ea14c8) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 12 ++-- docs/sql-error-conditions-sqlstates.md | 2 +- docs/sql-error-conditions.md | 12 ++-- .../catalyst/expressions/collectionOperations.scala | 18 ++ .../spark/sql/errors/QueryExecutionErrors.scala | 4 ++-- .../expressions/CollectionExpressionsSuite.scala | 1 - .../resources/sql-tests/results/ansi/array.sql.out | 17 ++--- .../sql-tests/results/ansi/string-functions.sql.out | 2 +- .../sql-tests/results/ansi/try_element_at.sql.out| 2 +- .../test/resources/sql-tests/results/array.sql.out | 17 ++--- .../sql-tests/results/string-functions.sql.out | 2 +- .../sql-tests/results/try_element_at.sql.out | 2 +- .../apache/spark/sql/DataFrameFunctionsSuite.scala | 20 +++- .../sql/errors/QueryExecutionAnsiErrorsSuite.scala | 4 ++-- 14 files changed, 78 insertions(+), 37 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7d16365c677..5622a12e19e 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -524,12 +524,6 @@ ], "sqlState" : "23505" }, - "ELEMENT_AT_BY_INDEX_ZERO" : { -"message" : [ - "The index 0 is invalid. An index shall be either < 0 or > 0 (the first element has index 1)." -], -"sqlState" : "22003" - }, "EMPTY_JSON_FIELD_VALUE" : { "message" : [ "Failed to parse an empty string for data type ." @@ -903,6 +897,12 @@ ], "sqlState" : "42602" }, + "INVALID_INDEX_OF_ZERO" : { +"message" : [ + "The index 0 is invalid. An index shall be either < 0 or > 0 (the first element has index 1)." +], +"sqlState" : "22003" + }, "INVALID_JSON_ROOT_FIELD" : { "message" : [ "Cannot convert JSON root field to target Spark type." diff --git a/docs/sql-error-conditions-sqlstates.md b/docs/sql-error-conditions-sqlstates.md index ed6dace4c65..51175125c3f 100644 --- a/docs/sql-error-conditions-sqlstates.md +++ b/docs/sql-error-conditions-sqlstates.md @@ -71,7 +71,7 @@ Spark SQL uses the following `SQLSTATE` classes: - ARITHMETIC_OVERFLOW, CAST_OVERFLOW, CAST_OVERFLOW_IN_TABLE_INSERT, DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION, ELEMENT_AT_BY_INDEX_ZERO, ARITHMETIC_OVERFLOW, CAST_OVERFLOW, CAST_OVERFLOW_IN_TABLE_INSERT, DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION, INVALID_INDEX_OF_ZERO, baseArr.numElements()) @@ -4748,6 +4751,7 @@ case class ArrayInsert(srcArrayExpr: Expression, posExpr: Expression, itemExpr: values, elementType, resLength, s"$prettyName failed.") val assignment = CodeGenerator.createArrayAssignment(values, elementType, arr, adjustedAllocIdx, i, first.dataType.asInstanceOf[ArrayType].containsNull) + val errorContext = getContextOrNullCode(ctx) s""" |int $itemInsertionIndex = 0; @@ -4755,6 +4759,10 @@ case class ArrayInsert(srcArrayExpr: Expression, posExpr: Expression, itemExpr: |int $adjustedAllocIdx = 0; |boolean $insertedItemIsNull = ${itemExpr.isNull}; | + |if ($pos == 0) { + | throw QueryExecutionErrors.invalidIndexOfZeroError($errorContext); + |} + | |if ($pos < 0 && (java.lang.Math.abs($pos) > $arr.numElements())) { | | $resLength = java.lang.Math.abs($pos) + 1; @@ -4855,6 +4863,8 @@ case class ArrayInsert(srcArray
[spark] branch master updated: [SPARK-43011][SQL] `array_insert` should fail with 0 index
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 3e9574c54f1 [SPARK-43011][SQL] `array_insert` should fail with 0 index 3e9574c54f1 is described below commit 3e9574c54f149b13ca768c0930c634eb67ea14c8 Author: Ruifeng Zheng AuthorDate: Tue Apr 4 10:22:16 2023 +0300 [SPARK-43011][SQL] `array_insert` should fail with 0 index ### What changes were proposed in this pull request? Make `array_insert` fail when input index `pos` is zero. ### Why are the changes needed? see https://github.com/apache/spark/pull/40563#discussion_r1155673089 ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? updated UT Closes #40641 from zhengruifeng/sql_array_insert_fails_zero. Authored-by: Ruifeng Zheng Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 12 ++-- docs/sql-error-conditions-sqlstates.md | 2 +- docs/sql-error-conditions.md | 12 ++-- .../catalyst/expressions/collectionOperations.scala | 18 ++ .../spark/sql/errors/QueryExecutionErrors.scala | 4 ++-- .../expressions/CollectionExpressionsSuite.scala | 1 - .../resources/sql-tests/results/ansi/array.sql.out | 17 ++--- .../sql-tests/results/ansi/string-functions.sql.out | 2 +- .../sql-tests/results/ansi/try_element_at.sql.out| 2 +- .../test/resources/sql-tests/results/array.sql.out | 17 ++--- .../sql-tests/results/string-functions.sql.out | 2 +- .../sql-tests/results/try_element_at.sql.out | 2 +- .../apache/spark/sql/DataFrameFunctionsSuite.scala | 20 +++- .../sql/errors/QueryExecutionAnsiErrorsSuite.scala | 4 ++-- 14 files changed, 78 insertions(+), 37 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8369c7c5666..d330ea09f30 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -536,12 +536,6 @@ ], "sqlState" : "23505" }, - "ELEMENT_AT_BY_INDEX_ZERO" : { -"message" : [ - "The index 0 is invalid. An index shall be either < 0 or > 0 (the first element has index 1)." -], -"sqlState" : "22003" - }, "EMPTY_JSON_FIELD_VALUE" : { "message" : [ "Failed to parse an empty string for data type ." @@ -915,6 +909,12 @@ ], "sqlState" : "42602" }, + "INVALID_INDEX_OF_ZERO" : { +"message" : [ + "The index 0 is invalid. An index shall be either < 0 or > 0 (the first element has index 1)." +], +"sqlState" : "22003" + }, "INVALID_JSON_ROOT_FIELD" : { "message" : [ "Cannot convert JSON root field to target Spark type." diff --git a/docs/sql-error-conditions-sqlstates.md b/docs/sql-error-conditions-sqlstates.md index 1eea335ac9b..6b4c7e62f71 100644 --- a/docs/sql-error-conditions-sqlstates.md +++ b/docs/sql-error-conditions-sqlstates.md @@ -71,7 +71,7 @@ Spark SQL uses the following `SQLSTATE` classes: - ARITHMETIC_OVERFLOW, CAST_OVERFLOW, CAST_OVERFLOW_IN_TABLE_INSERT, DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION, ELEMENT_AT_BY_INDEX_ZERO, ARITHMETIC_OVERFLOW, CAST_OVERFLOW, CAST_OVERFLOW_IN_TABLE_INSERT, DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION, INVALID_INDEX_OF_ZERO, baseArr.numElements()) @@ -4895,6 +4898,7 @@ case class ArrayInsert(srcArrayExpr: Expression, posExpr: Expression, itemExpr: values, elementType, resLength, s"$prettyName failed.") val assignment = CodeGenerator.createArrayAssignment(values, elementType, arr, adjustedAllocIdx, i, first.dataType.asInstanceOf[ArrayType].containsNull) + val errorContext = getContextOrNullCode(ctx) s""" |int $itemInsertionIndex = 0; @@ -4902,6 +4906,10 @@ case class ArrayInsert(srcArrayExpr: Expression, posExpr: Expression, itemExpr: |int $adjustedAllocIdx = 0; |boolean $insertedItemIsNull = ${itemExpr.isNull}; | + |if ($pos == 0) { + | throw QueryExecutionErrors.invalidIndexOfZeroError($errorContext); + |} + | |if ($pos < 0 && (java.lang.Math.abs($pos) > $arr.numElements())) { | | $resLength = java.lang.Math.abs($pos) + 1; @@ -5002,6 +5010,8 @@ case class ArrayInsert(srcArrayExpr: Expression, posExpr: Expression, itemExpr: override protected def withNewChildrenInternal( n
[spark] branch branch-3.4 updated: [MINOR][DOCS] Add Java 8 types to value types of Scala/Java APIs
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 47b29128e36 [MINOR][DOCS] Add Java 8 types to value types of Scala/Java APIs 47b29128e36 is described below commit 47b29128e3694203844b98b8f4d23201b0a1cdfb Author: Max Gekk AuthorDate: Mon Apr 3 17:09:37 2023 +0300 [MINOR][DOCS] Add Java 8 types to value types of Scala/Java APIs ### What changes were proposed in this pull request? In the PR, I propose to update the doc page https://spark.apache.org/docs/latest/sql-ref-datatypes.html about value types in Scala and Java APIs. https://user-images.githubusercontent.com/1580697/229509678-7f860c2b-050e-4e40-b83d-821e70d1e194.png;> ### Why are the changes needed? To provide full info about supported "external" value types in Scala/Java APIs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By building docs and checking them by eyes. Closes #40644 from MaxGekk/datatypes-java8-docs. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit d3338c6879fd3e0b986654889f2e1e6407988dcc) Signed-off-by: Max Gekk --- docs/sql-ref-datatypes.md | 8 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 64c6d335429..d2a8ad61bf5 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -127,9 +127,9 @@ You can access them by doing |**StringType**|String|StringType| |**BinaryType**|Array[Byte]|BinaryType| |**BooleanType**|Boolean|BooleanType| -|**TimestampType**|java.sql.Timestamp|TimestampType| +|**TimestampType**|java.time.Instant or java.sql.Timestamp|TimestampType| |**TimestampNTZType**|java.time.LocalDateTime|TimestampNTZType| -|**DateType**|java.sql.Date|DateType| +|**DateType**|java.time.LocalDate or java.sql.Date|DateType| |**YearMonthIntervalType**|java.time.Period|YearMonthIntervalType| |**DayTimeIntervalType**|java.time.Duration|DayTimeIntervalType| |**ArrayType**|scala.collection.Seq|ArrayType(*elementType*, [*containsNull]*)**Note:** The default value of *containsNull* is true.| @@ -158,9 +158,9 @@ please use factory methods provided in |**StringType**|String|DataTypes.StringType| |**BinaryType**|byte[]|DataTypes.BinaryType| |**BooleanType**|boolean or Boolean|DataTypes.BooleanType| -|**TimestampType**|java.sql.Timestamp|DataTypes.TimestampType| +|**TimestampType**|java.time.Instant or java.sql.Timestamp|DataTypes.TimestampType| |**TimestampNTZType**|java.time.LocalDateTime|DataTypes.TimestampNTZType| -|**DateType**|java.sql.Date|DataTypes.DateType| +|**DateType**|java.time.LocalDate or java.sql.Date|DataTypes.DateType| |**YearMonthIntervalType**|java.time.Period|DataTypes.YearMonthIntervalType| |**DayTimeIntervalType**|java.time.Duration|DataTypes.DayTimeIntervalType| |**ArrayType**|java.util.List|DataTypes.createArrayType(*elementType*)**Note:** The value of *containsNull* will be true.DataTypes.createArrayType(*elementType*, *containsNull*).| - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (707408d98dc -> d3338c6879f)
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 707408d98dc [SPARK-42974][CORE] Restore `Utils.createTempDir` to use the `ShutdownHookManager` and clean up `JavaUtils.createTempDir` method add d3338c6879f [MINOR][DOCS] Add Java 8 types to value types of Scala/Java APIs No new revisions were added by this update. Summary of changes: docs/sql-ref-datatypes.md | 8 1 file changed, 4 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-42979][SQL] Define literal constructors as keywords
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 5a17537aa4a [SPARK-42979][SQL] Define literal constructors as keywords 5a17537aa4a is described below commit 5a17537aa4a777429431542cfa6184591476e54a Author: Max Gekk AuthorDate: Thu Mar 30 17:43:54 2023 +0300 [SPARK-42979][SQL] Define literal constructors as keywords ### What changes were proposed in this pull request? In the PR, I propose to define literal constructors `DATE`, `TIMESTAMP`, `TIMESTAMP_NTZ`, `TIMESTAMP_LTZ`, `INTERVAL`, and `X` as Spark SQL keywords. ### Why are the changes needed? The non-keywords literal constructors cause some inconveniences while analysing/transforming the lexer tree. For example, while forming the stable column aliases, see https://github.com/apache/spark/pull/40126. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *SQLKeywordSuite" $ build/sbt "test:testOnly *.ResolveAliasesSuite" ``` Closes #40593 from MaxGekk/typed-literal-keywords. Authored-by: Max Gekk Signed-off-by: Max Gekk --- docs/sql-ref-ansi-compliance.md| 1 + .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 + .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 12 ++- .../spark/sql/catalyst/parser/AstBuilder.scala | 23 +++--- .../catalyst/analysis/ResolveAliasesSuite.scala| 4 ++-- 5 files changed, 27 insertions(+), 14 deletions(-) diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 36d1f8f73eb..d4bb0e93bee 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -672,6 +672,7 @@ Below is a list of all the keywords in Spark SQL. |WINDOW|non-reserved|non-reserved|reserved| |WITH|reserved|non-reserved|reserved| |WITHIN|reserved|non-reserved|reserved| +|X|non-reserved|non-reserved|non-reserved| |YEAR|non-reserved|non-reserved|non-reserved| |YEARS|non-reserved|non-reserved|non-reserved| |ZONE|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 4d446b494f7..c9930fa0986 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -208,6 +208,7 @@ GRANT: 'GRANT'; GROUP: 'GROUP'; GROUPING: 'GROUPING'; HAVING: 'HAVING'; +BINARY_HEX: 'X'; HOUR: 'HOUR'; HOURS: 'HOURS'; IF: 'IF'; diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index ab54aef35df..a112b6e31fe 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -928,11 +928,19 @@ primaryExpression (FILTER LEFT_PAREN WHERE where=booleanExpression RIGHT_PAREN)? ( OVER windowSpec)? #percentile ; +literalType +: DATE +| TIMESTAMP | TIMESTAMP_LTZ | TIMESTAMP_NTZ +| INTERVAL +| BINARY_HEX +| unsupportedType=identifier +; + constant : NULL #nullLiteral | COLON identifier #parameterLiteral | interval #intervalLiteral -| identifier stringLit #typeConstructor +| literalType stringLit #typeConstructor | number #numericLiteral | booleanValue #booleanLiteral | stringLit+ #stringLiteral @@ -1227,6 +1235,7 @@ ansiNonReserved | BETWEEN | BIGINT | BINARY +| BINARY_HEX | BOOLEAN | BUCKET | BUCKETS @@ -1514,6 +1523,7 @@ nonReserved | BETWEEN | BIGINT | BINARY +| BINARY_HEX | BOOLEAN | BOTH | BUCKET 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/par
[spark] branch master updated: [SPARK-42873][SQL] Define Spark SQL types as keywords
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 907cefeebb5 [SPARK-42873][SQL] Define Spark SQL types as keywords 907cefeebb5 is described below commit 907cefeebb5e15ab6b4970ca8b6e42a8410a7c46 Author: Max Gekk AuthorDate: Wed Mar 29 18:52:06 2023 +0300 [SPARK-42873][SQL] Define Spark SQL types as keywords ### What changes were proposed in this pull request? In the PR, I propose to define Spark SQL types as keywords. ### Why are the changes needed? The non-keywords types cause some inconveniences while analysing/transforming the lexer tree. For example, while forming the stable column aliases, see https://github.com/apache/spark/pull/40126. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "test:testOnly *.ResolveAliasesSuite" $ build/sbt "test:testOnly *.ParserUtilsSuite" ``` Closes #40565 from MaxGekk/datatype-keywords. Authored-by: Max Gekk Signed-off-by: Max Gekk --- docs/sql-ref-ansi-compliance.md| 24 +++ .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 26 +++- .../spark/sql/catalyst/parser/SqlBaseParser.g4 | 74 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 61 +- .../spark/sql/errors/QueryParsingErrors.scala | 6 +- .../catalyst/analysis/ResolveAliasesSuite.scala| 6 +- .../sql/catalyst/parser/ParserUtilsSuite.scala | 10 +-- 7 files changed, 164 insertions(+), 43 deletions(-) diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 4124e958e39..36d1f8f73eb 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -366,10 +366,14 @@ Below is a list of all the keywords in Spark SQL. |AT|non-reserved|non-reserved|reserved| |AUTHORIZATION|reserved|non-reserved|reserved| |BETWEEN|non-reserved|non-reserved|reserved| +|BIGINT|non-reserved|non-reserved|reserved| +|BINARY|non-reserved|non-reserved|reserved| +|BOOLEAN|non-reserved|non-reserved|reserved| |BOTH|reserved|non-reserved|reserved| |BUCKET|non-reserved|non-reserved|non-reserved| |BUCKETS|non-reserved|non-reserved|non-reserved| |BY|non-reserved|non-reserved|reserved| +|BYTE|non-reserved|non-reserved|non-reserved| |CACHE|non-reserved|non-reserved|non-reserved| |CASCADE|non-reserved|non-reserved|non-reserved| |CASE|reserved|non-reserved|reserved| @@ -377,6 +381,8 @@ Below is a list of all the keywords in Spark SQL. |CATALOG|non-reserved|non-reserved|non-reserved| |CATALOGS|non-reserved|non-reserved|non-reserved| |CHANGE|non-reserved|non-reserved|non-reserved| +|CHAR|non-reserved|non-reserved|reserved| +|CHARACTER|non-reserved|non-reserved|reserved| |CHECK|reserved|non-reserved|reserved| |CLEAR|non-reserved|non-reserved|non-reserved| |CLUSTER|non-reserved|non-reserved|non-reserved| @@ -403,6 +409,7 @@ Below is a list of all the keywords in Spark SQL. |CURRENT_TIMESTAMP|reserved|non-reserved|reserved| |CURRENT_USER|reserved|non-reserved|reserved| |DATA|non-reserved|non-reserved|non-reserved| +|DATE|non-reserved|non-reserved|reserved| |DATABASE|non-reserved|non-reserved|non-reserved| |DATABASES|non-reserved|non-reserved|non-reserved| |DATEADD|non-reserved|non-reserved|non-reserved| @@ -411,6 +418,8 @@ Below is a list of all the keywords in Spark SQL. |DAYS|non-reserved|non-reserved|non-reserved| |DAYOFYEAR|non-reserved|non-reserved|non-reserved| |DBPROPERTIES|non-reserved|non-reserved|non-reserved| +|DEC|non-reserved|non-reserved|reserved| +|DECIMAL|non-reserved|non-reserved|reserved| |DEFAULT|non-reserved|non-reserved|non-reserved| |DEFINED|non-reserved|non-reserved|non-reserved| |DELETE|non-reserved|non-reserved|reserved| @@ -423,6 +432,7 @@ Below is a list of all the keywords in Spark SQL. |DISTINCT|reserved|non-reserved|reserved| |DISTRIBUTE|non-reserved|non-reserved|non-reserved| |DIV|non-reserved|non-reserved|not a keyword| +|DOUBLE|non-reserved|non-reserved|reserved| |DROP|non-reserved|non-reserved|reserved| |ELSE|reserved|non-reserved|reserved| |END|reserved|non-reserved|reserved| @@ -443,6 +453,7 @@ Below is a list of all the keywords in Spark SQL. |FILTER|reserved|non-reserved|reserved| |FILEFORMAT|non-reserved|non-reserved|non-reserved| |FIRST|non-reserved|non-reserved|non-reserved| +|FLOAT|non-reserved|non-reserved|reserved| |FOLLOWING|non-reserved|non-reserved|non-reserved| |FOR|reserved|non-reserved|reserved| |FOREIGN|reserved|non-reserved|reserved| @@ -471,6 +482,8 @@ Below is a list of all the keywords in Spark SQL. |INPATH|non-reserved|non-reserved|non-reserved| |INPUTFORMAT|non-reserved|non-reserved|non-reserved| |INSERT|non-r
[spark] branch branch-3.4 updated: [SPARK-42924][SQL][CONNECT][PYTHON] Clarify the comment of parameterized SQL args
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 dde9de60483 [SPARK-42924][SQL][CONNECT][PYTHON] Clarify the comment of parameterized SQL args dde9de60483 is described below commit dde9de60483ebb0b57013387dcfb8d5075cb6d0c Author: Max Gekk AuthorDate: Mon Mar 27 08:54:18 2023 +0300 [SPARK-42924][SQL][CONNECT][PYTHON] Clarify the comment of parameterized SQL args ### What changes were proposed in this pull request? In the PR, I propose to clarify the comment of `args` in parameterized `sql()`. ### Why are the changes needed? To make the comment more clear and highlight that input strings are parsed (not evaluated), and considered as SQL literal expressions. Also while parsing the fragments w/ SQL comments in the string values are skipped. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By checking coding style: ``` $ ./dev/lint-python $ ./dev/scalastyle ``` Closes #40508 from MaxGekk/parameterized-sql-doc. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit c55c7ea6fc92c3733543d5f3d99eb00921cbe564) Signed-off-by: Max Gekk --- .../jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala | 10 -- .../common/src/main/protobuf/spark/connect/commands.proto | 5 - .../common/src/main/protobuf/spark/connect/relations.proto | 5 - python/pyspark/pandas/sql_formatter.py | 6 -- python/pyspark/sql/connect/proto/commands_pb2.pyi | 6 +- python/pyspark/sql/connect/proto/relations_pb2.pyi | 6 +- python/pyspark/sql/session.py | 6 -- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 10 -- 8 files changed, 42 insertions(+), 12 deletions(-) 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 f1e82507393..548545b969d 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 @@ -213,7 +213,10 @@ class SparkSession private[sql] ( * @param sqlText * A SQL statement with named parameters to execute. * @param args - * A map of parameter names to literal values. + * A map of parameter names to string values that are parsed as SQL literal expressions. For + * example, map keys: "rank", "name", "birthdate"; map values: "1", "'Steven'", + * "DATE'2023-03-21'". The fragments of string values belonged to SQL comments are skipped + * while parsing. * * @since 3.4.0 */ @@ -229,7 +232,10 @@ class SparkSession private[sql] ( * @param sqlText * A SQL statement with named parameters to execute. * @param args - * A map of parameter names to literal values. + * A map of parameter names to string values that are parsed as SQL literal expressions. For + * example, map keys: "rank", "name", "birthdate"; map values: "1", "'Steven'", + * "DATE'2023-03-21'". The fragments of string values belonged to SQL comments are skipped + * while parsing. * * @since 3.4.0 */ diff --git a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto index 3ffbe83bded..604421fdd4f 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -53,7 +53,10 @@ message SqlCommand { // (Required) SQL Query. string sql = 1; - // (Optional) A map of parameter names to literal values. + // (Optional) A map of parameter names to string values that are parsed as + // SQL literal expressions. For example, map keys: "rank", "name", "birthdate"; + // map values: "1", "'Steven'", "DATE'2023-03-21'". The fragments of string values + // belonged to SQL comments are skipped while parsing. map args = 2; } diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index 9dec94411a9..976bd68e7fe 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -108,7 +108,10 @@ message SQL { // (Req
[spark] branch master updated: [SPARK-42924][SQL][CONNECT][PYTHON] Clarify the comment of parameterized SQL args
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 c55c7ea6fc9 [SPARK-42924][SQL][CONNECT][PYTHON] Clarify the comment of parameterized SQL args c55c7ea6fc9 is described below commit c55c7ea6fc92c3733543d5f3d99eb00921cbe564 Author: Max Gekk AuthorDate: Mon Mar 27 08:54:18 2023 +0300 [SPARK-42924][SQL][CONNECT][PYTHON] Clarify the comment of parameterized SQL args ### What changes were proposed in this pull request? In the PR, I propose to clarify the comment of `args` in parameterized `sql()`. ### Why are the changes needed? To make the comment more clear and highlight that input strings are parsed (not evaluated), and considered as SQL literal expressions. Also while parsing the fragments w/ SQL comments in the string values are skipped. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By checking coding style: ``` $ ./dev/lint-python $ ./dev/scalastyle ``` Closes #40508 from MaxGekk/parameterized-sql-doc. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala | 10 -- .../common/src/main/protobuf/spark/connect/commands.proto | 5 - .../common/src/main/protobuf/spark/connect/relations.proto | 5 - python/pyspark/pandas/sql_formatter.py | 6 -- python/pyspark/sql/connect/proto/commands_pb2.pyi | 6 +- python/pyspark/sql/connect/proto/relations_pb2.pyi | 6 +- python/pyspark/sql/session.py | 6 -- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 10 -- 8 files changed, 42 insertions(+), 12 deletions(-) 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 f1e82507393..548545b969d 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 @@ -213,7 +213,10 @@ class SparkSession private[sql] ( * @param sqlText * A SQL statement with named parameters to execute. * @param args - * A map of parameter names to literal values. + * A map of parameter names to string values that are parsed as SQL literal expressions. For + * example, map keys: "rank", "name", "birthdate"; map values: "1", "'Steven'", + * "DATE'2023-03-21'". The fragments of string values belonged to SQL comments are skipped + * while parsing. * * @since 3.4.0 */ @@ -229,7 +232,10 @@ class SparkSession private[sql] ( * @param sqlText * A SQL statement with named parameters to execute. * @param args - * A map of parameter names to literal values. + * A map of parameter names to string values that are parsed as SQL literal expressions. For + * example, map keys: "rank", "name", "birthdate"; map values: "1", "'Steven'", + * "DATE'2023-03-21'". The fragments of string values belonged to SQL comments are skipped + * while parsing. * * @since 3.4.0 */ diff --git a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto index 3ffbe83bded..604421fdd4f 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -53,7 +53,10 @@ message SqlCommand { // (Required) SQL Query. string sql = 1; - // (Optional) A map of parameter names to literal values. + // (Optional) A map of parameter names to string values that are parsed as + // SQL literal expressions. For example, map keys: "rank", "name", "birthdate"; + // map values: "1", "'Steven'", "DATE'2023-03-21'". The fragments of string values + // belonged to SQL comments are skipped while parsing. map args = 2; } diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index 9dec94411a9..976bd68e7fe 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -108,7 +108,10 @@ message SQL { // (Required) The SQL query. string query = 1; - // (Optional) A map of parameter names to literal values. +
[spark] branch master updated: [SPARK-40822][SQL] Stable derived column aliases
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 83a40743153 [SPARK-40822][SQL] Stable derived column aliases 83a40743153 is described below commit 83a40743153c445eca3798f7c24a362231f76022 Author: Max Gekk AuthorDate: Tue Mar 21 09:14:22 2023 +0300 [SPARK-40822][SQL] Stable derived column aliases ### What changes were proposed in this pull request? In the PR, I propose to change auto-generation of column aliases (the case when an user doesn't assign any alias explicitly). Before the changes, Spark SQL generates such alias from `Expression` but this PR proposes to take the parse tree (output of lexer), and generate an alias using the term tokens from the tree. New helper function `ParserUtils.toExprAlias` takes a `ParseTree` from `Antlr4`, and converts it to a `String` using the following simple rule: - Concatenate all terminal nodes of the lexer tree without any gaps. - Upper case keywords and numeric literals. For example, the sequence of tokens "1", "in", "(", "1.0d" ")" is converted to the alias "1IN(1.0D)". By default, the feature is off, and the SQL config `spark.sql.stableDerivedColumnAlias.enabled` allows to enable it. Closes #39332 ### Why are the changes needed? To improve user experience with Spark SQL. It is always best practice to name the result of any expressions in a queries select list, if one plans to reference them later. This yields the most readable results and stability. However, sometimes queries are generated or we’re just lazy and trust in the auto generated names. The problem is that the auto-generated names are produced by pretty printing the expression tree which is, while “generally” readable, not meant to be stable across [...] ```sql spark-sql> DESC SELECT substring('hello', 5); substring(hello, 5, 2147483647) string ``` the auto-generated column alias `substring(hello, 5, 2147483647)` contains not-obvious elements. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? By running new test: ``` $ build/sbt "test:testOnly *.ResolveAliasesSuite" ``` Closes #40126 from MaxGekk/stable-derived-column-aliases-2. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 6 ++- .../spark/sql/catalyst/analysis/Analyzer.scala | 15 ++- .../spark/sql/catalyst/parser/AstBuilder.scala | 23 +++ .../spark/sql/catalyst/parser/ParserUtils.scala| 33 +-- .../org/apache/spark/sql/internal/SQLConf.scala| 9 .../catalyst/analysis/ResolveAliasesSuite.scala| 48 ++ 6 files changed, 112 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 6d0862290cf..17f7c49052a 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -81,7 +81,8 @@ LEFT_BRACKET: '['; RIGHT_BRACKET: ']'; // NOTE: If you add a new token in the list below, you should update the list of keywords -// and reserved tag in `docs/sql-ref-ansi-compliance.md#sql-keywords`. +// and reserved tag in `docs/sql-ref-ansi-compliance.md#sql-keywords`, and +// modify `ParserUtils.toExprAlias()` which assumes all keywords are between `ADD` and `ZONE`. // // Start of the keywords list @@ -426,6 +427,9 @@ DOUBLEQUOTED_STRING :'"' ( ~('"'|'\\') | ('\\' .) )* '"' ; +// NOTE: If you move a numeric literal, you should modify `ParserUtils.toExprAlias()` +// which assumes all numeric literals are between `BIGINT_LITERAL` and `BIGDECIMAL_LITERAL`. + BIGINT_LITERAL : DIGIT+ 'L' ; 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 3a2dff78cba..8821e652a31 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 @@ -478,11 +478,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case _: Attribute => true case _ => false } - def metaForAutoGeneratedAlias = { -new MetadataBuilder() - .putString("__autoGeneratedAlias", "true") - .b
[spark] branch master updated: [SPARK-42791][SQL][FOLLOWUP] Re-generate golden files for `array_prepend`
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 04333bb862f [SPARK-42791][SQL][FOLLOWUP] Re-generate golden files for `array_prepend` 04333bb862f is described below commit 04333bb862f284c153bb3a8cc5600aefcc08d0d1 Author: yangjie01 AuthorDate: Mon Mar 20 21:55:07 2023 +0300 [SPARK-42791][SQL][FOLLOWUP] Re-generate golden files for `array_prepend` ### What changes were proposed in this pull request? This pr re-generates golden files for `array_prepend` functions. It seems that the newly added case in https://github.com/apache/spark/pull/38947 is missing from the golden files due to lack of rebase when merging https://github.com/apache/spark/pull/40449. ### Why are the changes needed? Re-generates golden files for `array_prepend` functions to Pass GitHub Actions ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Manually checked with Scala 2.13 Closes #40492 from LuciferYang/SPARK-42791-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: Max Gekk --- .../sql-tests/analyzer-results/ansi/array.sql.out | 63 ++ .../sql-tests/analyzer-results/array.sql.out | 63 ++ 2 files changed, 126 insertions(+) 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 83fa42695d9..a9c18ab8690 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 @@ -657,3 +657,66 @@ select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)) -- !query analysis Project [array_append(array(cast(null as string)), cast(null as string)) AS array_append(array(CAST(NULL AS STRING)), CAST(NULL AS STRING))#x] +- OneRowRelation + + +-- !query +select array_prepend(array(1, 2, 3), 4) +-- !query analysis +Project [array_prepend(array(1, 2, 3), 4) AS array_prepend(array(1, 2, 3), 4)#x] ++- OneRowRelation + + +-- !query +select array_prepend(array('a', 'b', 'c'), 'd') +-- !query analysis +Project [array_prepend(array(a, b, c), d) AS array_prepend(array(a, b, c), d)#x] ++- OneRowRelation + + +-- !query +select array_prepend(array(1, 2, 3, NULL), NULL) +-- !query analysis +Project [array_prepend(array(1, 2, 3, cast(null as int)), cast(null as int)) AS array_prepend(array(1, 2, 3, NULL), NULL)#x] ++- OneRowRelation + + +-- !query +select array_prepend(array('a', 'b', 'c', NULL), NULL) +-- !query analysis +Project [array_prepend(array(a, b, c, cast(null as string)), cast(null as string)) AS array_prepend(array(a, b, c, NULL), NULL)#x] ++- OneRowRelation + + +-- !query +select array_prepend(CAST(null AS ARRAY), 'a') +-- !query analysis +Project [array_prepend(cast(null as array), a) AS array_prepend(NULL, a)#x] ++- OneRowRelation + + +-- !query +select array_prepend(CAST(null AS ARRAY), CAST(null as String)) +-- !query analysis +Project [array_prepend(cast(null as array), cast(null as string)) AS array_prepend(NULL, CAST(NULL AS STRING))#x] ++- OneRowRelation + + +-- !query +select array_prepend(array(), 1) +-- !query analysis +Project [array_prepend(cast(array() as array), 1) AS array_prepend(array(), 1)#x] ++- OneRowRelation + + +-- !query +select array_prepend(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query analysis +Project [array_prepend(cast(array() as array), cast(null as string)) AS array_prepend(array(), CAST(NULL AS STRING))#x] ++- OneRowRelation + + +-- !query +select array_prepend(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query analysis +Project [array_prepend(array(cast(null as string)), cast(null as string)) AS array_prepend(array(CAST(NULL AS STRING)), CAST(NULL AS STRING))#x] ++- OneRowRelation 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 83fa42695d9..a9c18ab8690 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 @@ -657,3 +657,66 @@ select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)) -- !query analysis Project [array_append(array(cast(null as string)), cast(null as string)) AS array_append(array(CAST(NULL AS STRING)), CAST(NULL AS STRING))#x] +- OneRowRelation + + +-- !query +select array_prepend(array(1, 2, 3), 4) +-- !query analysis +Project [array_prepend(array(1, 2, 3), 4) AS array_prepend(array(1, 2, 3), 4)#x] ++- OneRowRelation + + +-- !query +select array_prepend(array('a', 'b', 'c'), 'd') +-- !query analysis +Project
[spark] branch branch-3.4 updated: [SPARK-42706][SQL][DOCS][3.4] Document the Spark SQL error classes in user-facing documentation
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 9c1cb473c63 [SPARK-42706][SQL][DOCS][3.4] Document the Spark SQL error classes in user-facing documentation 9c1cb473c63 is described below commit 9c1cb473c639ff907f8fe7aaa16da2d07d1a0628 Author: itholic AuthorDate: Wed Mar 15 09:49:23 2023 +0300 [SPARK-42706][SQL][DOCS][3.4] Document the Spark SQL error classes in user-facing documentation ### What changes were proposed in this pull request? Cherry-pick for https://github.com/apache/spark/pull/40336. This PR proposes to document Spark SQL error classes to [Spark SQL Guide](https://spark.apache.org/docs/latest/sql-programming-guide.html). - Error Conditions https://user-images.githubusercontent.com/44108233/223706823-7817b57d-c032-4817-a440-7f79119fa0b4.png;> - SQLSTATE Codes https://user-images.githubusercontent.com/44108233/223706860-3f64b00b-fa0d-47e0-b154-0d7be92b8637.png;> - Error Classes that includes sub-error classes (`INVALID_FORMAT` as an example) https://user-images.githubusercontent.com/44108233/223709925-74144f41-8836-45dc-b851-5d96ac8aa38c.png;> ### Why are the changes needed? To improve the usability for error messages for Spark SQL. ### Does this PR introduce _any_ user-facing change? No API change, but yes, it's user-facing documentation. ### How was this patch tested? Manually built docs and check the contents one-by-one compare to [error-classes.json](https://github.com/apache/spark/blob/master/core/src/main/resources/error/error-classes.json). Closes #40336 from itholic/SPARK-42706. Authored-by: itholic ### What changes were proposed in this pull request? ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #40433 from itholic/SPARK-42706-3.4. Authored-by: itholic Signed-off-by: Max Gekk --- docs/_data/menu-sql.yaml | 45 + docs/sql-error-conditions-connect-error-class.md | 44 + ...ror-conditions-datatype-mismatch-error-class.md | 238 ...tions-incompatible-data-to-table-error-class.md | 64 + ...tions-incomplete-type-definition-error-class.md | 40 + ...onsistent-behavior-cross-version-error-class.md | 82 ++ ...-error-conditions-invalid-format-error-class.md | 68 + ...error-conditions-invalid-options-error-class.md | 36 + ...nditions-invalid-parameter-value-error-class.md | 44 + ...-error-conditions-invalid-schema-error-class.md | 42 + ...ions-invalid-subquery-expression-error-class.md | 32 + ...ns-not-null-constraint-violation-error-class.md | 36 + docs/sql-error-conditions-sqlstates.md | 720 +++ ...ror-conditions-unresolved-column-error-class.md | 36 + ...rror-conditions-unresolved-field-error-class.md | 36 + ...or-conditions-unresolved-map-key-error-class.md | 36 + ...ditions-unsupported-deserializer-error-class.md | 36 + ...r-conditions-unsupported-feature-error-class.md | 140 ++ ...conditions-unsupported-generator-error-class.md | 44 + ...conditions-unsupported-save-mode-error-class.md | 36 + ...ted-subquery-expression-category-error-class.md | 76 ++ ...-error-conditions-wrong-num-args-error-class.md | 36 + docs/sql-error-conditions.md | 1342 23 files changed, 3309 insertions(+) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 7510cf8faf4..bf7a88d90d0 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -102,3 +102,48 @@ url: sql-ref-syntax.html#data-retrieval-statements - text: Auxiliary Statements url: sql-ref-syntax.html#auxiliary-statements +- text: Error Conditions + url: sql-error-conditions.html + subitems: +- text: SQLSTATE Codes + url: sql-error-conditions-sqlstates.html +- text: CONNECT error class + url: sql-error-conditions-connect-error-class.html +- text: DATATYPE_MISMATCH error class + url: sql-error-conditions-datatype-mismatch-error-class.html +- text: INCOMPATIBLE_DATA_TO_TABLE error class + url: sql-error-conditions-incompatible-data-to-table-error-class.html +- text: INCOMPLETE_TYPE_DEFINITION error class + url: sql-error-conditions-incomplete-type-definition-error-class.html +- text: INCONSISTENT_BEHAVIOR_CROSS_VERSION error class + url: sql-error-conditions-inconsistent-behavior-cross-version-error-class.html +- text: INVALID_FORMAT error class + url: sql-error-conditions-invalid-format-error-class.html +- text: INVALID_OPTIONS error class + url: sql-error-conditions-invalid-opti
[spark] branch master updated (f8966e7eee1 -> ac30c07dff4)
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 f8966e7eee1 [SPARK-42702][SPARK-42623][SQL] Support parameterized query in subquery and CTE add ac30c07dff4 [SPARK-42743][SQL] Support analyze TimestampNTZ columns No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/catalog/interface.scala | 9 +-- .../sql/catalyst/util/TimestampFormatter.scala | 8 ++ .../execution/command/AnalyzeColumnCommand.scala | 5 ++-- .../spark/sql/execution/command/CommandUtils.scala | 6 ++--- .../spark/sql/StatisticsCollectionTestBase.scala | 29 +- .../apache/spark/sql/hive/StatisticsSuite.scala| 3 ++- 6 files changed, 44 insertions(+), 16 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-38735][SQL][TESTS] Add tests for the error class: INTERNAL_ERROR
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 b61be00278c [SPARK-38735][SQL][TESTS] Add tests for the error class: INTERNAL_ERROR b61be00278c is described below commit b61be00278c79037e258bd93f88ada336e8a6324 Author: eugene gusev AuthorDate: Thu Mar 9 22:34:49 2023 +0300 [SPARK-38735][SQL][TESTS] Add tests for the error class: INTERNAL_ERROR ### What changes were proposed in this pull request? This PR aims to add tests for the error class INTERNAL_ERROR to QueryExecutionErrorsSuite ### Why are the changes needed? The changes improve test coverage, and document expected error messages in tests ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the tests Closes #40236 from the8thC/internal-error-tests. Authored-by: eugene gusev Signed-off-by: Max Gekk --- .../sql/errors/QueryExecutionErrorsSuite.scala | 57 ++ 1 file changed, 57 insertions(+) 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 90180d5e600..d6a310df39e 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 @@ -28,6 +28,11 @@ import org.mockito.Mockito.{mock, spy, when} import org.apache.spark._ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator +import org.apache.spark.sql.catalyst.expressions.{Grouping, Literal, Parameter} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean import org.apache.spark.sql.catalyst.util.BadRecordException import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider @@ -765,6 +770,58 @@ class QueryExecutionErrorsSuite ) } } + + test("INTERNAL_ERROR: Calling eval on Unevaluable expression") { +val e = intercept[SparkException] { + Parameter("foo").eval() +} +checkError( + exception = e, + errorClass = "INTERNAL_ERROR", + parameters = Map("message" -> "Cannot evaluate expression: parameter(foo)"), + sqlState = "XX000") + } + + test("INTERNAL_ERROR: Calling doGenCode on unresolved") { +val e = intercept[SparkException] { + val ctx = new CodegenContext + Grouping(Parameter("foo")).genCode(ctx) +} +checkError( + exception = e, + errorClass = "INTERNAL_ERROR", + parameters = Map( +"message" -> ("Cannot generate code for expression: " + + "grouping(parameter(foo))")), + sqlState = "XX000") + } + + test("INTERNAL_ERROR: Calling terminate on UnresolvedGenerator") { +val e = intercept[SparkException] { + UnresolvedGenerator(FunctionIdentifier("foo"), Seq.empty).terminate() +} +checkError( + exception = e, + errorClass = "INTERNAL_ERROR", + parameters = Map("message" -> "Cannot terminate expression: 'foo()"), + sqlState = "XX000") + } + + test("INTERNAL_ERROR: Initializing JavaBean with non existing method") { +val e = intercept[SparkException] { + val initializeWithNonexistingMethod = InitializeJavaBean( +Literal.fromObject(new java.util.LinkedList[Int]), +Map("nonexistent" -> Literal(1))) + initializeWithNonexistingMethod.eval() +} +checkError( + exception = e, + errorClass = "INTERNAL_ERROR", + parameters = Map( +"message" -> ("""A method named "nonexistent" is not declared in """ + + "any enclosing class nor any supertype")), + sqlState = "XX000") + } } class FakeFileSystemSetPermission extends LocalFileSystem { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (c0b1735c0bf -> 3f5d24f3e48)
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 c0b1735c0bf [SPARK-42630][CONNECT][PYTHON] Introduce UnparsedDataType and delay parsing DDL string until SparkConnectClient is available add 3f5d24f3e48 [SPARK-42706][SQL][DOCS] Document the Spark SQL error classes in user-facing documentation No new revisions were added by this update. Summary of changes: docs/_data/menu-sql.yaml | 45 + docs/sql-error-conditions-connect-error-class.md | 44 + ...ror-conditions-datatype-mismatch-error-class.md | 238 ...tions-incompatible-data-to-table-error-class.md | 64 + ...tions-incomplete-type-definition-error-class.md | 40 + ...onsistent-behavior-cross-version-error-class.md | 82 ++ ...-error-conditions-invalid-format-error-class.md | 68 + ...error-conditions-invalid-options-error-class.md | 36 + ...nditions-invalid-parameter-value-error-class.md | 44 + ...-error-conditions-invalid-schema-error-class.md | 42 + ...ions-invalid-subquery-expression-error-class.md | 32 + ...ns-not-null-constraint-violation-error-class.md | 36 + docs/sql-error-conditions-sqlstates.md | 720 +++ ...ror-conditions-unresolved-column-error-class.md | 36 + ...rror-conditions-unresolved-field-error-class.md | 36 + ...or-conditions-unresolved-map-key-error-class.md | 36 + ...ditions-unsupported-deserializer-error-class.md | 36 + ...r-conditions-unsupported-feature-error-class.md | 140 ++ ...conditions-unsupported-generator-error-class.md | 44 + ...conditions-unsupported-save-mode-error-class.md | 36 + ...ted-subquery-expression-category-error-class.md | 76 ++ ...-error-conditions-wrong-num-args-error-class.md | 36 + docs/sql-error-conditions.md | 1342 23 files changed, 3309 insertions(+) create mode 100644 docs/sql-error-conditions-connect-error-class.md create mode 100644 docs/sql-error-conditions-datatype-mismatch-error-class.md create mode 100644 docs/sql-error-conditions-incompatible-data-to-table-error-class.md create mode 100644 docs/sql-error-conditions-incomplete-type-definition-error-class.md create mode 100644 docs/sql-error-conditions-inconsistent-behavior-cross-version-error-class.md create mode 100644 docs/sql-error-conditions-invalid-format-error-class.md create mode 100644 docs/sql-error-conditions-invalid-options-error-class.md create mode 100644 docs/sql-error-conditions-invalid-parameter-value-error-class.md create mode 100644 docs/sql-error-conditions-invalid-schema-error-class.md create mode 100644 docs/sql-error-conditions-invalid-subquery-expression-error-class.md create mode 100644 docs/sql-error-conditions-not-null-constraint-violation-error-class.md create mode 100644 docs/sql-error-conditions-sqlstates.md create mode 100644 docs/sql-error-conditions-unresolved-column-error-class.md create mode 100644 docs/sql-error-conditions-unresolved-field-error-class.md create mode 100644 docs/sql-error-conditions-unresolved-map-key-error-class.md create mode 100644 docs/sql-error-conditions-unsupported-deserializer-error-class.md create mode 100644 docs/sql-error-conditions-unsupported-feature-error-class.md create mode 100644 docs/sql-error-conditions-unsupported-generator-error-class.md create mode 100644 docs/sql-error-conditions-unsupported-save-mode-error-class.md create mode 100644 docs/sql-error-conditions-unsupported-subquery-expression-category-error-class.md create mode 100644 docs/sql-error-conditions-wrong-num-args-error-class.md create mode 100644 docs/sql-error-conditions.md - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42701][SQL] Add the `try_aes_decrypt()` function
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 7d10330720f [SPARK-42701][SQL] Add the `try_aes_decrypt()` function 7d10330720f is described below commit 7d10330720f600d7d1aca3ea1ccfcf1f74f41136 Author: Max Gekk AuthorDate: Thu Mar 9 09:13:05 2023 +0300 [SPARK-42701][SQL] Add the `try_aes_decrypt()` function ### What changes were proposed in this pull request? In the PR, I propose to add new function `try_aes_decrypt()` which binds to new expression `TryAesDecrypt` that is a runtime replaceable expression of the combination of `TryEval` and `AesDecrypt`. ### Why are the changes needed? The changes improve user experience with Spark SQL. The existing function `aes_decrypt()` fails w/ an exception as soon as it faces to some invalid input that cannot be decrypted, and the rest (even if the values can be decrypted) is ignored. New function returns `NULL` on bad inputs and decrypts other values. ### Does this PR introduce _any_ user-facing change? No. This PR just extends existing API. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "sql/test:testOnly org.apache.spark.sql.expressions.ExpressionInfoSuite" $ build/sbt "sql/testOnly *ExpressionsSchemaSuite" ``` Closes #40340 from MaxGekk/try_aes_decrypt. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../sql/catalyst/analysis/FunctionRegistry.scala | 1 + .../spark/sql/catalyst/expressions/misc.scala | 34 ++ .../sql-functions/sql-expression-schema.md | 1 + 3 files changed, 36 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 103e6aae603..ad82a836199 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -452,6 +452,7 @@ object FunctionRegistry { expressionBuilder("try_sum", TrySumExpressionBuilder, setAlias = true), expression[TryToBinary]("try_to_binary"), expressionBuilder("try_to_timestamp", TryToTimestampExpressionBuilder, setAlias = true), +expression[TryAesDecrypt]("try_aes_decrypt"), // aggregate functions expression[HyperLogLogPlusPlus]("approx_count_distinct"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index bf9dd700dfa..300fab0386c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -432,4 +432,38 @@ case class AesDecrypt( copy(newChildren(0), newChildren(1), newChildren(2), newChildren(3)) } } + +@ExpressionDescription( + usage = "_FUNC_(expr, key[, mode[, padding]]) - This is a special version of `aes_decrypt` that performs the same operation, but returns a NULL value instead of raising an error if the decryption cannot be performed.", + examples = """ +Examples: + > SELECT _FUNC_(unhex('6E7CA17BBB468D3084B5744BCA729FB7B2B7BCB8E4472847D02670489D95FA97DBBA7D3210'), '', 'GCM'); + Spark SQL + > SELECT _FUNC_(unhex('--468D3084B5744BCA729FB7B2B7BCB8E4472847D02670489D95FA97DBBA7D3210'), '', 'GCM'); + NULL + """, + since = "3.5.0", + group = "misc_funcs") +// scalastyle:on line.size.limit +case class TryAesDecrypt( +input: Expression, +key: Expression, +mode: Expression, +padding: Expression, +replacement: Expression) extends RuntimeReplaceable with InheritAnalysisRules { + + def this(input: Expression, key: Expression, mode: Expression, padding: Expression) = +this(input, key, mode, padding, TryEval(AesDecrypt(input, key, mode, padding))) + def this(input: Expression, key: Expression, mode: Expression) = +this(input, key, mode, Literal("DEFAULT")) + def this(input: Expression, key: Expression) = +this(input, key, Literal("GCM")) + + override def prettyName: String = "try_aes_decrypt" + + override def parameters: Seq[Expression] = Seq(input, key, mode, padding) + + override protected def withNewChildInternal(newChild: Expression): Expression = +this.copy(replacement = newChild) +} // scalastyle:on line.size.limit diff --git a/sql/core/src/test/resources/sql-functions/sql-
[spark] branch branch-3.3 updated (20870c3d157 -> d98a1d8d4bd)
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 20870c3d157 [SPARK-42647][PYTHON] Change alias for numpy deprecated and removed types add d98a1d8d4bd [SPARK-42635][SQL][3.3] Fix the TimestampAdd expression No new revisions were added by this update. Summary of changes: .../spark/sql/catalyst/util/DateTimeUtils.scala| 22 +++-- .../expressions/DateExpressionsSuite.scala | 95 +- 2 files changed, 107 insertions(+), 10 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-42635][SQL] Fix the TimestampAdd expression
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 cf5c90df3af [SPARK-42635][SQL] Fix the TimestampAdd expression cf5c90df3af is described below commit cf5c90df3afc7b68d61b11b78577f71fd4d85135 Author: Chenhao Li AuthorDate: Fri Mar 3 09:38:44 2023 +0300 [SPARK-42635][SQL] Fix the TimestampAdd expression ### What changes were proposed in this pull request? This PR fixed the counter-intuitive behaviors of the `TimestampAdd` expression mentioned in https://issues.apache.org/jira/browse/SPARK-42635. See the following *user-facing* changes for details. ### Does this PR introduce _any_ user-facing change? Yes. This PR fixes the three problems mentioned in SPARK-42635: 1. When the time is close to daylight saving time transition, the result may be discontinuous and not monotonic. 2. Adding month, quarter, and year silently ignores `Int` overflow during unit conversion. 3. Adding sub-month units (week, day, hour, minute, second, millisecond, microsecond)silently ignores `Long` overflow during unit conversion. Some examples of the result changes: Old results: ``` // In America/Los_Angeles timezone: timestampadd(DAY, 1, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 (this is correct, put it here for comparison) timestampadd(HOUR, 23, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(HOUR, 24, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(SECOND, 86400 - 1, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 timestampadd(SECOND, 86400, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 // In UTC timezone: timestampadd(quarter, 1431655764, 1970-01-01 00:00:00) = 1969-09-01 00:00:00 timestampadd(day, 106751992, 1970-01-01 00:00:00) = -290308-12-22 15:58:10.448384 ``` New results: ``` // In America/Los_Angeles timezone: timestampadd(DAY, 1, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(HOUR, 23, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(HOUR, 24, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 timestampadd(SECOND, 86400 - 1, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 timestampadd(SECOND, 86400, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 // In UTC timezone: timestampadd(quarter, 1431655764, 1970-01-01 00:00:00) = throw overflow exception timestampadd(day, 106751992, 1970-01-01 00:00:00) = throw overflow exception ``` ### How was this patch tested? Pass existing tests and some new tests. Closes #40237 from chenhao-db/SPARK-42635. Authored-by: Chenhao Li Signed-off-by: Max Gekk (cherry picked from commit 392bdc1595879ea03c90e3f4b550aa5ce7b32bdf) Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/DateTimeUtils.scala| 22 +++-- .../expressions/DateExpressionsSuite.scala | 98 +- 2 files changed, 110 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index af0666a98fa..2ae11f6568a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -1227,25 +1227,29 @@ object DateTimeUtils { try { unit.toUpperCase(Locale.ROOT) match { case "MICROSECOND" => - timestampAddDayTime(micros, quantity, zoneId) + timestampAddInterval(micros, 0, 0, quantity, zoneId) case "MILLISECOND" => - timestampAddDayTime(micros, quantity * MICROS_PER_MILLIS, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_MILLIS), zoneId) case "SECOND" => - timestampAddDayTime(micros, quantity * MICROS_PER_SECOND, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_SECOND), zoneId) case "MINUTE" => - timestampAddDayTime(micros, quantity * MICROS_PER_MINUTE, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_MINUTE), zoneId) case "HOUR" => - timestampAddDayTime(micros, quantity * MICROS_PER_HOUR, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_HOUR), zoneId) case "DAY" | "DAYOFYEAR" => - timestampAddDayTime(micros, quantity * MICROS_PER_DAY, zoneId) + timestampAddInterval(micros, 0, quantity, 0, zoneId)
[spark] branch master updated: [SPARK-42635][SQL] Fix the TimestampAdd expression
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 392bdc15958 [SPARK-42635][SQL] Fix the TimestampAdd expression 392bdc15958 is described below commit 392bdc1595879ea03c90e3f4b550aa5ce7b32bdf Author: Chenhao Li AuthorDate: Fri Mar 3 09:38:44 2023 +0300 [SPARK-42635][SQL] Fix the TimestampAdd expression ### What changes were proposed in this pull request? This PR fixed the counter-intuitive behaviors of the `TimestampAdd` expression mentioned in https://issues.apache.org/jira/browse/SPARK-42635. See the following *user-facing* changes for details. ### Does this PR introduce _any_ user-facing change? Yes. This PR fixes the three problems mentioned in SPARK-42635: 1. When the time is close to daylight saving time transition, the result may be discontinuous and not monotonic. 2. Adding month, quarter, and year silently ignores `Int` overflow during unit conversion. 3. Adding sub-month units (week, day, hour, minute, second, millisecond, microsecond)silently ignores `Long` overflow during unit conversion. Some examples of the result changes: Old results: ``` // In America/Los_Angeles timezone: timestampadd(DAY, 1, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 (this is correct, put it here for comparison) timestampadd(HOUR, 23, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(HOUR, 24, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(SECOND, 86400 - 1, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 timestampadd(SECOND, 86400, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 // In UTC timezone: timestampadd(quarter, 1431655764, 1970-01-01 00:00:00) = 1969-09-01 00:00:00 timestampadd(day, 106751992, 1970-01-01 00:00:00) = -290308-12-22 15:58:10.448384 ``` New results: ``` // In America/Los_Angeles timezone: timestampadd(DAY, 1, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(HOUR, 23, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 timestampadd(HOUR, 24, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 timestampadd(SECOND, 86400 - 1, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 timestampadd(SECOND, 86400, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 // In UTC timezone: timestampadd(quarter, 1431655764, 1970-01-01 00:00:00) = throw overflow exception timestampadd(day, 106751992, 1970-01-01 00:00:00) = throw overflow exception ``` ### How was this patch tested? Pass existing tests and some new tests. Closes #40237 from chenhao-db/SPARK-42635. Authored-by: Chenhao Li Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/DateTimeUtils.scala| 22 +++-- .../expressions/DateExpressionsSuite.scala | 98 +- 2 files changed, 110 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index af0666a98fa..2ae11f6568a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -1227,25 +1227,29 @@ object DateTimeUtils { try { unit.toUpperCase(Locale.ROOT) match { case "MICROSECOND" => - timestampAddDayTime(micros, quantity, zoneId) + timestampAddInterval(micros, 0, 0, quantity, zoneId) case "MILLISECOND" => - timestampAddDayTime(micros, quantity * MICROS_PER_MILLIS, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_MILLIS), zoneId) case "SECOND" => - timestampAddDayTime(micros, quantity * MICROS_PER_SECOND, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_SECOND), zoneId) case "MINUTE" => - timestampAddDayTime(micros, quantity * MICROS_PER_MINUTE, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_MINUTE), zoneId) case "HOUR" => - timestampAddDayTime(micros, quantity * MICROS_PER_HOUR, zoneId) + timestampAddInterval(micros, 0, 0, +Math.multiplyExact(quantity.toLong, MICROS_PER_HOUR), zoneId) case "DAY" | "DAYOFYEAR" => - timestampAddDayTime(micros, quantity * MICROS_PER_DAY, zoneId) + timestampAddInterval(micros, 0, quantity, 0, zoneId) case "WEEK" => - timestampAddDayTime(micros, quantity * MICROS_PER_DAY * DAYS_PER_WEEK, zoneI
[spark] branch branch-3.3 updated: [SPARK-42553][SQL][3.3] Ensure at least one time unit after "interval"
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 7e90f4226dd [SPARK-42553][SQL][3.3] Ensure at least one time unit after "interval" 7e90f4226dd is described below commit 7e90f4226dd5e4fae1af5e88a334d87b330019cb Author: jiangyzanze AuthorDate: Thu Mar 2 18:25:02 2023 +0300 [SPARK-42553][SQL][3.3] Ensure at least one time unit after "interval" ### What changes were proposed in this pull request? This PR aims to ensure "at least one time unit should be given for interval literal" by modifying SqlBaseParser. This is a backport of https://github.com/apache/spark/pull/40195 ### Why are the changes needed? INTERVAL is a Non-Reserved keyword in spark. But when I run ```shell scala> spark.sql("select interval from mytable") ``` I get ``` org.apache.spark.sql.catalyst.parser.ParseException: at least one time unit should be given for interval literal(line 1, pos 7)== SQL == select interval from mytable ---^^^ at org.apache.spark.sql.errors.QueryParsingErrors$.invalidIntervalLiteralError(QueryParsingErrors.scala:196) .. ``` It is a bug because "Non-Reserved keywords" have a special meaning in particular contexts and can be used as identifiers in other contexts. So by design, INTERVAL can be used as a column name. Currently the interval's grammar is ``` interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? ; ``` There is no need to make the time unit nullable, we can ensure "at least one time unit should be given for interval literal" if the interval's grammar is ``` interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test: PlanParsserSuite."SPARK-42553: NonReserved keyword 'interval' can be column name" Local test ```shell scala> val myDF = spark.sparkContext.makeRDD(1 to 5).toDF("interval") myDF: org.apache.spark.sql.DataFrame = [interval: int] scala> myDF.createOrReplaceTempView("mytable") scala> spark.sql("select interval from mytable;").show() ++ |interval| ++ | 1| | 2| | 3| | 4| | 5| ++ ``` Closes #40253 from jiang13021/branch-3.3-42553. Authored-by: jiangyzanze Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/parser/SqlBaseParser.g4| 2 +- .../apache/spark/sql/catalyst/parser/AstBuilder.scala| 5 ++--- .../org/apache/spark/sql/errors/QueryParsingErrors.scala | 4 .../sql/catalyst/parser/ExpressionParserSuite.scala | 3 --- .../spark/sql/catalyst/parser/PlanParserSuite.scala | 9 + .../src/test/resources/sql-tests/inputs/interval.sql | 1 - .../resources/sql-tests/results/ansi/interval.sql.out| 16 +--- .../test/resources/sql-tests/results/interval.sql.out| 16 +--- 8 files changed, 14 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 701d4bc5aa7..3ec4b9a833d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -875,7 +875,7 @@ booleanValue ; interval -: INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? +: INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; errorCapturingMultiUnitsInterval 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 ecc5360a4f7..4152e24d3e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2551,15 +2551,14 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit innerCtx.unitToUnitInterval) } visitMultiUnitsInterval(innerCtx.multiUnitsInterval) -} else if (ctx.errorCapturingUnitToUnitInterval != null) { +} else { + assert(ctx.errorCapturingUnitToUnitInterval != null) val innerCtx = ctx.errorCapturingUnitToUnitInterval
[spark] branch branch-3.4 updated: [SPARK-42553][SQL] Ensure at least one time unit after "interval"
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 39f220c5410 [SPARK-42553][SQL] Ensure at least one time unit after "interval" 39f220c5410 is described below commit 39f220c54108e82e0915d32bd173cd7f67f08cbf Author: jiangyzanze AuthorDate: Thu Mar 2 09:37:25 2023 +0300 [SPARK-42553][SQL] Ensure at least one time unit after "interval" ### What changes were proposed in this pull request? THis PR aims to ensure "at least one time unit should be given for interval literal" by modifying SqlBaseParser ### Why are the changes needed? INTERVAL is a Non-Reserved keyword in spark. But when I run ```shell scala> spark.sql("select interval from mytable") ``` I get ``` org.apache.spark.sql.catalyst.parser.ParseException: at least one time unit should be given for interval literal(line 1, pos 7)== SQL == select interval from mytable ---^^^ at org.apache.spark.sql.errors.QueryParsingErrors$.invalidIntervalLiteralError(QueryParsingErrors.scala:196) .. ``` It is a bug because "Non-Reserved keywords" have a special meaning in particular contexts and can be used as identifiers in other contexts. So by design, INTERVAL can be used as a column name. Currently the interval's grammar is ``` interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? ; ``` There is no need to make the time unit nullable, we can ensure "at least one time unit should be given for interval literal" if the interval's grammar is ``` interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? local test ```shell scala> val myDF = spark.sparkContext.makeRDD(1 to 5).toDF("interval") myDF: org.apache.spark.sql.DataFrame = [interval: int] scala> myDF.createOrReplaceTempView("mytable") scala> spark.sql("select interval from mytable;").show() ++ |interval| ++ | 1| | 2| | 3| | 4| | 5| ++ ``` Closes #40195 from jiang13021/jiang13021_fix_42553. Authored-by: jiangyzanze Signed-off-by: Max Gekk (cherry picked from commit 738a81e3173bd6571e038196fc161737ca105f58) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../apache/spark/sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 5 ++--- .../apache/spark/sql/errors/QueryParsingErrors.scala | 4 .../sql/catalyst/parser/ExpressionParserSuite.scala| 10 -- .../spark/sql/catalyst/parser/PlanParserSuite.scala| 9 + .../src/test/resources/sql-tests/inputs/interval.sql | 1 - .../resources/sql-tests/results/ansi/interval.sql.out | 18 -- .../test/resources/sql-tests/results/interval.sql.out | 18 -- 9 files changed, 12 insertions(+), 60 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 28d213db93c..2780c98bfc6 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -2017,11 +2017,6 @@ "Can only have a single from-to unit in the interval literal syntax." ] }, - "_LEGACY_ERROR_TEMP_0025" : { -"message" : [ - "At least one time unit should be given for interval literal." -] - }, "_LEGACY_ERROR_TEMP_0026" : { "message" : [ "Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: ." diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index aa5f538bbf6..6142700f095 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -955,7 +955,7 @@ booleanValue ; interval -: INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? +: INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; errorCapturingMultiUnitsInterval diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/pars
[spark] branch master updated (41d3103f4d6 -> 738a81e3173)
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 41d3103f4d6 [SPARK-41823][CONNECT][FOLLOW-UP][TESTS] Disable ANSI mode in ProtoToParsedPlanTestSuite add 738a81e3173 [SPARK-42553][SQL] Ensure at least one time unit after "interval" No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 5 - .../apache/spark/sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 5 ++--- .../apache/spark/sql/errors/QueryParsingErrors.scala | 4 .../sql/catalyst/parser/ExpressionParserSuite.scala| 10 -- .../spark/sql/catalyst/parser/PlanParserSuite.scala| 9 + .../src/test/resources/sql-tests/inputs/interval.sql | 1 - .../resources/sql-tests/results/ansi/interval.sql.out | 18 -- .../test/resources/sql-tests/results/interval.sql.out | 18 -- 9 files changed, 12 insertions(+), 60 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-42337][SQL][FOLLOWUP] Update the error message for INVALID_TEMP_OBJ_REFERENCE
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 de56622347e [SPARK-42337][SQL][FOLLOWUP] Update the error message for INVALID_TEMP_OBJ_REFERENCE de56622347e is described below commit de56622347e0cd3be17442e472aee0c13f8551f3 Author: allisonwang-db AuthorDate: Wed Mar 1 07:55:02 2023 +0300 [SPARK-42337][SQL][FOLLOWUP] Update the error message for INVALID_TEMP_OBJ_REFERENCE ### What changes were proposed in this pull request? This PR is a follow-up for #39910. It updates the error message of the error class INVALID_TEMP_OBJ_REFERENCE. ### Why are the changes needed? To make the error message more user-friendly. ### Does this PR introduce _any_ user-facing change? Yes. This PR updates the error message for INVALID_TEMP_OBJ_REFERENCE. ### How was this patch tested? Existing tests Closes #40198 from allisonwang-db/spark-42337-follow-up. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 2 +- .../spark/sql/errors/QueryCompilationErrors.scala | 8 ++-- .../results/postgreSQL/create_view.sql.out | 52 +++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 8 ++-- .../spark/sql/connector/DataSourceV2SQLSuite.scala | 4 +- .../apache/spark/sql/execution/SQLViewSuite.scala | 8 ++-- .../spark/sql/execution/SQLViewTestSuite.scala | 8 ++-- .../sql/hive/execution/HiveSQLViewSuite.scala | 4 +- 8 files changed, 47 insertions(+), 47 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 408c97acaa3..3dba15c6584 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1041,7 +1041,7 @@ }, "INVALID_TEMP_OBJ_REFERENCE" : { "message" : [ - "Cannot create a persistent by referencing a temporary . Please make the temporary persistent, or make the persistent temporary." + "Cannot create the persistent object of the type because it references to the temporary object of the type . Please make the temporary object persistent, or make the persistent object temporary." ] }, "INVALID_TYPED_LITERAL" : { 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 1c257966aaf..e4475980cf9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2692,9 +2692,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "INVALID_TEMP_OBJ_REFERENCE", messageParameters = Map( -"obj" -> "view", +"obj" -> "VIEW", "objName" -> toSQLId(name.nameParts), -"tempObj" -> "view", +"tempObj" -> "VIEW", "tempObjName" -> toSQLId(nameParts))) } @@ -2704,9 +2704,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new AnalysisException( errorClass = "INVALID_TEMP_OBJ_REFERENCE", messageParameters = Map( -"obj" -> "view", +"obj" -> "VIEW", "objName" -> toSQLId(name.nameParts), -"tempObj" -> "function", +"tempObj" -> "FUNCTION", "tempObjName" -> toSQLId(funcName))) } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index 2768f7b53b4..59e574c27e3 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -280,9 +280,9 @@ org.apache.spark.sql.AnalysisException { "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", "messageParameters" : { -"obj" : "view", +"obj" : "VIEW", "objName" : "`spark_catalog`.`temp_view_test`.`v1_temp`", -"tempObj" : "view", +"tempObj" : "VIEW", "tempObjName" : "`temp_table`" } } @@ -344,9 +344,9 @@ org.apache.spark.sql.AnalysisExcepti
[spark] branch branch-3.3 updated: [SPARK-42516][SQL] Always capture the session time zone config while creating views
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 82fe3947288 [SPARK-42516][SQL] Always capture the session time zone config while creating views 82fe3947288 is described below commit 82fe39472886da5bf811f158478952cd99044031 Author: Max Gekk AuthorDate: Wed Feb 22 14:03:20 2023 +0300 [SPARK-42516][SQL] Always capture the session time zone config while creating views ### What changes were proposed in this pull request? In the PR, I propose to capture the session time zone config (`spark.sql.session.timeZone`) as a view property, and use it while re-parsing/analysing the view. If the SQL config is not set while creating a view, use the default value of the config. ### Why are the changes needed? To improve user experience with Spark SQL. The current behaviour might confuse users because query results depends on whether or not the session time zone was set explicitly while creating a view. ### Does this PR introduce _any_ user-facing change? Yes. Before the changes, the current value of the session time zone is used in view analysis but this behaviour can be restored via another SQL config `spark.sql.legacy.useCurrentConfigsForView`. ### How was this patch tested? By running the new test via: ``` $ build/sbt "test:testOnly *.PersistedViewTestSuite" ``` Closes #40103 from MaxGekk/view-tz-conf. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit 00e56905f77955f67e3809d724b33aebcc79cb5e) Signed-off-by: Max Gekk --- .../org/apache/spark/sql/execution/command/views.scala | 9 - .../apache/spark/sql/execution/SQLViewTestSuite.scala | 17 + 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index eca48a69924..4173db3092a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -398,8 +398,15 @@ object ViewHelper extends SQLConfHelper with Logging { val modifiedConfs = conf.getAllConfs.filter { case (k, _) => conf.isModifiable(k) && shouldCaptureConfig(k) } +// Some configs have dynamic default values, such as SESSION_LOCAL_TIMEZONE whose +// default value relies on the JVM system timezone. We need to always capture them to +// to make sure we apply the same configs when reading the view. +val alwaysCaptured = Seq(SQLConf.SESSION_LOCAL_TIMEZONE) + .filter(c => !modifiedConfs.contains(c.key)) + .map(c => (c.key, conf.getConf(c))) + val props = new mutable.HashMap[String, String] -for ((key, value) <- modifiedConfs) { +for ((key, value) <- modifiedConfs ++ alwaysCaptured) { props.put(s"$VIEW_SQL_CONFIG_PREFIX$key", value) } props.toMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 316b1cfd5e8..1874aa15f8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogFunction import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.Repartition +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.withDefaultTimeZone import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -678,6 +679,22 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { } } + test("capture the session time zone config while creating a view") { +val viewName = "v1_capture_test" +withView(viewName) { + assert(get.sessionLocalTimeZone === "America/Los_Angeles") + createView(viewName, +"""select hour(ts) as H from ( + | select cast('2022-01-01T00:00:00.000 America/Los_Angeles' as timestamp) as ts + |)""".stripMargin, Seq("H")) + withDefaultTimeZone(java.time.ZoneId.of("UTC-09:00")) { +withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "UTC-10:00") { + checkAnswer(sql(s"select H from $viewName"), Row(0)) +} + } +} +
[spark] branch branch-3.4 updated: [SPARK-42516][SQL] Always capture the session time zone config while creating views
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 033eb37f12e [SPARK-42516][SQL] Always capture the session time zone config while creating views 033eb37f12e is described below commit 033eb37f12e3753ea22eb9a069f716981cc58be1 Author: Max Gekk AuthorDate: Wed Feb 22 14:03:20 2023 +0300 [SPARK-42516][SQL] Always capture the session time zone config while creating views ### What changes were proposed in this pull request? In the PR, I propose to capture the session time zone config (`spark.sql.session.timeZone`) as a view property, and use it while re-parsing/analysing the view. If the SQL config is not set while creating a view, use the default value of the config. ### Why are the changes needed? To improve user experience with Spark SQL. The current behaviour might confuse users because query results depends on whether or not the session time zone was set explicitly while creating a view. ### Does this PR introduce _any_ user-facing change? Yes. Before the changes, the current value of the session time zone is used in view analysis but this behaviour can be restored via another SQL config `spark.sql.legacy.useCurrentConfigsForView`. ### How was this patch tested? By running the new test via: ``` $ build/sbt "test:testOnly *.PersistedViewTestSuite" ``` Closes #40103 from MaxGekk/view-tz-conf. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit 00e56905f77955f67e3809d724b33aebcc79cb5e) Signed-off-by: Max Gekk --- .../org/apache/spark/sql/execution/command/views.scala | 9 - .../apache/spark/sql/execution/SQLViewTestSuite.scala | 17 + 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 3ad98fa0d0c..f998e134a0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -398,8 +398,15 @@ object ViewHelper extends SQLConfHelper with Logging { val modifiedConfs = conf.getAllConfs.filter { case (k, _) => conf.isModifiable(k) && shouldCaptureConfig(k) } +// Some configs have dynamic default values, such as SESSION_LOCAL_TIMEZONE whose +// default value relies on the JVM system timezone. We need to always capture them to +// to make sure we apply the same configs when reading the view. +val alwaysCaptured = Seq(SQLConf.SESSION_LOCAL_TIMEZONE) + .filter(c => !modifiedConfs.contains(c.key)) + .map(c => (c.key, conf.getConf(c))) + val props = new mutable.HashMap[String, String] -for ((key, value) <- modifiedConfs) { +for ((key, value) <- modifiedConfs ++ alwaysCaptured) { props.put(s"$VIEW_SQL_CONFIG_PREFIX$key", value) } props.toMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 1d4c52d3ae5..592f1c2607d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogFunction import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.Repartition +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.withDefaultTimeZone import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf._ @@ -706,6 +707,22 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { } } + test("capture the session time zone config while creating a view") { +val viewName = "v1_capture_test" +withView(viewName) { + assert(get.sessionLocalTimeZone === "America/Los_Angeles") + createView(viewName, +"""select hour(ts) as H from ( + | select cast('2022-01-01T00:00:00.000 America/Los_Angeles' as timestamp) as ts + |)""".stripMargin, Seq("H")) + withDefaultTimeZone(java.time.ZoneId.of("UTC-09:00")) { +withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "UTC-10:00") { + checkAnswer(sql(s"select H from $viewName"), Row(0)) +} +
[spark] branch master updated: [SPARK-42516][SQL] Always capture the session time zone config while creating views
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 00e56905f77 [SPARK-42516][SQL] Always capture the session time zone config while creating views 00e56905f77 is described below commit 00e56905f77955f67e3809d724b33aebcc79cb5e Author: Max Gekk AuthorDate: Wed Feb 22 14:03:20 2023 +0300 [SPARK-42516][SQL] Always capture the session time zone config while creating views ### What changes were proposed in this pull request? In the PR, I propose to capture the session time zone config (`spark.sql.session.timeZone`) as a view property, and use it while re-parsing/analysing the view. If the SQL config is not set while creating a view, use the default value of the config. ### Why are the changes needed? To improve user experience with Spark SQL. The current behaviour might confuse users because query results depends on whether or not the session time zone was set explicitly while creating a view. ### Does this PR introduce _any_ user-facing change? Yes. Before the changes, the current value of the session time zone is used in view analysis but this behaviour can be restored via another SQL config `spark.sql.legacy.useCurrentConfigsForView`. ### How was this patch tested? By running the new test via: ``` $ build/sbt "test:testOnly *.PersistedViewTestSuite" ``` Closes #40103 from MaxGekk/view-tz-conf. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../org/apache/spark/sql/execution/command/views.scala | 9 - .../apache/spark/sql/execution/SQLViewTestSuite.scala | 17 + 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 3ad98fa0d0c..f998e134a0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -398,8 +398,15 @@ object ViewHelper extends SQLConfHelper with Logging { val modifiedConfs = conf.getAllConfs.filter { case (k, _) => conf.isModifiable(k) && shouldCaptureConfig(k) } +// Some configs have dynamic default values, such as SESSION_LOCAL_TIMEZONE whose +// default value relies on the JVM system timezone. We need to always capture them to +// to make sure we apply the same configs when reading the view. +val alwaysCaptured = Seq(SQLConf.SESSION_LOCAL_TIMEZONE) + .filter(c => !modifiedConfs.contains(c.key)) + .map(c => (c.key, conf.getConf(c))) + val props = new mutable.HashMap[String, String] -for ((key, value) <- modifiedConfs) { +for ((key, value) <- modifiedConfs ++ alwaysCaptured) { props.put(s"$VIEW_SQL_CONFIG_PREFIX$key", value) } props.toMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index f64be6fcd2c..34dfdf12357 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogFunction import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.Repartition +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.withDefaultTimeZone import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf._ @@ -714,6 +715,22 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession { } } + test("capture the session time zone config while creating a view") { +val viewName = "v1_capture_test" +withView(viewName) { + assert(get.sessionLocalTimeZone === "America/Los_Angeles") + createView(viewName, +"""select hour(ts) as H from ( + | select cast('2022-01-01T00:00:00.000 America/Los_Angeles' as timestamp) as ts + |)""".stripMargin, Seq("H")) + withDefaultTimeZone(java.time.ZoneId.of("UTC-09:00")) { +withSQLConf(SESSION_LOCAL_TIMEZONE.key -> "UTC-10:00") { + checkAnswer(sql(s"select H from $viewName"), Row(0)) +} + } +} + } + def getShowCreateDDL(view: String, serde: Boolean = false): String = { val result = if (
[spark] branch master updated: [SPARK-42323][SQL] Assign name to `_LEGACY_ERROR_TEMP_2332`
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 21a5b416a6a [SPARK-42323][SQL] Assign name to `_LEGACY_ERROR_TEMP_2332` 21a5b416a6a is described below commit 21a5b416a6a16fe6d960cc04931f5cbf858954ae Author: itholic AuthorDate: Sun Feb 19 13:25:46 2023 +0300 [SPARK-42323][SQL] Assign name to `_LEGACY_ERROR_TEMP_2332` ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2332, "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39977 from itholic/LEGACY_2332. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 10 - .../spark/sql/execution/datasources/rules.scala| 7 --- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 24 ++ .../spark/sql/hive/execution/SQLQuerySuite.scala | 24 ++ 4 files changed, 41 insertions(+), 24 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8cf46cb7aad..8910ca86de4 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1578,6 +1578,11 @@ ], "sqlState" : "0A000" }, + "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY" : { +"message" : [ + "Unsupported data source type for direct query on files: " +] + }, "UNSUPPORTED_DATATYPE" : { "message" : [ "Unsupported data type ." @@ -5107,11 +5112,6 @@ "failed to evaluate expression : " ] }, - "_LEGACY_ERROR_TEMP_2332" : { -"message" : [ - "" -] - }, "_LEGACY_ERROR_TEMP_2400" : { "message" : [ "The expression must evaluate to a constant value, but got ." 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/datasources/rules.scala index dc4fed49c1c..9255aa2effc 100644 --- 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/datasources/rules.scala @@ -78,9 +78,10 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { case e: Exception => // the provider is valid, but failed to create a logical plan u.failAnalysis( -errorClass = "_LEGACY_ERROR_TEMP_2332", -messageParameters = Map("msg" -> e.getMessage), -cause = e) +errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", +messageParameters = Map("dataSourceType" -> u.multipartIdentifier.head), +cause = e + ) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 20fa5fee3aa..377be2e720a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1634,15 +1634,23 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkErrorTableNotFound(e, "`no_db`.`no_table`", ExpectedContext("no_db.no_table", 14, 13 + "no_db.no_table".length)) -e = intercept[AnalysisException] { - sql("select * from json.invalid_file") -} -assert(e.message.contains("Path does not exist")) +checkError( + exception = intercept[AnalysisException] { +sql("select * from json.invalid_file") + }, + errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", + parameters = Map("dataSourceType" -> "json"), + context = ExpectedContext("json.invalid_file", 14, 30) +) -e = intercept[AnalysisException] { - sql(s"select id from `org.apache.spark.sql.hive.orc`.`file_path`") -} -assert(e.message.contains("Hive built-in ORC data source must be used with Hive support")) +checkError( + exception = intercept[AnalysisException] { +sql(s"select id from `org.apache.spark.sq
[spark] branch master updated: [SPARK-42430][DOC][FOLLOW-UP] Revise the java doc for TimestampNTZ & ANSI interval types
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 8cfd5bf1ca4 [SPARK-42430][DOC][FOLLOW-UP] Revise the java doc for TimestampNTZ & ANSI interval types 8cfd5bf1ca4 is described below commit 8cfd5bf1ca4042541232ef1787349ddb876adcfa Author: Gengliang Wang AuthorDate: Sat Feb 18 10:30:32 2023 +0300 [SPARK-42430][DOC][FOLLOW-UP] Revise the java doc for TimestampNTZ & ANSI interval types ### What changes were proposed in this pull request? As https://github.com/apache/spark/pull/40005#pullrequestreview-1299089504 pointed out, the java doc for data type recommends using factory methods provided in org.apache.spark.sql.types.DataTypes. Since the ANSI interval types missed the `DataTypes` as well, this PR also revise their doc. ### Why are the changes needed? Unify the data type doc ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Local preview https://user-images.githubusercontent.com/1097932/219821685-321c2fd1-6248-4930-9c61-eec68f0dcb50.png;> Closes #40074 from gengliangwang/reviseNTZDoc. Authored-by: Gengliang Wang Signed-off-by: Max Gekk --- docs/sql-ref-datatypes.md | 8 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 3095a196a35..64c6d335429 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -128,7 +128,7 @@ You can access them by doing |**BinaryType**|Array[Byte]|BinaryType| |**BooleanType**|Boolean|BooleanType| |**TimestampType**|java.sql.Timestamp|TimestampType| -|**TimestampNTZType**|java.time.LocalDateTime| TimestampNTZType| +|**TimestampNTZType**|java.time.LocalDateTime|TimestampNTZType| |**DateType**|java.sql.Date|DateType| |**YearMonthIntervalType**|java.time.Period|YearMonthIntervalType| |**DayTimeIntervalType**|java.time.Duration|DayTimeIntervalType| @@ -159,10 +159,10 @@ please use factory methods provided in |**BinaryType**|byte[]|DataTypes.BinaryType| |**BooleanType**|boolean or Boolean|DataTypes.BooleanType| |**TimestampType**|java.sql.Timestamp|DataTypes.TimestampType| -|**TimestampNTZType**|java.time.LocalDateTime| TimestampNTZType| +|**TimestampNTZType**|java.time.LocalDateTime|DataTypes.TimestampNTZType| |**DateType**|java.sql.Date|DataTypes.DateType| -|**YearMonthIntervalType**|java.time.Period|YearMonthIntervalType| -|**DayTimeIntervalType**|java.time.Duration|DayTimeIntervalType| +|**YearMonthIntervalType**|java.time.Period|DataTypes.YearMonthIntervalType| +|**DayTimeIntervalType**|java.time.Duration|DataTypes.DayTimeIntervalType| |**ArrayType**|java.util.List|DataTypes.createArrayType(*elementType*)**Note:** The value of *containsNull* will be true.DataTypes.createArrayType(*elementType*, *containsNull*).| |**MapType**|java.util.Map|DataTypes.createMapType(*keyType*, *valueType*)**Note:** The value of *valueContainsNull* will be true.DataTypes.createMapType(*keyType*, *valueType*, *valueContainsNull*)| |**StructType**|org.apache.spark.sql.Row|DataTypes.createStructType(*fields*)**Note:** *fields* is a List or an array of StructFields.Also, two fields with the same name are not allowed.| - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-42430][DOC][FOLLOW-UP] Revise the java doc for TimestampNTZ & ANSI interval types
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 2b54f076794 [SPARK-42430][DOC][FOLLOW-UP] Revise the java doc for TimestampNTZ & ANSI interval types 2b54f076794 is described below commit 2b54f0767949355178c2a184d6136a56a3377c85 Author: Gengliang Wang AuthorDate: Sat Feb 18 10:30:32 2023 +0300 [SPARK-42430][DOC][FOLLOW-UP] Revise the java doc for TimestampNTZ & ANSI interval types ### What changes were proposed in this pull request? As https://github.com/apache/spark/pull/40005#pullrequestreview-1299089504 pointed out, the java doc for data type recommends using factory methods provided in org.apache.spark.sql.types.DataTypes. Since the ANSI interval types missed the `DataTypes` as well, this PR also revise their doc. ### Why are the changes needed? Unify the data type doc ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Local preview https://user-images.githubusercontent.com/1097932/219821685-321c2fd1-6248-4930-9c61-eec68f0dcb50.png;> Closes #40074 from gengliangwang/reviseNTZDoc. Authored-by: Gengliang Wang Signed-off-by: Max Gekk (cherry picked from commit 8cfd5bf1ca4042541232ef1787349ddb876adcfa) Signed-off-by: Max Gekk --- docs/sql-ref-datatypes.md | 8 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 3095a196a35..64c6d335429 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -128,7 +128,7 @@ You can access them by doing |**BinaryType**|Array[Byte]|BinaryType| |**BooleanType**|Boolean|BooleanType| |**TimestampType**|java.sql.Timestamp|TimestampType| -|**TimestampNTZType**|java.time.LocalDateTime| TimestampNTZType| +|**TimestampNTZType**|java.time.LocalDateTime|TimestampNTZType| |**DateType**|java.sql.Date|DateType| |**YearMonthIntervalType**|java.time.Period|YearMonthIntervalType| |**DayTimeIntervalType**|java.time.Duration|DayTimeIntervalType| @@ -159,10 +159,10 @@ please use factory methods provided in |**BinaryType**|byte[]|DataTypes.BinaryType| |**BooleanType**|boolean or Boolean|DataTypes.BooleanType| |**TimestampType**|java.sql.Timestamp|DataTypes.TimestampType| -|**TimestampNTZType**|java.time.LocalDateTime| TimestampNTZType| +|**TimestampNTZType**|java.time.LocalDateTime|DataTypes.TimestampNTZType| |**DateType**|java.sql.Date|DataTypes.DateType| -|**YearMonthIntervalType**|java.time.Period|YearMonthIntervalType| -|**DayTimeIntervalType**|java.time.Duration|DayTimeIntervalType| +|**YearMonthIntervalType**|java.time.Period|DataTypes.YearMonthIntervalType| +|**DayTimeIntervalType**|java.time.Duration|DataTypes.DayTimeIntervalType| |**ArrayType**|java.util.List|DataTypes.createArrayType(*elementType*)**Note:** The value of *containsNull* will be true.DataTypes.createArrayType(*elementType*, *containsNull*).| |**MapType**|java.util.Map|DataTypes.createMapType(*keyType*, *valueType*)**Note:** The value of *valueContainsNull* will be true.DataTypes.createMapType(*keyType*, *valueType*, *valueContainsNull*)| |**StructType**|org.apache.spark.sql.Row|DataTypes.createStructType(*fields*)**Note:** *fields* is a List or an array of StructFields.Also, two fields with the same name are not allowed.| - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-42337][SQL] Add error class INVALID_TEMP_OBJ_REFERENCE
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 f99580e6f87 [SPARK-42337][SQL] Add error class INVALID_TEMP_OBJ_REFERENCE f99580e6f87 is described below commit f99580e6f875e5dd93b30cdb51e3b7504077aa5f Author: allisonwang-db AuthorDate: Sat Feb 18 10:17:26 2023 +0300 [SPARK-42337][SQL] Add error class INVALID_TEMP_OBJ_REFERENCE ### What changes were proposed in this pull request? This PR adds a new error class `INVALID_TEMP_OBJ_REFERENCE ` and replaces two existing error classes with this new one: - _LEGACY_ERROR_TEMP_1283 - _LEGACY_ERROR_TEMP_1284 ### Why are the changes needed? To improve the error messages. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes a user-facing error message. ### How was this patch tested? Existing unit tests. Closes #39910 from allisonwang-db/spark-42337-persistent-over-temp-err. Authored-by: allisonwang-db Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 15 +-- .../spark/sql/errors/QueryCompilationErrors.scala | 16 ++-- .../results/postgreSQL/create_view.sql.out | 104 + .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 22 +++-- .../spark/sql/connector/DataSourceV2SQLSuite.scala | 8 +- .../apache/spark/sql/execution/SQLViewSuite.scala | 34 --- .../spark/sql/execution/SQLViewTestSuite.scala | 32 --- .../sql/hive/execution/HiveSQLViewSuite.scala | 13 ++- 8 files changed, 151 insertions(+), 93 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 6d9aa59e57f..8cf46cb7aad 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1034,6 +1034,11 @@ }, "sqlState" : "42823" }, + "INVALID_TEMP_OBJ_REFERENCE" : { +"message" : [ + "Cannot create a persistent by referencing a temporary . Please make the temporary persistent, or make the persistent temporary." +] + }, "INVALID_TYPED_LITERAL" : { "message" : [ "The value of the typed literal is invalid: ." @@ -3343,16 +3348,6 @@ "Not allowed to create a permanent view without explicitly assigning an alias for expression ." ] }, - "_LEGACY_ERROR_TEMP_1283" : { -"message" : [ - "Not allowed to create a permanent view by referencing a temporary view . Please create a temp view instead by CREATE TEMP VIEW." -] - }, - "_LEGACY_ERROR_TEMP_1284" : { -"message" : [ - "Not allowed to create a permanent view by referencing a temporary function ``." -] - }, "_LEGACY_ERROR_TEMP_1285" : { "message" : [ "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the", 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 9840ebf2b8a..556b3a62da3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2690,20 +2690,24 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { name: TableIdentifier, nameParts: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1283", + errorClass = "INVALID_TEMP_OBJ_REFERENCE", messageParameters = Map( -"name" -> name.toString, -"nameParts" -> nameParts)) +"obj" -> "view", +"objName" -> toSQLId(name.nameParts), +"tempObj" -> "view", +"tempObjName" -> toSQLId(nameParts))) } def notAllowedToCreatePermanentViewByReferencingTempFuncError( name: TableIdentifier, funcName: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1284", + errorClass = "INVALID_TEMP_OBJ_REFERENCE", messageParameters = Map( -"name" -> name.toString, -"funcName" -> funcName)) +"obj" -> "view", +"objName" -> toSQLId(name.nameParts), +"tempObj" -> "function", +"tempObjName" -> toSQLId(funcName))) } def queryF
[spark] branch master updated: [SPARK-38324][SQL] The second range is not [0, 59] in the day time ANSI interval
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 9f7582c8cbe [SPARK-38324][SQL] The second range is not [0, 59] in the day time ANSI interval 9f7582c8cbe is described below commit 9f7582c8cbeae70c31c183567bf5320d1c3210fe Author: haoyan.zhang AuthorDate: Fri Feb 17 09:29:42 2023 +0300 [SPARK-38324][SQL] The second range is not [0, 59] in the day time ANSI interval ### What changes were proposed in this pull request? Throw an error when the second value in day(hour, minute) to second interval out of range [0, 59] ### Why are the changes needed? Currently an invalid second value will not get an error ``` spark.sql("select INTERVAL '10 01:01:99' DAY TO SECOND") DataFrame[INTERVAL '10 01:02:39' DAY TO SECOND: interval day to second]{} ``` But minute range check is ok ``` spark.sql("select INTERVAL '10 01:60:01' DAY TO SECOND") requirement failed: minute 60 outside range [0, 59](line 1, pos 16) ``` ### We need check second value too Does this PR introduce any user-facing change? no ### How was this patch tested? New unit tests. Closes #40033 from haoyanzhang/master. Authored-by: haoyan.zhang Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/util/IntervalUtils.scala | 13 - .../spark/sql/catalyst/util/IntervalUtilsSuite.scala | 18 ++ 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7b574e987d9..455a74e06c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -381,7 +381,7 @@ object IntervalUtils { micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR) val minutes = toLongWithRange(minuteStr, minute, 0, 59) micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE) -micros = Math.addExact(micros, sign * parseSecondNano(second)) +micros = Math.addExact(micros, sign * parseSecondNano(second, 0, 59)) micros } @@ -391,7 +391,7 @@ object IntervalUtils { micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR) val minutes = toLongWithRange(minuteStr, minute, 0, 59) micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE) -micros = Math.addExact(micros, sign * parseSecondNano(second)) +micros = Math.addExact(micros, sign * parseSecondNano(second, 0, 59)) micros } @@ -399,7 +399,7 @@ object IntervalUtils { var micros = 0L val minutes = toLongWithRange(minuteStr, minute, 0, MAX_MINUTE) micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE) -micros = Math.addExact(micros, sign * parseSecondNano(second)) +micros = Math.addExact(micros, sign * parseSecondNano(second, 0, 59)) micros } @@ -549,9 +549,12 @@ object IntervalUtils { /** * Parse second_nano string in ss.n format to microseconds */ - private def parseSecondNano(secondNano: String): Long = { + private def parseSecondNano( + secondNano: String, + minSecond: Long = MIN_SECOND, + maxSecond: Long = MAX_SECOND): Long = { def parseSeconds(secondsStr: String): Long = { - toLongWithRange(secondStr, secondsStr, MIN_SECOND, MAX_SECOND) * MICROS_PER_SECOND + toLongWithRange(secondStr, secondsStr, minSecond, maxSecond) * MICROS_PER_SECOND } secondNano.split("\\.") match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 0e65886a2eb..3ba6787045e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -665,4 +665,22 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { assert(toYearMonthIntervalString(months, ANSI_STYLE, MONTH, MONTH) === month) } } + + test("SPARK-38324: The second range is not [0, 59] in the day time ANSI interval") { +import org.apache.spark.sql.types.DayTimeIntervalType._ +Seq( + ("10 12:40:60", 60, DAY, SECOND), + ("10 12:40:60.9", 60, DAY, SECOND), + ("10 12:40:99.9", 99, DAY, SECOND), + ("12:40:60", 60, HOUR, SECOND), + ("12:40:60.9", 60, HOUR, SECOND), + ("12:40:99.9", 99
[spark] branch branch-3.4 updated: [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE`
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 7e2642db062 [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE` 7e2642db062 is described below commit 7e2642db062cc45c44cbf549d6431bd72915fa17 Author: itholic AuthorDate: Thu Feb 16 22:04:17 2023 +0300 [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE` ### What changes were proposed in this pull request? This PR proposes to integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE`. And also introduce new error class `UNSUPPORTED_ARROWTYPE`. ### Why are the changes needed? We should assign proper name for LEGACY errors. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Updated UT. Closes #39979 from itholic/LEGACY_2099. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 9855b137032bf9504dff96eb5bb9951accacac0f) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json| 11 ++- .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 13 ++--- .../main/scala/org/apache/spark/sql/util/ArrowUtils.scala | 4 ++-- .../scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala | 9 - .../scala/org/apache/spark/sql/execution/Columnar.scala | 2 +- .../org/apache/spark/sql/execution/arrow/ArrowWriter.scala | 2 +- .../spark/sql/execution/arrow/ArrowConvertersSuite.scala| 10 ++ 7 files changed, 34 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 770223625cf..a0970550d72 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1567,6 +1567,12 @@ ], "sqlState" : "42703" }, + "UNSUPPORTED_ARROWTYPE" : { +"message" : [ + "Unsupported arrow type ." +], +"sqlState" : "0A000" + }, "UNSUPPORTED_DATATYPE" : { "message" : [ "Unsupported data type ." @@ -4094,11 +4100,6 @@ "Could not compare cost with ." ] }, - "_LEGACY_ERROR_TEMP_2099" : { -"message" : [ - "Unsupported data type: ." -] - }, "_LEGACY_ERROR_TEMP_2100" : { "message" : [ "not support type: ." 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 2bafa2e2c03..17c5b2f4f10 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 @@ -25,6 +25,7 @@ import java.time.temporal.ChronoField import java.util.concurrent.TimeoutException import com.fasterxml.jackson.core.{JsonParser, JsonToken} +import org.apache.arrow.vector.types.pojo.ArrowType import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, Path} import org.apache.hadoop.fs.permission.FsPermission import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} @@ -1124,10 +1125,16 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("cost" -> cost)) } - def unsupportedDataTypeError(dt: String): SparkUnsupportedOperationException = { + def unsupportedArrowTypeError(typeName: ArrowType): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2099", - messageParameters = Map("dt" -> dt)) + errorClass = "UNSUPPORTED_ARROWTYPE", + messageParameters = Map("typeName" -> typeName.toString)) + } + + def unsupportedDataTypeError(typeName: DataType): SparkUnsupportedOperationException = { +new SparkUnsupportedOperationException( + errorClass = "UNSUPPORTED_DATATYPE", + messageParameters = Map("typeName" -> toSQLType(typeName))) } def notSupportTypeError(dataType: DataType): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index e854eba0383..6c6635bac57 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -56,7 +56,7 @@ private[sql] object Ar
[spark] branch master updated: [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE`
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 9855b137032 [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE` 9855b137032 is described below commit 9855b137032bf9504dff96eb5bb9951accacac0f Author: itholic AuthorDate: Thu Feb 16 22:04:17 2023 +0300 [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE` ### What changes were proposed in this pull request? This PR proposes to integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE`. And also introduce new error class `UNSUPPORTED_ARROWTYPE`. ### Why are the changes needed? We should assign proper name for LEGACY errors. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Updated UT. Closes #39979 from itholic/LEGACY_2099. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json| 11 ++- .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 13 ++--- .../main/scala/org/apache/spark/sql/util/ArrowUtils.scala | 4 ++-- .../scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala | 9 - .../scala/org/apache/spark/sql/execution/Columnar.scala | 2 +- .../org/apache/spark/sql/execution/arrow/ArrowWriter.scala | 2 +- .../spark/sql/execution/arrow/ArrowConvertersSuite.scala| 10 ++ 7 files changed, 34 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 01fd43eec7f..6d9aa59e57f 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1567,6 +1567,12 @@ ], "sqlState" : "42703" }, + "UNSUPPORTED_ARROWTYPE" : { +"message" : [ + "Unsupported arrow type ." +], +"sqlState" : "0A000" + }, "UNSUPPORTED_DATATYPE" : { "message" : [ "Unsupported data type ." @@ -4094,11 +4100,6 @@ "Could not compare cost with ." ] }, - "_LEGACY_ERROR_TEMP_2099" : { -"message" : [ - "Unsupported data type: ." -] - }, "_LEGACY_ERROR_TEMP_2100" : { "message" : [ "not support type: ." 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 2bafa2e2c03..17c5b2f4f10 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 @@ -25,6 +25,7 @@ import java.time.temporal.ChronoField import java.util.concurrent.TimeoutException import com.fasterxml.jackson.core.{JsonParser, JsonToken} +import org.apache.arrow.vector.types.pojo.ArrowType import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, Path} import org.apache.hadoop.fs.permission.FsPermission import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} @@ -1124,10 +1125,16 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("cost" -> cost)) } - def unsupportedDataTypeError(dt: String): SparkUnsupportedOperationException = { + def unsupportedArrowTypeError(typeName: ArrowType): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2099", - messageParameters = Map("dt" -> dt)) + errorClass = "UNSUPPORTED_ARROWTYPE", + messageParameters = Map("typeName" -> typeName.toString)) + } + + def unsupportedDataTypeError(typeName: DataType): SparkUnsupportedOperationException = { +new SparkUnsupportedOperationException( + errorClass = "UNSUPPORTED_DATATYPE", + messageParameters = Map("typeName" -> toSQLType(typeName))) } def notSupportTypeError(dataType: DataType): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index e854eba0383..6c6635bac57 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -56,7 +56,7 @@ private[sql] object ArrowUtils { case _: YearMonthIntervalType => new ArrowType.Interval(IntervalUnit.YEAR_MONTH) case _
[spark] branch branch-3.4 updated: [SPARK-42324][SQL] Assign name to _LEGACY_ERROR_TEMP_1001
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 4f29e2e16a7 [SPARK-42324][SQL] Assign name to _LEGACY_ERROR_TEMP_1001 4f29e2e16a7 is described below commit 4f29e2e16a76aecd052d1163c2fab3c66fa4bf42 Author: itholic AuthorDate: Tue Feb 14 10:22:35 2023 +0500 [SPARK-42324][SQL] Assign name to _LEGACY_ERROR_TEMP_1001 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_1001, "UNRESOLVED_USING_COLUMN_FOR_JOIN". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39963 from itholic/LEGACY_1001. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 57ae4f49702d200585c56e3f000cca1d729f6f64) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 6 -- .../spark/sql/errors/QueryCompilationErrors.scala | 8 +++ .../analysis/ResolveNaturalJoinSuite.scala | 25 ++ 4 files changed, 30 insertions(+), 20 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index b2da25adebb..770223625cf 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1561,6 +1561,12 @@ ], "sqlState" : "42883" }, + "UNRESOLVED_USING_COLUMN_FOR_JOIN" : { +"message" : [ + "USING column cannot be resolved on the side of the join. The -side columns: []." +], +"sqlState" : "42703" + }, "UNSUPPORTED_DATATYPE" : { "message" : [ "Unsupported data type ." @@ -2175,11 +2181,6 @@ "LEGACY store assignment policy is disallowed in Spark data source V2. Please set the configuration to other values." ] }, - "_LEGACY_ERROR_TEMP_1001" : { -"message" : [ - "USING column `` cannot be resolved on the side of the join. The -side columns: []." -] - }, "_LEGACY_ERROR_TEMP_1002" : { "message" : [ "Unable to generate an encoder for inner class `` without access to the scope that this class was defined in.", 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 eff8c114a97..46cc0b0fbf0 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 @@ -3450,12 +3450,14 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val leftKeys = joinNames.map { keyName => left.output.find(attr => resolver(attr.name, keyName)).getOrElse { -throw QueryCompilationErrors.unresolvedUsingColForJoinError(keyName, left, "left") +throw QueryCompilationErrors.unresolvedUsingColForJoinError( + keyName, left.schema.fieldNames.sorted.map(toSQLId).mkString(", "), "left") } } val rightKeys = joinNames.map { keyName => right.output.find(attr => resolver(attr.name, keyName)).getOrElse { -throw QueryCompilationErrors.unresolvedUsingColForJoinError(keyName, right, "right") +throw QueryCompilationErrors.unresolvedUsingColForJoinError( + keyName, right.schema.fieldNames.sorted.map(toSQLId).mkString(", "), "right") } } val joinPairs = leftKeys.zip(rightKeys) 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 fbcffe04d32..9840ebf2b8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -194,13 +194,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def unresolvedUsingColForJoinError( - colName: String, plan: LogicalPlan, side: String): Throwable = { + colName: String, suggestion: String, side: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1001"
[spark] branch master updated: [SPARK-42324][SQL] Assign name to _LEGACY_ERROR_TEMP_1001
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 57ae4f49702 [SPARK-42324][SQL] Assign name to _LEGACY_ERROR_TEMP_1001 57ae4f49702 is described below commit 57ae4f49702d200585c56e3f000cca1d729f6f64 Author: itholic AuthorDate: Tue Feb 14 10:22:35 2023 +0500 [SPARK-42324][SQL] Assign name to _LEGACY_ERROR_TEMP_1001 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_1001, "UNRESOLVED_USING_COLUMN_FOR_JOIN". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39963 from itholic/LEGACY_1001. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 6 -- .../spark/sql/errors/QueryCompilationErrors.scala | 8 +++ .../analysis/ResolveNaturalJoinSuite.scala | 25 ++ 4 files changed, 30 insertions(+), 20 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index e329932acf1..01fd43eec7f 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1561,6 +1561,12 @@ ], "sqlState" : "42883" }, + "UNRESOLVED_USING_COLUMN_FOR_JOIN" : { +"message" : [ + "USING column cannot be resolved on the side of the join. The -side columns: []." +], +"sqlState" : "42703" + }, "UNSUPPORTED_DATATYPE" : { "message" : [ "Unsupported data type ." @@ -2175,11 +2181,6 @@ "LEGACY store assignment policy is disallowed in Spark data source V2. Please set the configuration to other values." ] }, - "_LEGACY_ERROR_TEMP_1001" : { -"message" : [ - "USING column `` cannot be resolved on the side of the join. The -side columns: []." -] - }, "_LEGACY_ERROR_TEMP_1002" : { "message" : [ "Unable to generate an encoder for inner class `` without access to the scope that this class was defined in.", 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 eff8c114a97..46cc0b0fbf0 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 @@ -3450,12 +3450,14 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor val leftKeys = joinNames.map { keyName => left.output.find(attr => resolver(attr.name, keyName)).getOrElse { -throw QueryCompilationErrors.unresolvedUsingColForJoinError(keyName, left, "left") +throw QueryCompilationErrors.unresolvedUsingColForJoinError( + keyName, left.schema.fieldNames.sorted.map(toSQLId).mkString(", "), "left") } } val rightKeys = joinNames.map { keyName => right.output.find(attr => resolver(attr.name, keyName)).getOrElse { -throw QueryCompilationErrors.unresolvedUsingColForJoinError(keyName, right, "right") +throw QueryCompilationErrors.unresolvedUsingColForJoinError( + keyName, right.schema.fieldNames.sorted.map(toSQLId).mkString(", "), "right") } } val joinPairs = leftKeys.zip(rightKeys) 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 fbcffe04d32..9840ebf2b8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -194,13 +194,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { } def unresolvedUsingColForJoinError( - colName: String, plan: LogicalPlan, side: String): Throwable = { + colName: String, suggestion: String, side: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1001", + errorClass = "UNRESOLVED_USING_COLUMN_FOR_JOIN", messageParameters = Map( -
[spark] branch branch-3.4 updated: [SPARK-42327][SQL] Assign name to `_LEGACY_ERROR_TEMP_2177`
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 8baa84f2055 [SPARK-42327][SQL] Assign name to `_LEGACY_ERROR_TEMP_2177` 8baa84f2055 is described below commit 8baa84f2055ac4191501cbfdf7690d1cba59d163 Author: itholic AuthorDate: Mon Feb 13 21:33:53 2023 +0500 [SPARK-42327][SQL] Assign name to `_LEGACY_ERROR_TEMP_2177` ### What changes were proposed in this pull request? This PR proposes to assign the name `MALFORMED_RECORD_IN_PARSING` to `_LEGACY_ERROR_TEMP_2177` and improve the error message. ### Why are the changes needed? We should assign proper name to LEGACY errors, and show actionable error messages. ### Does this PR introduce _any_ user-facing change? No, but error message improvements. ### How was this patch tested? Updated UTs. Closes #39980 from itholic/LEGACY_2177. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 6f47ac43e55c332f63876cf4f8ecf1b41b277651) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++-- .../sql/catalyst/util/FailureSafeParser.scala | 3 +- .../spark/sql/errors/QueryExecutionErrors.scala| 6 +- .../expressions/JsonExpressionsSuite.scala | 8 ++- .../org/apache/spark/sql/CsvFunctionsSuite.scala | 9 ++- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 66 -- .../sql/execution/datasources/csv/CSVSuite.scala | 6 +- .../sql/execution/datasources/json/JsonSuite.scala | 22 ++-- .../spark/sql/hive/thriftserver/CliSuite.scala | 4 +- .../ThriftServerWithSparkContextSuite.scala| 4 +- 10 files changed, 95 insertions(+), 44 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 86bde20b7fa..b2da25adebb 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1063,6 +1063,12 @@ "Malformed Protobuf messages are detected in message deserialization. Parse Mode: . To process malformed protobuf message as null result, try setting the option 'mode' as 'PERMISSIVE'." ] }, + "MALFORMED_RECORD_IN_PARSING" : { +"message" : [ + "Malformed records are detected in record parsing: .", + "Parse Mode: . To process malformed records as null result, try setting the option 'mode' as 'PERMISSIVE'." +] + }, "MISSING_AGGREGATION" : { "message" : [ "The non-aggregating expression is based on columns which are not participating in the GROUP BY clause.", @@ -4414,11 +4420,6 @@ "Cannot create array with elements of data due to exceeding the limit elements for ArrayData. " ] }, - "_LEGACY_ERROR_TEMP_2177" : { -"message" : [ - "Malformed records are detected in record parsing. Parse Mode: . To process malformed records as null result, try setting the option 'mode' as 'PERMISSIVE'." -] - }, "_LEGACY_ERROR_TEMP_2178" : { "message" : [ "Remote operations not supported." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala index 5a9e52a51a2..fcdcd21b6dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala @@ -65,7 +65,8 @@ class FailureSafeParser[IN]( case DropMalformedMode => Iterator.empty case FailFastMode => - throw QueryExecutionErrors.malformedRecordsDetectedInRecordParsingError(e) + throw QueryExecutionErrors.malformedRecordsDetectedInRecordParsingError( +toResultRow(e.partialResult(), e.record).toString, e) } } } 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 4134da135e3..fd3809ccd31 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 @@ -1743,10 +1743,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "additionalErrorMessage" -> additionalErrorMessage)) } - def malformedRecordsDetectedInRecordParsingError(e: BadRecordException): Throwable = { + def malformedRecordsDetectedInReco
[spark] branch master updated: [SPARK-42327][SQL] Assign name to `_LEGACY_ERROR_TEMP_2177`
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 6f47ac43e55 [SPARK-42327][SQL] Assign name to `_LEGACY_ERROR_TEMP_2177` 6f47ac43e55 is described below commit 6f47ac43e55c332f63876cf4f8ecf1b41b277651 Author: itholic AuthorDate: Mon Feb 13 21:33:53 2023 +0500 [SPARK-42327][SQL] Assign name to `_LEGACY_ERROR_TEMP_2177` ### What changes were proposed in this pull request? This PR proposes to assign the name `MALFORMED_RECORD_IN_PARSING` to `_LEGACY_ERROR_TEMP_2177` and improve the error message. ### Why are the changes needed? We should assign proper name to LEGACY errors, and show actionable error messages. ### Does this PR introduce _any_ user-facing change? No, but error message improvements. ### How was this patch tested? Updated UTs. Closes #39980 from itholic/LEGACY_2177. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++-- .../sql/catalyst/util/FailureSafeParser.scala | 3 +- .../spark/sql/errors/QueryExecutionErrors.scala| 6 +- .../expressions/JsonExpressionsSuite.scala | 8 ++- .../org/apache/spark/sql/CsvFunctionsSuite.scala | 9 ++- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 66 -- .../sql/execution/datasources/csv/CSVSuite.scala | 6 +- .../sql/execution/datasources/json/JsonSuite.scala | 22 ++-- .../spark/sql/hive/thriftserver/CliSuite.scala | 4 +- .../ThriftServerWithSparkContextSuite.scala| 4 +- 10 files changed, 95 insertions(+), 44 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index e96383399d2..e329932acf1 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1063,6 +1063,12 @@ "Malformed Protobuf messages are detected in message deserialization. Parse Mode: . To process malformed protobuf message as null result, try setting the option 'mode' as 'PERMISSIVE'." ] }, + "MALFORMED_RECORD_IN_PARSING" : { +"message" : [ + "Malformed records are detected in record parsing: .", + "Parse Mode: . To process malformed records as null result, try setting the option 'mode' as 'PERMISSIVE'." +] + }, "MISSING_AGGREGATION" : { "message" : [ "The non-aggregating expression is based on columns which are not participating in the GROUP BY clause.", @@ -4414,11 +4420,6 @@ "Cannot create array with elements of data due to exceeding the limit elements for ArrayData. " ] }, - "_LEGACY_ERROR_TEMP_2177" : { -"message" : [ - "Malformed records are detected in record parsing. Parse Mode: . To process malformed records as null result, try setting the option 'mode' as 'PERMISSIVE'." -] - }, "_LEGACY_ERROR_TEMP_2178" : { "message" : [ "Remote operations not supported." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala index 5a9e52a51a2..fcdcd21b6dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala @@ -65,7 +65,8 @@ class FailureSafeParser[IN]( case DropMalformedMode => Iterator.empty case FailFastMode => - throw QueryExecutionErrors.malformedRecordsDetectedInRecordParsingError(e) + throw QueryExecutionErrors.malformedRecordsDetectedInRecordParsingError( +toResultRow(e.partialResult(), e.record).toString, e) } } } 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 4134da135e3..fd3809ccd31 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 @@ -1743,10 +1743,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "additionalErrorMessage" -> additionalErrorMessage)) } - def malformedRecordsDetectedInRecordParsingError(e: BadRecordException): Throwable = { + def malformedRecordsDetectedInRecordParsingError( + badRecord: String, e: BadRecordException): Throwable = { new SparkException( - errorClass
[spark] branch branch-3.4 updated: [SPARK-42313][SQL] Assign name to `_LEGACY_ERROR_TEMP_1152`
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 5e92386ba21 [SPARK-42313][SQL] Assign name to `_LEGACY_ERROR_TEMP_1152` 5e92386ba21 is described below commit 5e92386ba21386611ebc1edc05350e80c0bb6aa7 Author: itholic AuthorDate: Mon Feb 13 16:06:09 2023 +0500 [SPARK-42313][SQL] Assign name to `_LEGACY_ERROR_TEMP_1152` ### What changes were proposed in this pull request? This PR proposes to integrate _LEGACY_ERROR_TEMP_1152, "PATH_ALREADY_EXISTS". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39953 from itholic/LEGACY_1152. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk (cherry picked from commit 39dbcd7edd3edc9ef68c41d8190e2e9e74f4cedd) Signed-off-by: Max Gekk --- R/pkg/tests/fulltests/test_sparkSQL.R | 10 +++ core/src/main/resources/error/error-classes.json | 11 .../spark/sql/errors/QueryCompilationErrors.scala | 2 +- .../spark/sql/sources/HadoopFsRelationTest.scala | 31 +++--- 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index bec184750e9..b0c56f1c15d 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -622,7 +622,7 @@ test_that("read/write json files", { # Test errorifexists expect_error(write.df(df, jsonPath2, "json", mode = "errorifexists"), - "analysis error - Path file:.*already exists") + "Error in save : analysis error - \\[PATH_ALREADY_EXISTS\\].*") # Test write.json jsonPath3 <- tempfile(pattern = "jsonPath3", fileext = ".json") @@ -3990,13 +3990,13 @@ test_that("Call DataFrameWriter.save() API in Java without path and check argume paste("Error in save : org.apache.spark.SparkIllegalArgumentException:", "Expected exactly one path to be specified")) expect_error(write.json(df, jsonPath), - "Error in json : analysis error - Path file:.*already exists") + "Error in json : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.text(df, jsonPath), - "Error in text : analysis error - Path file:.*already exists") + "Error in text : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.orc(df, jsonPath), - "Error in orc : analysis error - Path file:.*already exists") + "Error in orc : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.parquet(df, jsonPath), - "Error in parquet : analysis error - Path file:.*already exists") + "Error in parquet : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.parquet(df, jsonPath, mode = 123), "mode should be character or omitted.") # Arguments checking in R side. diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 33f31a97acf..86bde20b7fa 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1224,6 +1224,12 @@ ], "sqlState" : "428FT" }, + "PATH_ALREADY_EXISTS" : { +"message" : [ + "Path already exists. Set mode as \"overwrite\" to overwrite the existing path." +], +"sqlState" : "42K04" + }, "PATH_NOT_FOUND" : { "message" : [ "Path does not exist: ." @@ -2765,11 +2771,6 @@ "Fail to resolve data source for the table since the table serde property has the duplicated key with extra options specified for this scan operation. To fix this, you can rollback to the legacy behavior of ignoring the extra options by setting the config to `false`, or address the conflicts of the same config." ] }, - "_LEGACY_ERROR_TEMP_1152" : { -"message" : [ - "Path already exists." -] - }, "_LEGACY_ERROR_TEMP_1153" : { "message" : [ "Cannot use for partition column." diff --git a/sql/catalyst/src/main/scala/org/apache/s
[spark] branch master updated: [SPARK-42313][SQL] Assign name to `_LEGACY_ERROR_TEMP_1152`
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 39dbcd7edd3 [SPARK-42313][SQL] Assign name to `_LEGACY_ERROR_TEMP_1152` 39dbcd7edd3 is described below commit 39dbcd7edd3edc9ef68c41d8190e2e9e74f4cedd Author: itholic AuthorDate: Mon Feb 13 16:06:09 2023 +0500 [SPARK-42313][SQL] Assign name to `_LEGACY_ERROR_TEMP_1152` ### What changes were proposed in this pull request? This PR proposes to integrate _LEGACY_ERROR_TEMP_1152, "PATH_ALREADY_EXISTS". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39953 from itholic/LEGACY_1152. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk --- R/pkg/tests/fulltests/test_sparkSQL.R | 10 +++ core/src/main/resources/error/error-classes.json | 11 .../spark/sql/errors/QueryCompilationErrors.scala | 2 +- .../spark/sql/sources/HadoopFsRelationTest.scala | 31 +++--- 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index bec184750e9..b0c56f1c15d 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -622,7 +622,7 @@ test_that("read/write json files", { # Test errorifexists expect_error(write.df(df, jsonPath2, "json", mode = "errorifexists"), - "analysis error - Path file:.*already exists") + "Error in save : analysis error - \\[PATH_ALREADY_EXISTS\\].*") # Test write.json jsonPath3 <- tempfile(pattern = "jsonPath3", fileext = ".json") @@ -3990,13 +3990,13 @@ test_that("Call DataFrameWriter.save() API in Java without path and check argume paste("Error in save : org.apache.spark.SparkIllegalArgumentException:", "Expected exactly one path to be specified")) expect_error(write.json(df, jsonPath), - "Error in json : analysis error - Path file:.*already exists") + "Error in json : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.text(df, jsonPath), - "Error in text : analysis error - Path file:.*already exists") + "Error in text : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.orc(df, jsonPath), - "Error in orc : analysis error - Path file:.*already exists") + "Error in orc : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.parquet(df, jsonPath), - "Error in parquet : analysis error - Path file:.*already exists") + "Error in parquet : analysis error - \\[PATH_ALREADY_EXISTS\\].*") expect_error(write.parquet(df, jsonPath, mode = 123), "mode should be character or omitted.") # Arguments checking in R side. diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index f7e4086263d..e96383399d2 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1224,6 +1224,12 @@ ], "sqlState" : "428FT" }, + "PATH_ALREADY_EXISTS" : { +"message" : [ + "Path already exists. Set mode as \"overwrite\" to overwrite the existing path." +], +"sqlState" : "42K04" + }, "PATH_NOT_FOUND" : { "message" : [ "Path does not exist: ." @@ -2765,11 +2771,6 @@ "Fail to resolve data source for the table since the table serde property has the duplicated key with extra options specified for this scan operation. To fix this, you can rollback to the legacy behavior of ignoring the extra options by setting the config to `false`, or address the conflicts of the same config." ] }, - "_LEGACY_ERROR_TEMP_1152" : { -"message" : [ - "Path already exists." -] - }, "_LEGACY_ERROR_TEMP_1153" : { "message" : [ "Cannot use for partition column." 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/Qu
[spark] branch branch-3.4 updated: [SPARK-42312][SQL] Assign name to _LEGACY_ERROR_TEMP_0042
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 2729e12d3ed [SPARK-42312][SQL] Assign name to _LEGACY_ERROR_TEMP_0042 2729e12d3ed is described below commit 2729e12d3ed0e0513da7fcb2a4c6bdc78aa955ac Author: itholic AuthorDate: Sun Feb 12 19:13:05 2023 +0500 [SPARK-42312][SQL] Assign name to _LEGACY_ERROR_TEMP_0042 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_0042, "INVALID_SET_SYNTAX". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39951 from itholic/LEGACY_0042. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk (cherry picked from commit 4a27c604eef6f06672a3d2aaa5e40285e15bacab) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 + .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../spark/sql/execution/SparkSqlParserSuite.scala | 28 +++--- 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 3082e482392..33f31a97acf 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1004,6 +1004,12 @@ }, "sqlState" : "42K07" }, + "INVALID_SET_SYNTAX" : { +"message" : [ + "Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use backquotes, e.g., SET `key`=`value`." +], +"sqlState" : "42000" + }, "INVALID_SQL_ARG" : { "message" : [ "The argument of `sql()` is invalid. Consider to replace it by a SQL literal." @@ -2052,11 +2058,6 @@ "Found duplicate clauses: ." ] }, - "_LEGACY_ERROR_TEMP_0042" : { -"message" : [ - "Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use quotes, e.g., SET `key`=`value`." -] - }, "_LEGACY_ERROR_TEMP_0043" : { "message" : [ "Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index accf5363d6c..57868020736 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -478,7 +478,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def unexpectedFormatForSetConfigurationError(ctx: ParserRuleContext): Throwable = { -new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0042", ctx) +new ParseException(errorClass = "INVALID_SET_SYNTAX", ctx) } def invalidPropertyKeyForSetQuotedConfigurationError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 57c991c34d9..d6a3b74ee4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -119,7 +119,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "SET spark.sql.key value" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0042", + errorClass = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql1, @@ -129,7 +129,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "SET spark.sql.key 'value'" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0042", + errorClass = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = s
[spark] branch master updated: [SPARK-42312][SQL] Assign name to _LEGACY_ERROR_TEMP_0042
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 4a27c604eef [SPARK-42312][SQL] Assign name to _LEGACY_ERROR_TEMP_0042 4a27c604eef is described below commit 4a27c604eef6f06672a3d2aaa5e40285e15bacab Author: itholic AuthorDate: Sun Feb 12 19:13:05 2023 +0500 [SPARK-42312][SQL] Assign name to _LEGACY_ERROR_TEMP_0042 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_0042, "INVALID_SET_SYNTAX". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39951 from itholic/LEGACY_0042. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 + .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../spark/sql/execution/SparkSqlParserSuite.scala | 28 +++--- 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 20685622bc5..f7e4086263d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1004,6 +1004,12 @@ }, "sqlState" : "42K07" }, + "INVALID_SET_SYNTAX" : { +"message" : [ + "Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use backquotes, e.g., SET `key`=`value`." +], +"sqlState" : "42000" + }, "INVALID_SQL_ARG" : { "message" : [ "The argument of `sql()` is invalid. Consider to replace it by a SQL literal." @@ -2052,11 +2058,6 @@ "Found duplicate clauses: ." ] }, - "_LEGACY_ERROR_TEMP_0042" : { -"message" : [ - "Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use quotes, e.g., SET `key`=`value`." -] - }, "_LEGACY_ERROR_TEMP_0043" : { "message" : [ "Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index accf5363d6c..57868020736 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -478,7 +478,7 @@ private[sql] object QueryParsingErrors extends QueryErrorsBase { } def unexpectedFormatForSetConfigurationError(ctx: ParserRuleContext): Throwable = { -new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0042", ctx) +new ParseException(errorClass = "INVALID_SET_SYNTAX", ctx) } def invalidPropertyKeyForSetQuotedConfigurationError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 57c991c34d9..d6a3b74ee4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -119,7 +119,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql1 = "SET spark.sql.key value" checkError( exception = parseException(sql1), - errorClass = "_LEGACY_ERROR_TEMP_0042", + errorClass = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql1, @@ -129,7 +129,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql2 = "SET spark.sql.key 'value'" checkError( exception = parseException(sql2), - errorClass = "_LEGACY_ERROR_TEMP_0042", + errorClass = "INVALID_SET_SYNTAX", parameters = Map.empty, context = ExpectedContext( fragment = sql2, @@ -139,7 +139,7 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { val sql3 =
[spark] branch branch-3.4 updated: [SPARK-42318][SPARK-42319][SQL] Assign name to _LEGACY_ERROR_TEMP_(2123|2125)
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 4c4585b29ef [SPARK-42318][SPARK-42319][SQL] Assign name to _LEGACY_ERROR_TEMP_(2123|2125) 4c4585b29ef is described below commit 4c4585b29efe5eb02f3ec40fbcd8bbfa0e3d2b12 Author: itholic AuthorDate: Thu Feb 9 00:17:53 2023 +0500 [SPARK-42318][SPARK-42319][SQL] Assign name to _LEGACY_ERROR_TEMP_(2123|2125) ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2123 and _LEGACY_ERROR_TEMP_2125, "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39891 from itholic/LEGACY_2125. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit b11fba0b8402905c0d682f90939e64882f0fd7f5) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 16 ++-- .../spark/sql/errors/QueryExecutionErrors.scala| 16 +++- .../org/apache/spark/sql/types/StructType.scala| 3 ++- .../org/apache/spark/sql/types/DataTypeSuite.scala | 8 +++- .../apache/spark/sql/types/StructTypeSuite.scala | 22 ++ .../execution/datasources/orc/OrcSourceSuite.scala | 12 6 files changed, 36 insertions(+), 41 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index e95988a260a..65e7fb70bfe 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -76,6 +76,12 @@ "Could not load Protobuf class with name . ." ] }, + "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE" : { +"message" : [ + "Failed to merge incompatible data types and ." +], +"sqlState" : "42825" + }, "CANNOT_MODIFY_CONFIG" : { "message" : [ "Cannot modify the value of the Spark config: .", @@ -4183,21 +4189,11 @@ "Failed parsing : ." ] }, - "_LEGACY_ERROR_TEMP_2123" : { -"message" : [ - "Failed to merge fields '' and ''. " -] - }, "_LEGACY_ERROR_TEMP_2124" : { "message" : [ "Failed to merge decimal types with incompatible scale and ." ] }, - "_LEGACY_ERROR_TEMP_2125" : { -"message" : [ - "Failed to merge incompatible data types and ." -] - }, "_LEGACY_ERROR_TEMP_2126" : { "message" : [ "Unsuccessful attempt to build maps with elements due to exceeding the map size limit ." 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 4abeeef0fc6..4134da135e3 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 @@ -1291,16 +1291,6 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("simpleString" -> StructType.simpleString, "raw" -> raw)) } - def failedMergingFieldsError(leftName: String, rightName: String, e: Throwable): Throwable = { -new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2123", - messageParameters = Map( -"leftName" -> leftName, -"rightName" -> rightName, -"message" -> e.getMessage), - cause = null) - } - def cannotMergeDecimalTypesWithIncompatibleScaleError( leftScale: Int, rightScale: Int): Throwable = { new SparkException( @@ -1313,10 +1303,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def cannotMergeIncompatibleDataTypesError(left: DataType, right: DataType): Throwable = { new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2125", + errorClass = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", messageParameters = Map( -"leftCatalogString" -> left.catalogString, -"rightCatalogString" -> right.catalogString), +"left" -> toSQLType(left), +"right" -> toSQLType(right)), cause = null) } diff --git a/sql/catalyst/src/main
[spark] branch master updated: [SPARK-42318][SPARK-42319][SQL] Assign name to _LEGACY_ERROR_TEMP_(2123|2125)
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 b11fba0b840 [SPARK-42318][SPARK-42319][SQL] Assign name to _LEGACY_ERROR_TEMP_(2123|2125) b11fba0b840 is described below commit b11fba0b8402905c0d682f90939e64882f0fd7f5 Author: itholic AuthorDate: Thu Feb 9 00:17:53 2023 +0500 [SPARK-42318][SPARK-42319][SQL] Assign name to _LEGACY_ERROR_TEMP_(2123|2125) ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2123 and _LEGACY_ERROR_TEMP_2125, "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39891 from itholic/LEGACY_2125. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 16 ++-- .../spark/sql/errors/QueryExecutionErrors.scala| 16 +++- .../org/apache/spark/sql/types/StructType.scala| 3 ++- .../org/apache/spark/sql/types/DataTypeSuite.scala | 8 +++- .../apache/spark/sql/types/StructTypeSuite.scala | 22 ++ .../execution/datasources/orc/OrcSourceSuite.scala | 12 6 files changed, 36 insertions(+), 41 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 1f960e80366..b5f6da1e45d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -76,6 +76,12 @@ "Could not load Protobuf class with name . ." ] }, + "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE" : { +"message" : [ + "Failed to merge incompatible data types and ." +], +"sqlState" : "42825" + }, "CANNOT_MODIFY_CONFIG" : { "message" : [ "Cannot modify the value of the Spark config: .", @@ -4183,21 +4189,11 @@ "Failed parsing : ." ] }, - "_LEGACY_ERROR_TEMP_2123" : { -"message" : [ - "Failed to merge fields '' and ''. " -] - }, "_LEGACY_ERROR_TEMP_2124" : { "message" : [ "Failed to merge decimal types with incompatible scale and ." ] }, - "_LEGACY_ERROR_TEMP_2125" : { -"message" : [ - "Failed to merge incompatible data types and ." -] - }, "_LEGACY_ERROR_TEMP_2126" : { "message" : [ "Unsuccessful attempt to build maps with elements due to exceeding the map size limit ." 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 4abeeef0fc6..4134da135e3 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 @@ -1291,16 +1291,6 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("simpleString" -> StructType.simpleString, "raw" -> raw)) } - def failedMergingFieldsError(leftName: String, rightName: String, e: Throwable): Throwable = { -new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2123", - messageParameters = Map( -"leftName" -> leftName, -"rightName" -> rightName, -"message" -> e.getMessage), - cause = null) - } - def cannotMergeDecimalTypesWithIncompatibleScaleError( leftScale: Int, rightScale: Int): Throwable = { new SparkException( @@ -1313,10 +1303,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def cannotMergeIncompatibleDataTypesError(left: DataType, right: DataType): Throwable = { new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2125", + errorClass = "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE", messageParameters = Map( -"leftCatalogString" -> left.catalogString, -"rightCatalogString" -> right.catalogString), +"left" -> toSQLType(left), +"right" -> toSQLType(right)), cause = null) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/S
[spark] branch branch-3.4 updated: [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127
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 c3584dc9d5d [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127 c3584dc9d5d is described below commit c3584dc9d5de953d0185b2192b34940a1ff4fc64 Author: itholic AuthorDate: Wed Feb 8 22:47:10 2023 +0500 [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2127, "DUPLICATED_MAP_KEY". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39890 from itholic/LEGACY_2127. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk (cherry picked from commit 4b50a46f1c6ba4ffe2c42f70a512879c28d11dcf) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 --- .../spark/sql/errors/QueryExecutionErrors.scala| 5 ++- .../expressions/CollectionExpressionsSuite.scala | 27 .../catalyst/expressions/ComplexTypeSuite.scala| 26 +++ .../expressions/HigherOrderFunctionsSuite.scala| 11 +-- .../catalyst/util/ArrayBasedMapBuilderSuite.scala | 37 +- 6 files changed, 86 insertions(+), 31 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index bed9d0e180c..e95988a260a 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -506,6 +506,12 @@ ], "sqlState" : "22012" }, + "DUPLICATED_MAP_KEY" : { +"message" : [ + "Duplicate map key was found, please check the input data. If you want to remove the duplicated keys, you can set to \"LAST_WIN\" so that the key inserted at last takes precedence." +], +"sqlState" : "23505" + }, "DUPLICATE_KEY" : { "message" : [ "Found duplicate keys ." @@ -4197,11 +4203,6 @@ "Unsuccessful attempt to build maps with elements due to exceeding the map size limit ." ] }, - "_LEGACY_ERROR_TEMP_2127" : { -"message" : [ - "Duplicate map key was found, please check the input data. If you want to remove the duplicated keys, you can set to so that the key inserted at last takes precedence." -] - }, "_LEGACY_ERROR_TEMP_2128" : { "message" : [ "The key array and value array of MapData must have the same length." 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 129d369085b..4abeeef0fc6 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 @@ -1330,11 +1330,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def duplicateMapKeyFoundError(key: Any): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2127", + errorClass = "DUPLICATED_MAP_KEY", messageParameters = Map( "key" -> key.toString(), -"mapKeyDedupPolicy" -> toSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key), -"lastWin" -> toSQLConf(SQLConf.MapKeyDedupPolicy.LAST_WIN.toString( +"mapKeyDedupPolicy" -> toSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key))) } def mapDataKeyArrayLengthDiffersFromValueArrayLengthError(): SparkRuntimeException = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 9b97430594d..5917d84df1e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -169,8 +169,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper MapType(IntegerType, IntegerType, valueContainsNull = true)) val mNull = Literal.create(null, MapType(Strin
[spark] branch master updated: [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127
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 4b50a46f1c6 [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127 4b50a46f1c6 is described below commit 4b50a46f1c6ba4ffe2c42f70a512879c28d11dcf Author: itholic AuthorDate: Wed Feb 8 22:47:10 2023 +0500 [SPARK-42314][SQL] Assign name to _LEGACY_ERROR_TEMP_2127 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2127, "DUPLICATED_MAP_KEY". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39890 from itholic/LEGACY_2127. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 --- .../spark/sql/errors/QueryExecutionErrors.scala| 5 ++- .../expressions/CollectionExpressionsSuite.scala | 27 .../catalyst/expressions/ComplexTypeSuite.scala| 26 +++ .../expressions/HigherOrderFunctionsSuite.scala| 11 +-- .../catalyst/util/ArrayBasedMapBuilderSuite.scala | 37 +- 6 files changed, 86 insertions(+), 31 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 481f6f84af5..1f960e80366 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -506,6 +506,12 @@ ], "sqlState" : "22012" }, + "DUPLICATED_MAP_KEY" : { +"message" : [ + "Duplicate map key was found, please check the input data. If you want to remove the duplicated keys, you can set to \"LAST_WIN\" so that the key inserted at last takes precedence." +], +"sqlState" : "23505" + }, "DUPLICATE_KEY" : { "message" : [ "Found duplicate keys ." @@ -4197,11 +4203,6 @@ "Unsuccessful attempt to build maps with elements due to exceeding the map size limit ." ] }, - "_LEGACY_ERROR_TEMP_2127" : { -"message" : [ - "Duplicate map key was found, please check the input data. If you want to remove the duplicated keys, you can set to so that the key inserted at last takes precedence." -] - }, "_LEGACY_ERROR_TEMP_2128" : { "message" : [ "The key array and value array of MapData must have the same length." 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 129d369085b..4abeeef0fc6 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 @@ -1330,11 +1330,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def duplicateMapKeyFoundError(key: Any): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2127", + errorClass = "DUPLICATED_MAP_KEY", messageParameters = Map( "key" -> key.toString(), -"mapKeyDedupPolicy" -> toSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key), -"lastWin" -> toSQLConf(SQLConf.MapKeyDedupPolicy.LAST_WIN.toString( +"mapKeyDedupPolicy" -> toSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key))) } def mapDataKeyArrayLengthDiffersFromValueArrayLengthError(): SparkRuntimeException = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 9b97430594d..5917d84df1e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -169,8 +169,13 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper MapType(IntegerType, IntegerType, valueContainsNull = true)) val mNull = Literal.create(null, MapType(StringType, StringType)) -checkExceptionInExpression[RuntimeException]( - MapConc
[spark] branch branch-3.4 updated: [SPARK-42305][SQL] Integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION`
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 a2dbde3f56d [SPARK-42305][SQL] Integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION` a2dbde3f56d is described below commit a2dbde3f56d5add8a4f5a3b49d182356590fddd6 Author: itholic AuthorDate: Wed Feb 8 19:23:39 2023 +0500 [SPARK-42305][SQL] Integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION` ### What changes were proposed in this pull request? This PR proposes to integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION`. **_LEGACY_ERROR_TEMP_1229** ```json "_LEGACY_ERROR_TEMP_1229" : { "message" : [ " can only support precision up to ." ] }, ``` **DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION** ```json "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION" : { "message" : [ "Decimal precision exceeds max precision ." ], "sqlState" : "22003" }, ``` ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39875 from itholic/LEGACY_1229. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit f8e06c1e1bb335180ce01ba5c8a079687ebcecf1) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../spark/sql/catalyst/parser/AstBuilder.scala | 5 ++--- .../spark/sql/errors/QueryCompilationErrors.scala| 8 .../org/apache/spark/sql/types/DecimalType.scala | 6 +++--- .../sql/catalyst/parser/ExpressionParserSuite.scala | 3 ++- .../sql-tests/results/ansi/literals.sql.out | 20 +++- .../resources/sql-tests/results/literals.sql.out | 20 +++- .../sql-tests/results/postgreSQL/numeric.sql.out | 9 + 8 files changed, 34 insertions(+), 42 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7126c0bc047..bed9d0e180c 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3100,11 +3100,6 @@ "Decimal scale () cannot be greater than precision ()." ] }, - "_LEGACY_ERROR_TEMP_1229" : { -"message" : [ - " can only support precision up to ." -] - }, "_LEGACY_ERROR_TEMP_1231" : { "message" : [ " is not a valid partition column in table ." 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 dfc6e21d4a0..aea496b872b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -29,9 +29,8 @@ import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.commons.codec.DecoderException import org.apache.commons.codec.binary.Hex -import org.apache.spark.SparkException +import org.apache.spark.{SparkArithmeticException, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} @@ -2604,7 +2603,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit try { Literal(BigDecimal(raw).underlying()) } catch { - case e: AnalysisException => + case e: SparkArithmeticException => throw new ParseException( errorClass = "_LEGACY_ERROR_TEMP_0061", messageParameters = Map("msg" -> e.getMessage), 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 a08f72e8313..4b0e914f279 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2253,14 +2253,6 @@ private[sql] object QueryCompilationEr
[spark] branch master updated: [SPARK-42305][SQL] Integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION`
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 f8e06c1e1bb [SPARK-42305][SQL] Integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION` f8e06c1e1bb is described below commit f8e06c1e1bb335180ce01ba5c8a079687ebcecf1 Author: itholic AuthorDate: Wed Feb 8 19:23:39 2023 +0500 [SPARK-42305][SQL] Integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION` ### What changes were proposed in this pull request? This PR proposes to integrate `_LEGACY_ERROR_TEMP_1229` into `DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION`. **_LEGACY_ERROR_TEMP_1229** ```json "_LEGACY_ERROR_TEMP_1229" : { "message" : [ " can only support precision up to ." ] }, ``` **DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION** ```json "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION" : { "message" : [ "Decimal precision exceeds max precision ." ], "sqlState" : "22003" }, ``` ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39875 from itholic/LEGACY_1229. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 - .../spark/sql/catalyst/parser/AstBuilder.scala | 5 ++--- .../spark/sql/errors/QueryCompilationErrors.scala| 8 .../org/apache/spark/sql/types/DecimalType.scala | 6 +++--- .../sql/catalyst/parser/ExpressionParserSuite.scala | 3 ++- .../sql-tests/results/ansi/literals.sql.out | 20 +++- .../resources/sql-tests/results/literals.sql.out | 20 +++- .../sql-tests/results/postgreSQL/numeric.sql.out | 9 + 8 files changed, 34 insertions(+), 42 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 3b8914c7f93..481f6f84af5 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3100,11 +3100,6 @@ "Decimal scale () cannot be greater than precision ()." ] }, - "_LEGACY_ERROR_TEMP_1229" : { -"message" : [ - " can only support precision up to ." -] - }, "_LEGACY_ERROR_TEMP_1231" : { "message" : [ " is not a valid partition column in table ." 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 dfc6e21d4a0..aea496b872b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -29,9 +29,8 @@ import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.commons.codec.DecoderException import org.apache.commons.codec.binary.Hex -import org.apache.spark.SparkException +import org.apache.spark.{SparkArithmeticException, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} @@ -2604,7 +2603,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit try { Literal(BigDecimal(raw).underlying()) } catch { - case e: AnalysisException => + case e: SparkArithmeticException => throw new ParseException( errorClass = "_LEGACY_ERROR_TEMP_0061", messageParameters = Map("msg" -> e.getMessage), 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 a08f72e8313..4b0e914f279 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2253,14 +2253,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "precision" -> precision.toString)) } - def decimalOnlySuppo
[spark] branch master updated (04550edd49e -> d4e5df827de)
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 04550edd49e [SPARK-41708][SQL][TEST][FOLLOWUP] Match non-space chars in path string add d4e5df827de [SPARK-42303][SQL] Assign name to _LEGACY_ERROR_TEMP_1326 No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json| 13 +++-- .../apache/spark/sql/errors/QueryCompilationErrors.scala| 4 ++-- .../scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 9 ++--- 3 files changed, 15 insertions(+), 11 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-42303][SQL] Assign name to _LEGACY_ERROR_TEMP_1326
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 a381068e147 [SPARK-42303][SQL] Assign name to _LEGACY_ERROR_TEMP_1326 a381068e147 is described below commit a381068e147c7425ee788a999d94064be9c8b47f Author: itholic AuthorDate: Wed Feb 8 19:05:29 2023 +0500 [SPARK-42303][SQL] Assign name to _LEGACY_ERROR_TEMP_1326 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_1326, "CANNOT_MODIFY_CONFIG". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39873 from itholic/LEGACY_1326. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit d4e5df827de9eab91cfe563a729f2f46c5830351) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json| 13 +++-- .../apache/spark/sql/errors/QueryCompilationErrors.scala| 4 ++-- .../scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 9 ++--- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 522cdd6095a..7126c0bc047 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -76,6 +76,13 @@ "Could not load Protobuf class with name . ." ] }, + "CANNOT_MODIFY_CONFIG" : { +"message" : [ + "Cannot modify the value of the Spark config: .", + "See also '/sql-migration-guide.html#ddl-statements'." +], +"sqlState" : "46110" + }, "CANNOT_PARSE_DECIMAL" : { "message" : [ "Cannot parse decimal." @@ -3510,12 +3517,6 @@ "Cannot modify the value of a static config: ." ] }, - "_LEGACY_ERROR_TEMP_1326" : { -"message" : [ - "Cannot modify the value of a Spark config: .", - "See also '/sql-migration-guide.html#ddl-statements'." -] - }, "_LEGACY_ERROR_TEMP_1327" : { "message" : [ "Command execution is not supported in runner ." 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 9da7c7bccd7..a08f72e8313 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2989,8 +2989,8 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def cannotModifyValueOfSparkConfigError(key: String, docroot: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1326", - messageParameters = Map("key" -> key, "docroot" -> docroot)) + errorClass = "CANNOT_MODIFY_CONFIG", + messageParameters = Map("key" -> toSQLConf(key), "docroot" -> docroot)) } def commandExecutionInRunnerUnsupportedError(runner: String): Throwable = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index b3b2912cd6c..30f4fdfbbcf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -22,6 +22,7 @@ import java.util.TimeZone import org.apache.hadoop.fs.Path import org.apache.logging.log4j.Level +import org.apache.spark.SPARK_DOC_ROOT import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.MIT @@ -204,9 +205,11 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { sql("RESET spark.app.id") assert(spark.conf.get("spark.app.id") === appId, "Should not change spark core ones") // spark core conf w/ entry registered -val e1 = intercept[AnalysisException](sql("RESET spark.executor.cores")) -val str_match = "Cannot modify the value of a Spark config: spark.executor.cores" -assert(e1.getMessage.contains(str_match)) +checkError( + exception = intercept[AnalysisException](sql("RESET spark.executor
[spark] branch branch-3.4 updated: [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092)
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 a05bc838fcb [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092) a05bc838fcb is described below commit a05bc838fcbc1bf918016c854dea9e557bd1b5b6 Author: itholic AuthorDate: Wed Feb 8 10:25:49 2023 +0500 [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092) ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2091 and _LEGACY_ERROR_TEMP_2091, "CANNOT_READ_FOOTER". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39889 from itholic/LEGACY_2092. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 3a9c867a59d60f29438d07bd042f1cd037110790) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 15 +- .../spark/sql/errors/QueryExecutionErrors.scala| 11 ++ .../datasources/parquet/ParquetFileFormat.scala| 2 +- .../execution/datasources/orc/OrcQuerySuite.scala | 12 +++ .../execution/datasources/orc/OrcSourceSuite.scala | 24 ++ .../parquet/ParquetFileFormatSuite.scala | 12 +++ .../spark/sql/hive/orc/OrcFileOperator.scala | 4 ++-- 7 files changed, 42 insertions(+), 38 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8a4afd3133b..522cdd6095a 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -99,6 +99,11 @@ ], "sqlState" : "22007" }, + "CANNOT_READ_FILE_FOOTER" : { +"message" : [ + "Could not read footer for file: ." +] + }, "CANNOT_RECOGNIZE_HIVE_TYPE" : { "message" : [ "Cannot recognize hive type string: , column: ." @@ -4041,16 +4046,6 @@ "The input filter of should be fully convertible." ] }, - "_LEGACY_ERROR_TEMP_2091" : { -"message" : [ - "Could not read footer for file: ." -] - }, - "_LEGACY_ERROR_TEMP_2092" : { -"message" : [ - "Could not read footer for file: ." -] - }, "_LEGACY_ERROR_TEMP_2093" : { "message" : [ "Found duplicate field(s) \"\": in case-insensitive mode." 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 7eca9c3cd10..129d369085b 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 @@ -1065,16 +1065,9 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { cause = null) } - def cannotReadFooterForFileError(file: Path, e: IOException): Throwable = { + def cannotReadFooterForFileError(file: Path, e: Exception): Throwable = { new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2091", - messageParameters = Map("file" -> file.toString()), - cause = e) - } - - def cannotReadFooterForFileError(file: FileStatus, e: RuntimeException): Throwable = { -new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2092", + errorClass = "CANNOT_READ_FILE_FOOTER", messageParameters = Map("file" -> file.toString()), cause = e) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 5789f252c3b..c7abf62c6d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -432,7 +432,7 @@ object ParquetFileFormat extends Logging { logWarning(s"Skipped the footer in the corrupted file: $currentFile", e) None } else { - throw QueryExecutionErrors.cannotReadFooterForFileError(currentFile, e) + throw QueryExecutionErrors.cannotReadFooterForFileError(currentFile.getPath, e) } } }.flatten d
[spark] branch master updated (df52c80cc27 -> 3a9c867a59d)
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 df52c80cc27 [SPARK-42358][CORE] Send ExecutorUpdated with the message argument in Master.removeWorker add 3a9c867a59d [SPARK-42315][SQL] Assign name to _LEGACY_ERROR_TEMP_(2091|2092) No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 15 +- .../spark/sql/errors/QueryExecutionErrors.scala| 11 ++ .../datasources/parquet/ParquetFileFormat.scala| 2 +- .../execution/datasources/orc/OrcQuerySuite.scala | 12 +++ .../execution/datasources/orc/OrcSourceSuite.scala | 24 ++ .../parquet/ParquetFileFormatSuite.scala | 12 +++ .../spark/sql/hive/orc/OrcFileOperator.scala | 4 ++-- 7 files changed, 42 insertions(+), 38 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated (320097a2ed0 -> 05ea27e0b0a)
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 320097a2ed0 [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117 add 05ea27e0b0a [SPARK-42301][SQL] Assign name to _LEGACY_ERROR_TEMP_1129 No new revisions were added by this update. Summary of changes: R/pkg/tests/fulltests/test_sparkSQL.R | 3 +-- core/src/main/resources/error/error-classes.json| 11 ++- .../apache/spark/sql/errors/QueryCompilationErrors.scala| 2 +- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 7 +-- .../spark/sql/execution/datasources/orc/OrcQuerySuite.scala | 11 +++ .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 13 - .../apache/spark/sql/test/DataFrameReaderWriterSuite.scala | 11 +++ .../org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala | 12 +++- 8 files changed, 42 insertions(+), 28 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-42301][SQL] Assign name to _LEGACY_ERROR_TEMP_1129
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 159c7ab0d96 [SPARK-42301][SQL] Assign name to _LEGACY_ERROR_TEMP_1129 159c7ab0d96 is described below commit 159c7ab0d964ec94f8ebffd315b77295478fec29 Author: itholic AuthorDate: Wed Feb 8 06:36:02 2023 +0500 [SPARK-42301][SQL] Assign name to _LEGACY_ERROR_TEMP_1129 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_1129, "UNABLE_TO_INFER_SCHEMA". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39871 from itholic/LEGACY_1129. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 05ea27e0b0ad43103f2bf2b1a9b499211492a0fe) Signed-off-by: Max Gekk --- R/pkg/tests/fulltests/test_sparkSQL.R | 3 +-- core/src/main/resources/error/error-classes.json| 11 ++- .../apache/spark/sql/errors/QueryCompilationErrors.scala| 2 +- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 7 +-- .../spark/sql/execution/datasources/orc/OrcQuerySuite.scala | 11 +++ .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 13 - .../apache/spark/sql/test/DataFrameReaderWriterSuite.scala | 11 +++ .../org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala | 12 +++- 8 files changed, 42 insertions(+), 28 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index e5408840e72..bec184750e9 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -4014,8 +4014,7 @@ test_that("Call DataFrameWriter.load() API in Java without path and check argume # It makes sure that we can omit path argument in read.df API and then it calls # DataFrameWriter.load() without path. expect_error(read.df(source = "json"), - paste("Error in load : analysis error - Unable to infer schema for JSON.", - "It must be specified manually")) + "Error in load : analysis error - \\[UNABLE_TO_INFER_SCHEMA\\].*") expect_error(read.df("arbitrary_path"), "Error in load : analysis error - \\[PATH_NOT_FOUND\\].*") expect_error(read.json("arbitrary_path"), diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ea2f28a7fc4..8a4afd3133b 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1388,6 +1388,12 @@ "Unable to convert SQL type to Protobuf type ." ] }, + "UNABLE_TO_INFER_SCHEMA" : { +"message" : [ + "Unable to infer schema for . It must be specified manually." +], +"sqlState" : "42KD9" + }, "UNBOUND_SQL_PARAMETER" : { "message" : [ "Found the unbound parameter: . Please, fix `args` and provide a mapping of the parameter to a SQL literal." @@ -2620,11 +2626,6 @@ "Failed to resolve the schema for for the partition column: . It must be specified manually." ] }, - "_LEGACY_ERROR_TEMP_1129" : { -"message" : [ - "Unable to infer schema for . It must be specified manually." -] - }, "_LEGACY_ERROR_TEMP_1131" : { "message" : [ "Data source does not support output mode." 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 85444060d3b..9da7c7bccd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1394,7 +1394,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { def dataSchemaNotSpecifiedError(format: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1129", + errorClass = "UNABLE_TO_INFER_SCHEMA", messageParameters = Map("format" -> format)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command
[spark] branch branch-3.4 updated: [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117
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 79f2641558c [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117 79f2641558c is described below commit 79f2641558c84822879346980c66f611e369863d Author: itholic AuthorDate: Wed Feb 8 06:33:19 2023 +0500 [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_1117, "REQUIRES_SINGLE_PART_NAMESPACE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39837 from itholic/LEGACY_1117. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 320097a2ed05099e805bffdf319ab05f81ee0cd5) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 6 +++--- .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 10 ++ 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index efa27e825ea..ea2f28a7fc4 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1262,6 +1262,12 @@ ], "sqlState" : "42614" }, + "REQUIRES_SINGLE_PART_NAMESPACE" : { +"message" : [ + " requires a single-part namespace, but got ." +], +"sqlState" : "42K05" + }, "ROUTINE_ALREADY_EXISTS" : { "message" : [ "Cannot create the function because it already exists.", @@ -2564,11 +2570,6 @@ "Sources support continuous: ." ] }, - "_LEGACY_ERROR_TEMP_1117" : { -"message" : [ - " requires a single-part namespace, but got ." -] - }, "_LEGACY_ERROR_TEMP_1119" : { "message" : [ " is not supported in JDBC catalog." 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 634e4ac094d..85444060d3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1240,12 +1240,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new TableAlreadyExistsException(ident.asMultipartIdentifier) } - def requiresSinglePartNamespaceError(ns: Seq[String]): Throwable = { + def requiresSinglePartNamespaceError(namespace: Seq[String]): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1117", + errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", messageParameters = Map( "sessionCatalog" -> CatalogManager.SESSION_CATALOG_NAME, -"ns" -> ns.mkString("[", ", ", "]"))) +"namespace" -> toSQLId(namespace))) } def namespaceAlreadyExistsError(namespace: Array[String]): Throwable = { 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 58ed4b2a55c..673d8029c24 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 @@ -2070,8 +2070,10 @@ class DataSourceV2SQLSuiteV1Filter // the session catalog, not the `global_temp` v2 catalog. sql(s"CREATE TABLE $globalTempDB.ns1.ns2.tbl (id bigint, data string) USING json") }, - errorClass = "_LEGACY_ERROR_TEMP_1117", - parameters = Map("sessionCatalog" -> "spark_catalog", "ns" -> "[global_temp, ns1, ns2]")) + errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + parameters = Map( +"sessionCatalog" -> "spark_catalog", +"namespace" -> "`global_temp`.`ns1`.`ns2`")) } test("table name same as catalog can be used") { @@ -2104,8 +2106,8 @@ class DataSourceV2SQLSuiteV1Filter
[spark] branch master updated: [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117
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 320097a2ed0 [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117 320097a2ed0 is described below commit 320097a2ed05099e805bffdf319ab05f81ee0cd5 Author: itholic AuthorDate: Wed Feb 8 06:33:19 2023 +0500 [SPARK-42254][SQL] Assign name to _LEGACY_ERROR_TEMP_1117 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_1117, "REQUIRES_SINGLE_PART_NAMESPACE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39837 from itholic/LEGACY_1117. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../org/apache/spark/sql/errors/QueryCompilationErrors.scala | 6 +++--- .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 10 ++ 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 8cc87a68d45..bef185c9616 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -1262,6 +1262,12 @@ ], "sqlState" : "42614" }, + "REQUIRES_SINGLE_PART_NAMESPACE" : { +"message" : [ + " requires a single-part namespace, but got ." +], +"sqlState" : "42K05" + }, "ROUTINE_ALREADY_EXISTS" : { "message" : [ "Cannot create the function because it already exists.", @@ -2564,11 +2570,6 @@ "Sources support continuous: ." ] }, - "_LEGACY_ERROR_TEMP_1117" : { -"message" : [ - " requires a single-part namespace, but got ." -] - }, "_LEGACY_ERROR_TEMP_1119" : { "message" : [ " is not supported in JDBC catalog." 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 634e4ac094d..85444060d3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1240,12 +1240,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { new TableAlreadyExistsException(ident.asMultipartIdentifier) } - def requiresSinglePartNamespaceError(ns: Seq[String]): Throwable = { + def requiresSinglePartNamespaceError(namespace: Seq[String]): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1117", + errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", messageParameters = Map( "sessionCatalog" -> CatalogManager.SESSION_CATALOG_NAME, -"ns" -> ns.mkString("[", ", ", "]"))) +"namespace" -> toSQLId(namespace))) } def namespaceAlreadyExistsError(namespace: Array[String]): Throwable = { 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 58ed4b2a55c..673d8029c24 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 @@ -2070,8 +2070,10 @@ class DataSourceV2SQLSuiteV1Filter // the session catalog, not the `global_temp` v2 catalog. sql(s"CREATE TABLE $globalTempDB.ns1.ns2.tbl (id bigint, data string) USING json") }, - errorClass = "_LEGACY_ERROR_TEMP_1117", - parameters = Map("sessionCatalog" -> "spark_catalog", "ns" -> "[global_temp, ns1, ns2]")) + errorClass = "REQUIRES_SINGLE_PART_NAMESPACE", + parameters = Map( +"sessionCatalog" -> "spark_catalog", +"namespace" -> "`global_temp`.`ns1`.`ns2`")) } test("table name same as catalog can be used") { @@ -2104,8 +2106,8 @@ class DataSourceV2SQLSuiteV1Filter def verify(sql: String): Unit = { checkErr
[spark] branch branch-3.4 updated: [SPARK-42249][SQL] Refining html link for documentation in error messages
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 2103bb1df6b [SPARK-42249][SQL] Refining html link for documentation in error messages 2103bb1df6b is described below commit 2103bb1df6bca3920d50920665c0aee1fdf0996b Author: itholic AuthorDate: Wed Feb 8 06:28:17 2023 +0500 [SPARK-42249][SQL] Refining html link for documentation in error messages ### What changes were proposed in this pull request? This PR proposes to refine html link for documentation in error messages by introducing `SPARK_DOC_ROOT` into `core/src/main/scala/org/apache/spark/package.scala` that contains global directory for documentation root link: `https://spark.apache.org/docs/latest` ### Why are the changes needed? To improve error class readability and make sure using right root document directory across all source codes. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Updated the whole existing tests related to this changes. Basically, run `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39820 from itholic/refine_html. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 5225a3224f8339f13b1d917dc4ba7c63dd713552) Signed-off-by: Max Gekk --- build/spark-build-info | 1 + core/src/main/resources/error/error-classes.json | 10 +++ core/src/main/scala/org/apache/spark/package.scala | 7 +++-- .../spark/sql/catalyst/JavaTypeInference.scala | 3 +- .../spark/sql/catalyst/ScalaReflection.scala | 3 +- .../catalyst/util/DateTimeFormatterHelper.scala| 6 ++-- .../spark/sql/errors/QueryCompilationErrors.scala | 9 +++--- .../spark/sql/errors/QueryExecutionErrors.scala| 19 .../analysis/ExpressionTypeCheckingSuite.scala | 20 - .../encoders/EncoderErrorMessageSuite.scala| 22 ++ .../catalyst/encoders/ExpressionEncoderSuite.scala | 6 ++-- .../expressions/CallMethodViaReflectionSuite.scala | 8 +++-- .../catalyst/expressions/ComplexTypeSuite.scala| 8 +++-- .../expressions/GeneratorExpressionSuite.scala | 8 +++-- .../expressions/StringExpressionsSuite.scala | 17 +++ .../scala/org/apache/spark/sql/RuntimeConfig.scala | 3 +- .../resources/sql-tests/results/ansi/date.sql.out | 4 +++ .../results/ansi/datetime-parsing-invalid.sql.out | 1 + .../results/ansi/string-functions.sql.out | 4 +++ .../sql-tests/results/ansi/timestamp.sql.out | 6 .../results/ansi/try_datetime_functions.sql.out| 1 + .../results/ceil-floor-with-scale-param.sql.out| 2 ++ .../sql-tests/results/csv-functions.sql.out| 1 + .../test/resources/sql-tests/results/date.sql.out | 4 +++ .../results/datetime-formatting-invalid.sql.out| 22 ++ .../sql-tests/results/datetime-legacy.sql.out | 1 + .../results/datetime-parsing-invalid.sql.out | 1 + .../sql-tests/results/json-functions.sql.out | 4 +++ .../results/sql-compatibility-functions.sql.out| 1 + .../sql-tests/results/string-functions.sql.out | 4 +++ .../results/table-valued-functions.sql.out | 2 ++ .../sql-tests/results/timestamp-ntz.sql.out| 1 + .../resources/sql-tests/results/timestamp.sql.out | 6 .../results/timestampNTZ/timestamp-ansi.sql.out| 4 +++ .../results/timestampNTZ/timestamp.sql.out | 4 +++ .../results/try_datetime_functions.sql.out | 1 + .../native/stringCastAndExpressions.sql.out| 3 ++ .../resources/sql-tests/results/udaf/udaf.sql.out | 1 + .../sql-tests/results/udf/udf-udaf.sql.out | 1 + .../apache/spark/sql/DataFrameFunctionsSuite.scala | 35 ++ .../org/apache/spark/sql/DateFunctionsSuite.scala | 5 ++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 ++-- .../apache/spark/sql/StringFunctionsSuite.scala| 5 ++-- .../test/scala/org/apache/spark/sql/UDFSuite.scala | 8 +++-- .../sql/errors/QueryCompilationErrorsSuite.scala | 4 ++- .../spark/sql/hive/execution/HiveUDAFSuite.scala | 4 ++- 46 files changed, 220 insertions(+), 75 deletions(-) diff --git a/build/spark-build-info b/build/spark-build-info index 26157e8cf8c..4a4ff9169b3 100755 --- a/build/spark-build-info +++ b/build/spark-build-info @@ -33,6 +33,7 @@ echo_build_properties() { echo branch=$(git rev-parse --abbrev-ref HEAD) echo date=$(date -u +%Y-%m-%dT%H:%M:%SZ) echo url=$(git config --get remote.origin.url | sed 's|https://\(.*\)@\(.*\)|https://\2|') + echo docroot=https://spark.apache.org/docs/latest } echo_build_properties $2 > "$SPARK_BUILD_INFO"
[spark] branch master updated (53c1c682acd -> 5225a3224f8)
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 53c1c682acd [SPARK-42352][BUILD] Upgrade maven to 3.8.7 add 5225a3224f8 [SPARK-42249][SQL] Refining html link for documentation in error messages No new revisions were added by this update. Summary of changes: build/spark-build-info | 1 + core/src/main/resources/error/error-classes.json | 10 +++ core/src/main/scala/org/apache/spark/package.scala | 7 +++-- .../spark/sql/catalyst/JavaTypeInference.scala | 3 +- .../spark/sql/catalyst/ScalaReflection.scala | 3 +- .../catalyst/util/DateTimeFormatterHelper.scala| 6 ++-- .../spark/sql/errors/QueryCompilationErrors.scala | 9 +++--- .../spark/sql/errors/QueryExecutionErrors.scala| 19 .../analysis/ExpressionTypeCheckingSuite.scala | 20 - .../encoders/EncoderErrorMessageSuite.scala| 22 ++ .../catalyst/encoders/ExpressionEncoderSuite.scala | 6 ++-- .../expressions/CallMethodViaReflectionSuite.scala | 8 +++-- .../catalyst/expressions/ComplexTypeSuite.scala| 8 +++-- .../expressions/GeneratorExpressionSuite.scala | 8 +++-- .../expressions/StringExpressionsSuite.scala | 17 +++ .../scala/org/apache/spark/sql/RuntimeConfig.scala | 3 +- .../resources/sql-tests/results/ansi/date.sql.out | 4 +++ .../results/ansi/datetime-parsing-invalid.sql.out | 1 + .../results/ansi/string-functions.sql.out | 4 +++ .../sql-tests/results/ansi/timestamp.sql.out | 6 .../results/ansi/try_datetime_functions.sql.out| 1 + .../results/ceil-floor-with-scale-param.sql.out| 2 ++ .../sql-tests/results/csv-functions.sql.out| 1 + .../test/resources/sql-tests/results/date.sql.out | 4 +++ .../results/datetime-formatting-invalid.sql.out| 22 ++ .../sql-tests/results/datetime-legacy.sql.out | 1 + .../results/datetime-parsing-invalid.sql.out | 1 + .../sql-tests/results/json-functions.sql.out | 4 +++ .../results/sql-compatibility-functions.sql.out| 1 + .../sql-tests/results/string-functions.sql.out | 4 +++ .../results/table-valued-functions.sql.out | 2 ++ .../sql-tests/results/timestamp-ntz.sql.out| 1 + .../resources/sql-tests/results/timestamp.sql.out | 6 .../results/timestampNTZ/timestamp-ansi.sql.out| 4 +++ .../results/timestampNTZ/timestamp.sql.out | 4 +++ .../results/try_datetime_functions.sql.out | 1 + .../native/stringCastAndExpressions.sql.out| 3 ++ .../resources/sql-tests/results/udaf/udaf.sql.out | 1 + .../sql-tests/results/udf/udf-udaf.sql.out | 1 + .../apache/spark/sql/DataFrameFunctionsSuite.scala | 35 ++ .../org/apache/spark/sql/DateFunctionsSuite.scala | 5 ++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 ++-- .../apache/spark/sql/StringFunctionsSuite.scala| 5 ++-- .../test/scala/org/apache/spark/sql/UDFSuite.scala | 8 +++-- .../sql/errors/QueryCompilationErrorsSuite.scala | 4 ++- .../spark/sql/hive/execution/HiveUDAFSuite.scala | 4 ++- 46 files changed, 220 insertions(+), 75 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-42306][SQL] Integrate `_LEGACY_ERROR_TEMP_1317` into `UNRESOLVED_COLUMN.WITH_SUGGESTION`
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 23ab7c43525 [SPARK-42306][SQL] Integrate `_LEGACY_ERROR_TEMP_1317` into `UNRESOLVED_COLUMN.WITH_SUGGESTION` 23ab7c43525 is described below commit 23ab7c435253bd677e99b3cb8d5b269371ff47c8 Author: itholic AuthorDate: Tue Feb 7 09:45:05 2023 +0300 [SPARK-42306][SQL] Integrate `_LEGACY_ERROR_TEMP_1317` into `UNRESOLVED_COLUMN.WITH_SUGGESTION` ### What changes were proposed in this pull request? This PR proposes to integrate `_LEGACY_ERROR_TEMP_1317` into `UNRESOLVED_COLUMN.WITH_SUGGESTION`. **_LEGACY_ERROR_TEMP_1317** ```json "_LEGACY_ERROR_TEMP_1317" : { "message" : [ "Cannot resolve column name \"\" among ()" ] }, ``` **UNRESOLVED_COLUMN.WITH_SUGGESTION** ```json "UNRESOLVED_COLUMN" : { "message" : [ "A column or function parameter with name cannot be resolved." ], "subClass" : { "WITHOUT_SUGGESTION" : { "message" : [ "" ] }, "WITH_SUGGESTION" : { "message" : [ "Did you mean one of the following? []." ] } }, "sqlState" : "42703" }, ``` ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39877 from itholic/LEGACY_1317. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 6b6bb6fa20f40aeedea2fb87008e9cce76c54e28) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 5 .../spark/sql/errors/QueryCompilationErrors.scala | 13 + .../main/scala/org/apache/spark/sql/Dataset.scala | 8 ++--- .../spark/sql/DataFrameNaFunctionsSuite.scala | 17 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 11 --- .../scala/org/apache/spark/sql/DatasetSuite.scala | 34 ++ 6 files changed, 51 insertions(+), 37 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 0591fe9078d..2b4fc4abe68 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3463,11 +3463,6 @@ "Invalid partition transformation: ." ] }, - "_LEGACY_ERROR_TEMP_1317" : { -"message" : [ - "Cannot resolve column name \"\" among ()" -] - }, "_LEGACY_ERROR_TEMP_1318" : { "message" : [ "Unable to parse ''." 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 de0a0124767..10649e1474a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2915,14 +2915,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { messageParameters = Map("expr" -> expr.sql)) } - def cannotResolveColumnNameAmongFieldsError( - colName: String, fieldsStr: String, extraMsg: String): AnalysisException = { + def unresolvedColumnWithSuggestionError( + objectName: String, suggestion: String): AnalysisException = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1317", + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", messageParameters = Map( -"colName" -> colName, -"fieldsStr" -> fieldsStr, -"extraMsg" -> extraMsg)) +"objectName" -> toSQLId(objectName), +"proposal" -> suggestion + ) +) } def cannotParseIntervalError(delayThreshold: String, e: Throwable): Throwable = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index c8e2a48859d..28177b90c7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -48,6 +48,7 @@ import org.apache
[spark] branch master updated (54b5cf687b8 -> 6b6bb6fa20f)
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 54b5cf687b8 [SPARK-41600][SPARK-41623][SPARK-41612][CONNECT] Implement Catalog.cacheTable, isCached and uncache add 6b6bb6fa20f [SPARK-42306][SQL] Integrate `_LEGACY_ERROR_TEMP_1317` into `UNRESOLVED_COLUMN.WITH_SUGGESTION` No new revisions were added by this update. Summary of changes: core/src/main/resources/error/error-classes.json | 5 .../spark/sql/errors/QueryCompilationErrors.scala | 13 + .../main/scala/org/apache/spark/sql/Dataset.scala | 8 ++--- .../spark/sql/DataFrameNaFunctionsSuite.scala | 17 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 11 --- .../scala/org/apache/spark/sql/DatasetSuite.scala | 34 ++ 6 files changed, 51 insertions(+), 37 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch branch-3.4 updated: [SPARK-42255][SQL] Assign name to _LEGACY_ERROR_TEMP_2430
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 0b15a34d1f5 [SPARK-42255][SQL] Assign name to _LEGACY_ERROR_TEMP_2430 0b15a34d1f5 is described below commit 0b15a34d1f5c66bb7f89248f90c1dc9abc15d1bf Author: itholic AuthorDate: Mon Feb 6 18:44:11 2023 +0300 [SPARK-42255][SQL] Assign name to _LEGACY_ERROR_TEMP_2430 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2430, "INCOMPATIBLE_COLUMN_TYPE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39839 from itholic/SPARK-2430. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk (cherry picked from commit 3b9d1c6f3c5dcff764f6c846bade5048a93ded07) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 10 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 89 ++- .../resources/sql-tests/results/except-all.sql.out | 11 +- .../sql-tests/results/intersect-all.sql.out| 11 +- .../native/widenSetOperationTypes.sql.out | 770 +++-- .../sql-tests/results/udf/udf-except-all.sql.out | 11 +- .../results/udf/udf-intersect-all.sql.out | 11 +- .../spark/sql/DataFrameSetOperationsSuite.scala| 21 +- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +- 10 files changed, 532 insertions(+), 424 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 0245e75d1c4..0591fe9078d 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -605,6 +605,12 @@ ], "sqlState" : "42818" }, + "INCOMPATIBLE_COLUMN_TYPE" : { +"message" : [ + " can only be performed on tables with compatible column types. The column of the table is type which is not compatible with at the same column of the first table.." +], +"sqlState" : "42825" + }, "INCOMPATIBLE_DATASOURCE_REGISTER" : { "message" : [ "Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: " @@ -5240,11 +5246,6 @@ "The sum of the LIMIT clause and the OFFSET clause must not be greater than the maximum 32-bit integer value (2,147,483,647) but found limit = , offset = ." ] }, - "_LEGACY_ERROR_TEMP_2430" : { -"message" : [ - " can only be performed on tables with compatible column types. The column of the table is type which is not compatible with at the same column of the first table.." -] - }, "_LEGACY_ERROR_TEMP_2431" : { "message" : [ "Invalid partitioning: is missing or is in a map or array." 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 cca54a8742d..e95c21ad985 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 @@ -577,13 +577,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // SPARK-18058: we shall not care about the nullability of columns if (!dataTypesAreCompatibleFn(dt1, dt2)) { e.failAnalysis( -errorClass = "_LEGACY_ERROR_TEMP_2430", +errorClass = "INCOMPATIBLE_COLUMN_TYPE", messageParameters = Map( "operator" -> toSQLStmt(operator.nodeName), - "ci" -> ordinalNumber(ci), - "ti" -> ordinalNumber(ti + 1), - "dt1" -> dt1.catalogString, - "dt2" -> dt2.catalogString, + "columnOrdinalNumber" -> ordinalNumber(ci), + "tableOrdinalNumber" -> ordinalNumber(ti + 1), + "dataType1" -> toSQLType(dt1), + &
[spark] branch master updated: [SPARK-42255][SQL] Assign name to _LEGACY_ERROR_TEMP_2430
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 3b9d1c6f3c5 [SPARK-42255][SQL] Assign name to _LEGACY_ERROR_TEMP_2430 3b9d1c6f3c5 is described below commit 3b9d1c6f3c5dcff764f6c846bade5048a93ded07 Author: itholic AuthorDate: Mon Feb 6 18:44:11 2023 +0300 [SPARK-42255][SQL] Assign name to _LEGACY_ERROR_TEMP_2430 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2430, "INCOMPATIBLE_COLUMN_TYPE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39839 from itholic/SPARK-2430. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 10 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 89 ++- .../resources/sql-tests/results/except-all.sql.out | 11 +- .../sql-tests/results/intersect-all.sql.out| 11 +- .../native/widenSetOperationTypes.sql.out | 770 +++-- .../sql-tests/results/udf/udf-except-all.sql.out | 11 +- .../results/udf/udf-intersect-all.sql.out | 11 +- .../spark/sql/DataFrameSetOperationsSuite.scala| 21 +- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +- 10 files changed, 532 insertions(+), 424 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ed9f8a63347..806a1159ac0 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -605,6 +605,12 @@ ], "sqlState" : "42818" }, + "INCOMPATIBLE_COLUMN_TYPE" : { +"message" : [ + " can only be performed on tables with compatible column types. The column of the table is type which is not compatible with at the same column of the first table.." +], +"sqlState" : "42825" + }, "INCOMPATIBLE_DATASOURCE_REGISTER" : { "message" : [ "Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: " @@ -5240,11 +5246,6 @@ "The sum of the LIMIT clause and the OFFSET clause must not be greater than the maximum 32-bit integer value (2,147,483,647) but found limit = , offset = ." ] }, - "_LEGACY_ERROR_TEMP_2430" : { -"message" : [ - " can only be performed on tables with compatible column types. The column of the table is type which is not compatible with at the same column of the first table.." -] - }, "_LEGACY_ERROR_TEMP_2431" : { "message" : [ "Invalid partitioning: is missing or is in a map or array." 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 cca54a8742d..e95c21ad985 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 @@ -577,13 +577,13 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // SPARK-18058: we shall not care about the nullability of columns if (!dataTypesAreCompatibleFn(dt1, dt2)) { e.failAnalysis( -errorClass = "_LEGACY_ERROR_TEMP_2430", +errorClass = "INCOMPATIBLE_COLUMN_TYPE", messageParameters = Map( "operator" -> toSQLStmt(operator.nodeName), - "ci" -> ordinalNumber(ci), - "ti" -> ordinalNumber(ti + 1), - "dt1" -> dt1.catalogString, - "dt2" -> dt2.catalogString, + "columnOrdinalNumber" -> ordinalNumber(ci), + "tableOrdinalNumber" -> ordinalNumber(ti + 1), + "dataType1" -> toSQLType(dt1), + "dataType2" -> toSQLType(dt2), "hint" -> extraHintForAnsiTypeC
[spark] branch branch-3.4 updated: [SPARK-42302][SQL] Assign name to _LEGACY_ERROR_TEMP_2135
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 b5609bd4a6e [SPARK-42302][SQL] Assign name to _LEGACY_ERROR_TEMP_2135 b5609bd4a6e is described below commit b5609bd4a6e3708c151fed205cb5c22fcde5ac6b Author: itholic AuthorDate: Mon Feb 6 13:51:40 2023 +0300 [SPARK-42302][SQL] Assign name to _LEGACY_ERROR_TEMP_2135 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2135, "FAILED_PARSE_EMPTY_STRING". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39872 from itholic/LEGACY_2135. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk (cherry picked from commit b6eadf00fc527df428a451ebb20777de9965b2de) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 4 ++-- .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 7 +++ .../spark/sql/execution/datasources/json/JsonSuite.scala | 12 +++- 4 files changed, 18 insertions(+), 16 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 305480a61c3..0245e75d1c4 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -506,6 +506,12 @@ ], "sqlState" : "22003" }, + "EMPTY_JSON_FIELD_VALUE" : { +"message" : [ + "Failed to parse an empty string for data type ." +], +"sqlState" : "42604" + }, "ENCODER_NOT_FOUND" : { "message" : [ "Not found an encoder of the type to Spark SQL internal representation. Consider to change the input type to one of supported at https://spark.apache.org/docs/latest/sql-ref-datatypes.html.; @@ -4237,11 +4243,6 @@ "Cannot parse field value for pattern as target spark data type []." ] }, - "_LEGACY_ERROR_TEMP_2135" : { -"message" : [ - "Failed to parse an empty string for data type ." -] - }, "_LEGACY_ERROR_TEMP_2138" : { "message" : [ "Cannot have circular references in bean class, but got the circular reference of class ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 3fe26e87499..bf07d65caa0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -420,12 +420,12 @@ class JacksonParser( case VALUE_STRING if parser.getTextLength < 1 && allowEmptyString => dataType match { case FloatType | DoubleType | TimestampType | DateType => - throw QueryExecutionErrors.failToParseEmptyStringForDataTypeError(dataType) + throw QueryExecutionErrors.emptyJsonFieldValueError(dataType) case _ => null } case VALUE_STRING if parser.getTextLength < 1 => - throw QueryExecutionErrors.failToParseEmptyStringForDataTypeError(dataType) + throw QueryExecutionErrors.emptyJsonFieldValueError(dataType) case token => // We cannot parse this token based on the given data type. So, we throw a 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 54ae497a114..c8c0ad67c45 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 @@ -1431,11 +1431,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "dataType" -> dataType.toString())) } - def failToParseEmptyStringForDataTypeError(dataType: DataType): SparkRuntimeException = { + def emptyJsonFieldValueError(dataType: DataType): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2135", - messageParameters = Map( -"dataType" -> dataType.catalogStrin
[spark] branch master updated: [SPARK-42302][SQL] Assign name to _LEGACY_ERROR_TEMP_2135
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 b6eadf00fc5 [SPARK-42302][SQL] Assign name to _LEGACY_ERROR_TEMP_2135 b6eadf00fc5 is described below commit b6eadf00fc527df428a451ebb20777de9965b2de Author: itholic AuthorDate: Mon Feb 6 13:51:40 2023 +0300 [SPARK-42302][SQL] Assign name to _LEGACY_ERROR_TEMP_2135 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2135, "FAILED_PARSE_EMPTY_STRING". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*"` Closes #39872 from itholic/LEGACY_2135. Lead-authored-by: itholic Co-authored-by: Haejoon Lee <44108233+itho...@users.noreply.github.com> Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 4 ++-- .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 7 +++ .../spark/sql/execution/datasources/json/JsonSuite.scala | 12 +++- 4 files changed, 18 insertions(+), 16 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 88565eb9f1a..ed9f8a63347 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -506,6 +506,12 @@ ], "sqlState" : "22003" }, + "EMPTY_JSON_FIELD_VALUE" : { +"message" : [ + "Failed to parse an empty string for data type ." +], +"sqlState" : "42604" + }, "ENCODER_NOT_FOUND" : { "message" : [ "Not found an encoder of the type to Spark SQL internal representation. Consider to change the input type to one of supported at https://spark.apache.org/docs/latest/sql-ref-datatypes.html.; @@ -4237,11 +4243,6 @@ "Cannot parse field value for pattern as target spark data type []." ] }, - "_LEGACY_ERROR_TEMP_2135" : { -"message" : [ - "Failed to parse an empty string for data type ." -] - }, "_LEGACY_ERROR_TEMP_2138" : { "message" : [ "Cannot have circular references in bean class, but got the circular reference of class ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 3fe26e87499..bf07d65caa0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -420,12 +420,12 @@ class JacksonParser( case VALUE_STRING if parser.getTextLength < 1 && allowEmptyString => dataType match { case FloatType | DoubleType | TimestampType | DateType => - throw QueryExecutionErrors.failToParseEmptyStringForDataTypeError(dataType) + throw QueryExecutionErrors.emptyJsonFieldValueError(dataType) case _ => null } case VALUE_STRING if parser.getTextLength < 1 => - throw QueryExecutionErrors.failToParseEmptyStringForDataTypeError(dataType) + throw QueryExecutionErrors.emptyJsonFieldValueError(dataType) case token => // We cannot parse this token based on the given data type. So, we throw a 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 54ae497a114..c8c0ad67c45 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 @@ -1431,11 +1431,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "dataType" -> dataType.toString())) } - def failToParseEmptyStringForDataTypeError(dataType: DataType): SparkRuntimeException = { + def emptyJsonFieldValueError(dataType: DataType): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2135", - messageParameters = Map( -"dataType" -> dataType.catalogString)) + errorClass = "EMPTY_JSON_FIELD_VALUE", + messageParameters = Map("dataType" -
[spark] branch branch-3.4 updated: [SPARK-42320][SQL] Assign name to _LEGACY_ERROR_TEMP_2188
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 2c671e4769d [SPARK-42320][SQL] Assign name to _LEGACY_ERROR_TEMP_2188 2c671e4769d is described below commit 2c671e4769d164bf2cfae8d153b84e60acfc9c67 Author: itholic AuthorDate: Mon Feb 6 13:48:15 2023 +0300 [SPARK-42320][SQL] Assign name to _LEGACY_ERROR_TEMP_2188 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2188, "CANNOT_RECOGNIZE_HIVE_TYPE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39888 from itholic/LEGACY_2188. Authored-by: itholic Signed-off-by: Max Gekk (cherry picked from commit 17e3ee03268f1e1a9f453900065871f4639f0a1d) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../spark/sql/errors/QueryExecutionErrors.scala| 6 +++--- .../spark/sql/hive/execution/HiveDDLSuite.scala| 23 +- 3 files changed, 23 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index ec9d92dd0a3..305480a61c3 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -99,6 +99,12 @@ ], "sqlState" : "22007" }, + "CANNOT_RECOGNIZE_HIVE_TYPE" : { +"message" : [ + "Cannot recognize hive type string: , column: ." +], +"sqlState" : "429BB" + }, "CANNOT_RESTORE_PERMISSIONS_FOR_PATH" : { "message" : [ "Failed to set permissions on created path back to ." @@ -4470,11 +4476,6 @@ ", db: , table: ." ] }, - "_LEGACY_ERROR_TEMP_2188" : { -"message" : [ - "Cannot recognize hive type string: , column: ." -] - }, "_LEGACY_ERROR_TEMP_2189" : { "message" : [ "Hive 2.2 and lower versions don't support getTablesByType. Please use Hive 2.3 or higher version." 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 b3bd7b727bf..54ae497a114 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 @@ -1844,10 +1844,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def cannotRecognizeHiveTypeError( e: ParseException, fieldType: String, fieldName: String): Throwable = { new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2188", + errorClass = "CANNOT_RECOGNIZE_HIVE_TYPE", messageParameters = Map( -"fieldType" -> fieldType, -"fieldName" -> fieldName), +"fieldType" -> toSQLType(fieldType), +"fieldName" -> toSQLId(fieldName)), cause = e) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 445477dc024..e09b923ee51 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -156,16 +156,21 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA } test("SPARK-22431: illegal nested type") { -val queries = Seq( - "CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q", - "CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive") +checkError( + exception = intercept[SparkException] { +spark.sql("CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q") + }, + errorClass = "CANNOT_RECOGNIZE_HIVE_TYPE", + parameters = Map("fieldType" -> "\"STRUCT<$A:STRING,B:INT>\"", "fieldName" -> "`q`") +) -queries.foreach(query => { - val err = intercept[SparkException] { -spark.sql(query) - }.getMessage - assert(err.contains("Cannot recognize hive type string")) -}) +checkError( +
[spark] branch master updated: [SPARK-42320][SQL] Assign name to _LEGACY_ERROR_TEMP_2188
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 17e3ee03268 [SPARK-42320][SQL] Assign name to _LEGACY_ERROR_TEMP_2188 17e3ee03268 is described below commit 17e3ee03268f1e1a9f453900065871f4639f0a1d Author: itholic AuthorDate: Mon Feb 6 13:48:15 2023 +0300 [SPARK-42320][SQL] Assign name to _LEGACY_ERROR_TEMP_2188 ### What changes were proposed in this pull request? This PR proposes to assign name to _LEGACY_ERROR_TEMP_2188, "CANNOT_RECOGNIZE_HIVE_TYPE". ### Why are the changes needed? We should assign proper name to _LEGACY_ERROR_TEMP_* ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes #39888 from itholic/LEGACY_2188. Authored-by: itholic Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 11 ++- .../spark/sql/errors/QueryExecutionErrors.scala| 6 +++--- .../spark/sql/hive/execution/HiveDDLSuite.scala| 23 +- 3 files changed, 23 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index afabc56a431..88565eb9f1a 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -99,6 +99,12 @@ ], "sqlState" : "22007" }, + "CANNOT_RECOGNIZE_HIVE_TYPE" : { +"message" : [ + "Cannot recognize hive type string: , column: ." +], +"sqlState" : "429BB" + }, "CANNOT_RESTORE_PERMISSIONS_FOR_PATH" : { "message" : [ "Failed to set permissions on created path back to ." @@ -4470,11 +4476,6 @@ ", db: , table: ." ] }, - "_LEGACY_ERROR_TEMP_2188" : { -"message" : [ - "Cannot recognize hive type string: , column: ." -] - }, "_LEGACY_ERROR_TEMP_2189" : { "message" : [ "Hive 2.2 and lower versions don't support getTablesByType. Please use Hive 2.3 or higher version." 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 b3bd7b727bf..54ae497a114 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 @@ -1844,10 +1844,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def cannotRecognizeHiveTypeError( e: ParseException, fieldType: String, fieldName: String): Throwable = { new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2188", + errorClass = "CANNOT_RECOGNIZE_HIVE_TYPE", messageParameters = Map( -"fieldType" -> fieldType, -"fieldName" -> fieldName), +"fieldType" -> toSQLType(fieldType), +"fieldName" -> toSQLId(fieldName)), cause = e) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 445477dc024..e09b923ee51 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -156,16 +156,21 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA } test("SPARK-22431: illegal nested type") { -val queries = Seq( - "CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q", - "CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive") +checkError( + exception = intercept[SparkException] { +spark.sql("CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q") + }, + errorClass = "CANNOT_RECOGNIZE_HIVE_TYPE", + parameters = Map("fieldType" -> "\"STRUCT<$A:STRING,B:INT>\"", "fieldName" -> "`q`") +) -queries.foreach(query => { - val err = intercept[SparkException] { -spark.sql(query) - }.getMessage - assert(err.contains("Cannot recognize hive type string")) -}) +checkError( + exception = intercept[SparkException] { +spark.sql("CREATE TABLE t(q STRUCT<`$a`:INT, co
[spark] branch master updated: [SPARK-41295][SPARK-41296][SQL] Rename the error classes
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 6bb68b5f75a [SPARK-41295][SPARK-41296][SQL] Rename the error classes 6bb68b5f75a is described below commit 6bb68b5f75ac883423a68583750258656f381c33 Author: narek_karapetian AuthorDate: Mon Feb 6 00:04:40 2023 +0300 [SPARK-41295][SPARK-41296][SQL] Rename the error classes ### What changes were proposed in this pull request? In the PR, I propose to assign the proper names to the legacy error classes: - `_LEGACY_ERROR_TEMP_1105` -> `INVALID_EXTRACT_FIELD_TYPE` - `_LEGACY_ERROR_TEMP_1106` -> `INVALID_EXTRACT_BASE_FIELD_TYPE` - `_LEGACY_ERROR_TEMP_1209` -> `AMBIGUOUS_REFERENCE_TO_FIELDS` Changed error messages for: - `_LEGACY_ERROR_TEMP_1106` from `Can't extract value from : need struct type but got .` to `"Can't extract a value from . Need a complex type [struct, array, map] but got ."` - `_LEGACY_ERROR_TEMP_1209` from `"Ambiguous reference to fields ."` to `"Ambiguous reference to the field . It appears times in the schema."` - `_LEGACY_ERROR_TEMP_1106` from `"Field name should be String Literal, but it's ."` to `"Field name should be a non-null string literal, but it's ."` and modify test suite to use checkError() which checks the error class name, context and etc. **Also this PR contains an additional change (AMBIGUOUS_REFERENCE_TO_FIELDS) which is not tracked in JIRA.** ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. fix a bug, you can clarify why it is a bug. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? Added additional test cases in `org.apache.spark.sql.errors.QueryCompilationErrorsSuite` Closes #39501 from NarekDW/SPARK-41295. Authored-by: narek_karapetian Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 33 +- .../expressions/complexTypeExtractors.scala| 6 +- .../spark/sql/errors/QueryCompilationErrors.scala | 16 ++--- .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 20 -- .../catalyst/expressions/ComplexTypeSuite.scala| 21 -- .../apache/spark/sql/ColumnExpressionSuite.scala | 76 ++ .../sql/errors/QueryCompilationErrorsSuite.scala | 69 +++- .../sql/hive/execution/HiveResolutionSuite.scala | 12 ++-- 8 files changed, 167 insertions(+), 86 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7ecd924ea8d..afabc56a431 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -17,6 +17,12 @@ ], "sqlState" : "42704" }, + "AMBIGUOUS_REFERENCE_TO_FIELDS" : { +"message" : [ + "Ambiguous reference to the field . It appears times in the schema." +], +"sqlState" : "42000" + }, "ARITHMETIC_OVERFLOW" : { "message" : [ ". If necessary set to \"false\" to bypass this error." @@ -750,12 +756,24 @@ ], "sqlState" : "42K05" }, + "INVALID_EXTRACT_BASE_FIELD_TYPE" : { +"message" : [ + "Can't extract a value from . Need a complex type [STRUCT, ARRAY, MAP] but got ." +], +"sqlState" : "42000" + }, "INVALID_EXTRACT_FIELD" : { "message" : [ "Cannot extract from ." ], "sqlState" : "42601" }, + "INVALID_EXTRACT_FIELD_TYPE" : { +"message" : [ + "Field name should be a non-null string literal, but it's ." +], +"sqlState" : "42000" + }, "INVALID_FIELD_NAME" : { "message" : [ "Field name is invalid: is not a struct." @@ -2491,16 +2509,6 @@ "The second argument should be a double literal." ] }, - "_LEGACY_ERROR_TEMP_1105" : { -"message" : [ - "Field name should be String Literal, but it's ." -] - }, - "_LEGACY_ERROR_TEMP_1106" : { -"message" : [ - "Can't extract value from : need struct type but got ." -] - }, "_LEGACY_ERROR_TEMP_1107" : { "message" : [ "Table declares capability but is not an instance of ." @@ -2972,11 +2980,6 @@ "The duration and time inputs
[spark] branch branch-3.4 updated: [SPARK-41295][SPARK-41296][SQL] Rename the error classes
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 889e67a0abe [SPARK-41295][SPARK-41296][SQL] Rename the error classes 889e67a0abe is described below commit 889e67a0abe9420b371623f7c11451759511763c Author: narek_karapetian AuthorDate: Mon Feb 6 00:04:40 2023 +0300 [SPARK-41295][SPARK-41296][SQL] Rename the error classes ### What changes were proposed in this pull request? In the PR, I propose to assign the proper names to the legacy error classes: - `_LEGACY_ERROR_TEMP_1105` -> `INVALID_EXTRACT_FIELD_TYPE` - `_LEGACY_ERROR_TEMP_1106` -> `INVALID_EXTRACT_BASE_FIELD_TYPE` - `_LEGACY_ERROR_TEMP_1209` -> `AMBIGUOUS_REFERENCE_TO_FIELDS` Changed error messages for: - `_LEGACY_ERROR_TEMP_1106` from `Can't extract value from : need struct type but got .` to `"Can't extract a value from . Need a complex type [struct, array, map] but got ."` - `_LEGACY_ERROR_TEMP_1209` from `"Ambiguous reference to fields ."` to `"Ambiguous reference to the field . It appears times in the schema."` - `_LEGACY_ERROR_TEMP_1106` from `"Field name should be String Literal, but it's ."` to `"Field name should be a non-null string literal, but it's ."` and modify test suite to use checkError() which checks the error class name, context and etc. **Also this PR contains an additional change (AMBIGUOUS_REFERENCE_TO_FIELDS) which is not tracked in JIRA.** ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. fix a bug, you can clarify why it is a bug. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? Added additional test cases in `org.apache.spark.sql.errors.QueryCompilationErrorsSuite` Closes #39501 from NarekDW/SPARK-41295. Authored-by: narek_karapetian Signed-off-by: Max Gekk (cherry picked from commit 6bb68b5f75ac883423a68583750258656f381c33) Signed-off-by: Max Gekk --- core/src/main/resources/error/error-classes.json | 33 +- .../expressions/complexTypeExtractors.scala| 6 +- .../spark/sql/errors/QueryCompilationErrors.scala | 16 ++--- .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 20 -- .../catalyst/expressions/ComplexTypeSuite.scala| 21 -- .../apache/spark/sql/ColumnExpressionSuite.scala | 76 ++ .../sql/errors/QueryCompilationErrorsSuite.scala | 69 +++- .../sql/hive/execution/HiveResolutionSuite.scala | 12 ++-- 8 files changed, 167 insertions(+), 86 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 42de98ccb87..ec9d92dd0a3 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -17,6 +17,12 @@ ], "sqlState" : "42704" }, + "AMBIGUOUS_REFERENCE_TO_FIELDS" : { +"message" : [ + "Ambiguous reference to the field . It appears times in the schema." +], +"sqlState" : "42000" + }, "ARITHMETIC_OVERFLOW" : { "message" : [ ". If necessary set to \"false\" to bypass this error." @@ -750,12 +756,24 @@ ], "sqlState" : "42K05" }, + "INVALID_EXTRACT_BASE_FIELD_TYPE" : { +"message" : [ + "Can't extract a value from . Need a complex type [STRUCT, ARRAY, MAP] but got ." +], +"sqlState" : "42000" + }, "INVALID_EXTRACT_FIELD" : { "message" : [ "Cannot extract from ." ], "sqlState" : "42601" }, + "INVALID_EXTRACT_FIELD_TYPE" : { +"message" : [ + "Field name should be a non-null string literal, but it's ." +], +"sqlState" : "42000" + }, "INVALID_FIELD_NAME" : { "message" : [ "Field name is invalid: is not a struct." @@ -2491,16 +2509,6 @@ "The second argument should be a double literal." ] }, - "_LEGACY_ERROR_TEMP_1105" : { -"message" : [ - "Field name should be String Literal, but it's ." -] - }, - "_LEGACY_ERROR_TEMP_1106" : { -"message" : [ - "Can't extract value from : need struct type but got ." -] - }, "_LEGACY_ERROR_TEMP_1107" : { "message" : [ "Table declares capability
[spark] branch branch-3.4 updated: [SPARK-42345][SQL] Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled
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 529ccc7ce37 [SPARK-42345][SQL] Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled 529ccc7ce37 is described below commit 529ccc7ce37f73c01d6c64b3639323f67e6ca323 Author: Gengliang Wang AuthorDate: Sun Feb 5 10:43:34 2023 +0300 [SPARK-42345][SQL] Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled ### What changes were proposed in this pull request? Rename TimestampNTZ data source inference configuration from `spark.sql.inferTimestampNTZInDataSources.enabled` to `spark.sql.sources.timestampNTZTypeInference.enabled` For more context on this configuration: https://github.com/apache/spark/pull/39777 https://github.com/apache/spark/pull/39812 https://github.com/apache/spark/pull/39868 ### Why are the changes needed? Since the configuration is for data source, we can put it under the prefix `spark.sql.sources`. The new naming is consistent with another configuration `spark.sql.sources.partitionColumnTypeInference.enabled`. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #39885 from gengliangwang/renameConf. Authored-by: Gengliang Wang Signed-off-by: Max Gekk (cherry picked from commit c5c1927d6a137c0e92417a0efad5da62ab253137) Signed-off-by: Max Gekk --- .../org/apache/spark/sql/internal/SQLConf.scala | 20 ++-- .../execution/datasources/PartitioningUtils.scala| 2 +- 2 files changed, 11 insertions(+), 11 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 363e763be4f..2f05c356160 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 @@ -1416,6 +1416,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val INFER_TIMESTAMP_NTZ_IN_DATA_SOURCES = +buildConf("spark.sql.sources.timestampNTZTypeInference.enabled") + .doc("For the schema inference of JSON/CSV/JDBC data sources and partition directories, " + +"this config determines whether to choose the TimestampNTZ type if a column can be " + +"either TimestampNTZ or TimestampLTZ type. If set to true, the inference result of " + +"the column will be TimestampNTZ type. Otherwise, the result will be TimestampLTZ type.") + .version("3.4.0") + .booleanConf + .createWithDefault(false) + val BUCKETING_ENABLED = buildConf("spark.sql.sources.bucketing.enabled") .doc("When false, we will treat bucketed table as normal table") .version("2.0.0") @@ -3518,16 +3528,6 @@ object SQLConf { .checkValues(TimestampTypes.values.map(_.toString)) .createWithDefault(TimestampTypes.TIMESTAMP_LTZ.toString) - val INFER_TIMESTAMP_NTZ_IN_DATA_SOURCES = -buildConf("spark.sql.inferTimestampNTZInDataSources.enabled") - .doc("For the schema inference of JSON/CSV/JDBC data sources and partition directories, " + -"this config determines whether to choose the TimestampNTZ type if a column can be " + -"either TimestampNTZ or TimestampLTZ type. If set to true, the inference result of " + -"the column will be TimestampNTZ type. Otherwise, the result will be TimestampLTZ type.") - .version("3.4.0") - .booleanConf - .createWithDefault(false) - val DATETIME_JAVA8API_ENABLED = buildConf("spark.sql.datetime.java8API.enabled") .doc("If the configuration property is set to true, java.time.Instant and " + "java.time.LocalDate classes of Java 8 API are used as external types for " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 38c3f71ab49..90c45fd11dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -490,7 +490,7 @@ object PartitioningUtils extends SQLConfHelper { val unescapedRaw = unescapePathName(raw) // try and parse the date, if no exception occurs this is a candidate to be resolved as // TimestampType or TimestampNTZType. The inference timestamp ty
[spark] branch master updated: [SPARK-42345][SQL] Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled
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 c5c1927d6a1 [SPARK-42345][SQL] Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled c5c1927d6a1 is described below commit c5c1927d6a137c0e92417a0efad5da62ab253137 Author: Gengliang Wang AuthorDate: Sun Feb 5 10:43:34 2023 +0300 [SPARK-42345][SQL] Rename TimestampNTZ inference conf as spark.sql.sources.timestampNTZTypeInference.enabled ### What changes were proposed in this pull request? Rename TimestampNTZ data source inference configuration from `spark.sql.inferTimestampNTZInDataSources.enabled` to `spark.sql.sources.timestampNTZTypeInference.enabled` For more context on this configuration: https://github.com/apache/spark/pull/39777 https://github.com/apache/spark/pull/39812 https://github.com/apache/spark/pull/39868 ### Why are the changes needed? Since the configuration is for data source, we can put it under the prefix `spark.sql.sources`. The new naming is consistent with another configuration `spark.sql.sources.partitionColumnTypeInference.enabled`. ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Closes #39885 from gengliangwang/renameConf. Authored-by: Gengliang Wang Signed-off-by: Max Gekk --- .../org/apache/spark/sql/internal/SQLConf.scala | 20 ++-- .../execution/datasources/PartitioningUtils.scala| 2 +- 2 files changed, 11 insertions(+), 11 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 363e763be4f..2f05c356160 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 @@ -1416,6 +1416,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val INFER_TIMESTAMP_NTZ_IN_DATA_SOURCES = +buildConf("spark.sql.sources.timestampNTZTypeInference.enabled") + .doc("For the schema inference of JSON/CSV/JDBC data sources and partition directories, " + +"this config determines whether to choose the TimestampNTZ type if a column can be " + +"either TimestampNTZ or TimestampLTZ type. If set to true, the inference result of " + +"the column will be TimestampNTZ type. Otherwise, the result will be TimestampLTZ type.") + .version("3.4.0") + .booleanConf + .createWithDefault(false) + val BUCKETING_ENABLED = buildConf("spark.sql.sources.bucketing.enabled") .doc("When false, we will treat bucketed table as normal table") .version("2.0.0") @@ -3518,16 +3528,6 @@ object SQLConf { .checkValues(TimestampTypes.values.map(_.toString)) .createWithDefault(TimestampTypes.TIMESTAMP_LTZ.toString) - val INFER_TIMESTAMP_NTZ_IN_DATA_SOURCES = -buildConf("spark.sql.inferTimestampNTZInDataSources.enabled") - .doc("For the schema inference of JSON/CSV/JDBC data sources and partition directories, " + -"this config determines whether to choose the TimestampNTZ type if a column can be " + -"either TimestampNTZ or TimestampLTZ type. If set to true, the inference result of " + -"the column will be TimestampNTZ type. Otherwise, the result will be TimestampLTZ type.") - .version("3.4.0") - .booleanConf - .createWithDefault(false) - val DATETIME_JAVA8API_ENABLED = buildConf("spark.sql.datetime.java8API.enabled") .doc("If the configuration property is set to true, java.time.Instant and " + "java.time.LocalDate classes of Java 8 API are used as external types for " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 38c3f71ab49..90c45fd11dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -490,7 +490,7 @@ object PartitioningUtils extends SQLConfHelper { val unescapedRaw = unescapePathName(raw) // try and parse the date, if no exception occurs this is a candidate to be resolved as // TimestampType or TimestampNTZType. The inference timestamp typ is controlled by the conf - // "spark.sql.inferTimestampNTZInDataSources.enabled". + // "spark.sql.sources.t