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 <haejoon....@databricks.com> 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 \"<colName>\" among (<fieldsStr>)<extraMsg>" ] }, ``` **UNRESOLVED_COLUMN.WITH_SUGGESTION** ```json "UNRESOLVED_COLUMN" : { "message" : [ "A column or function parameter with name <objectName> cannot be resolved." ], "subClass" : { "WITHOUT_SUGGESTION" : { "message" : [ "" ] }, "WITH_SUGGESTION" : { "message" : [ "Did you mean one of the following? [<proposal>]." ] } }, "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 <haejoon....@databricks.com> Signed-off-by: Max Gekk <max.g...@gmail.com> (cherry picked from commit 6b6bb6fa20f40aeedea2fb87008e9cce76c54e28) Signed-off-by: Max Gekk <max.g...@gmail.com> --- 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: <expr>." ] }, - "_LEGACY_ERROR_TEMP_1317" : { - "message" : [ - "Cannot resolve column name \"<colName>\" among (<fieldsStr>)<extraMsg>" - ] - }, "_LEGACY_ERROR_TEMP_1318" : { "message" : [ "Unable to parse '<delayThreshold>'." 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.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.errors.QueryCompilationErrors.toSQLId import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} @@ -250,11 +251,8 @@ class Dataset[T] private[sql]( } private def resolveException(colName: String, fields: Array[String]): AnalysisException = { - val extraMsg = if (fields.exists(sparkSession.sessionState.analyzer.resolver(_, colName))) { - s"; did you mean to quote the `$colName` column?" - } else "" - val fieldsStr = fields.mkString(", ") - QueryCompilationErrors.cannotResolveColumnNameAmongFieldsError(colName, fieldsStr, extraMsg) + QueryCompilationErrors.unresolvedColumnWithSuggestionError( + colName, fields.map(toSQLId).mkString(", ")) } private[sql] def numericColumns: Seq[Expression] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index b83a8850fbe..14baa75d81b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -299,10 +299,13 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { test("drop with col(*)") { val df = createDF() - val exception = intercept[AnalysisException] { - df.na.drop("any", Seq("*")) - } - assert(exception.getMessage.contains("Cannot resolve column name \"*\"")) + checkError( + exception = intercept[AnalysisException] { + df.na.drop("any", Seq("*")) + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`*`", "proposal" -> "`name`, `age`, `height`") + ) } test("fill with nested columns") { @@ -534,7 +537,11 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { val exception = intercept[AnalysisException] { df.na.replace("aa", Map( "n/a" -> "unknown")) } - assert(exception.getMessage.equals("Cannot resolve column name \"aa\" among (Col.1, Col.2)")) + checkError( + exception = exception, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`aa`", "proposal" -> "`Col`.`1`, `Col`.`2`") + ) } test("SPARK-34649: replace value of a nested column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 12eb52bc168..bf8d7816e47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -752,10 +752,13 @@ class DataFrameSuite extends QueryTest val df2 = df1.withMetadata("x", metadata) assert(df2.schema(0).metadata === metadata) - val err = intercept[AnalysisException] { - df1.withMetadata("x1", metadata) - } - assert(err.getMessage.contains("Cannot resolve column name")) + checkError( + exception = intercept[AnalysisException] { + df1.withMetadata("x1", metadata) + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`x1`", "proposal" -> "`x`") + ) } test("replace column using withColumn") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 70db5c1a655..86e640a4fa8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -354,15 +354,21 @@ class DatasetSuite extends QueryTest start = 0, stop = 8)) - var e = intercept[AnalysisException] { - ds.select(ds("`(_1)?+.+`")) - } - assert(e.getMessage.contains("Cannot resolve column name \"`(_1)?+.+`\"")) + checkError( + exception = intercept[AnalysisException] { + ds.select(ds("`(_1)?+.+`")) + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`(_1)?+.+`", "proposal" -> "`_1`, `_2`") + ) - e = intercept[AnalysisException] { - ds.select(ds("`(_1|_2)`")) - } - assert(e.getMessage.contains("Cannot resolve column name \"`(_1|_2)`\"")) + checkError( + exception = intercept[AnalysisException] { + ds.select(ds("`(_1|_2)`")) + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> "`(_1|_2)`", "proposal" -> "`_1`, `_2`") + ) } withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { @@ -2166,10 +2172,14 @@ class DatasetSuite extends QueryTest forAll(dotColumnTestModes) { (caseSensitive, colName) => val ds = Seq(SpecialCharClass("1", "2")).toDS withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { - val errorMsg = intercept[AnalysisException] { - ds(colName) - } - assert(errorMsg.getMessage.contains(s"did you mean to quote the `$colName` column?")) + val colName = if (caseSensitive == "true") "`Field`.`1`" else "`field`.`1`" + checkError( + exception = intercept[AnalysisException] { + ds(colName) + }, + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + parameters = Map("objectName" -> colName, "proposal" -> "`field`.`1`, `field 2`") + ) } } } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org