From b8b90ad58804eca528ed3107284448f7531c2a2b Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 15 Nov 2022 23:38:07 +0300 Subject: [PATCH] [SPARK-40755][SQL] Migrate type check failures of number formatting onto error classes ### What changes were proposed in this pull request? This pr aims to replaces TypeCheckFailure by DataTypeMismatch in type checks in the number formatting or parsing expressions, includes: 1. ToNumber (1): https://github.com/apache/spark/blob/1431975723d8df30a25b2333eddcfd0bb6c57677/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala#L83 2. ToCharacter (1): https://github.com/apache/spark/blob/1431975723d8df30a25b2333eddcfd0bb6c57677/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala#L227 3. ToNumberParser (1): https://github.com/apache/spark/blob/5556cfc59aa97a3ad4ea0baacebe19859ec0bcb7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala#L262 ### Why are the changes needed? Migration onto error classes unifies Spark SQL error messages. ### Does this PR introduce _any_ user-facing change? Yes. The PR changes user-facing error messages. ### How was this patch tested? 1. Add new UT 2. Update existed UT 3. Pass GA Closes #38531 from panbingkun/SPARK-40755. Authored-by: panbingkun Signed-off-by: Max Gekk --- .../main/resources/error/error-classes.json | 40 ++++ .../expressions/numberFormatExpressions.scala | 24 ++- .../sql/catalyst/util/ToNumberParser.scala | 82 ++++--- .../expressions/StringExpressionsSuite.scala | 202 ++++++++++++++---- .../results/postgreSQL/numeric.sql.out | 14 +- 5 files changed, 275 insertions(+), 87 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index dc852519339ee..32083c23df8c2 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -214,6 +214,46 @@ "Input to should all be the same type, but it's ." ] }, + "FORMAT_CONT_THOUSANDS_SEPS" : { + "message" : [ + "Thousands separators (, or G) must have digits in between them in the number format: ." + ] + }, + "FORMAT_CUR_MUST_BEFORE_DEC" : { + "message" : [ + "Currency characters must appear before any decimal point in the number format: ." + ] + }, + "FORMAT_CUR_MUST_BEFORE_DIGIT" : { + "message" : [ + "Currency characters must appear before digits in the number format: ." + ] + }, + "FORMAT_EMPTY" : { + "message" : [ + "The number format string cannot be empty." + ] + }, + "FORMAT_THOUSANDS_SEPS_MUST_BEFORE_DEC" : { + "message" : [ + "Thousands separators (, or G) may not appear after the decimal point in the number format: ." + ] + }, + "FORMAT_UNEXPECTED_TOKEN" : { + "message" : [ + "Unexpected found in the format string ; the structure of the format string must match: [MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]." + ] + }, + "FORMAT_WRONG_NUM_DIGIT" : { + "message" : [ + "The format string requires at least one number digit." + ] + }, + "FORMAT_WRONG_NUM_TOKEN" : { + "message" : [ + "At most one is allowed in the number format: ." + ] + }, "HASH_MAP_TYPE" : { "message" : [ "Input to the function cannot contain elements of the \"MAP\" type. In Spark, same maps may have different hashcode, thus hash expressions are prohibited on \"MAP\" elements. To restore previous behavior set \"spark.sql.legacy.allowHashOnMapType\" to \"true\"." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala index ee8ef4633d99a..df6593d5356a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/numberFormatExpressions.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch +import org.apache.spark.sql.catalyst.expressions.Cast._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper import org.apache.spark.sql.catalyst.util.ToNumberParser @@ -78,9 +80,16 @@ case class ToNumber(left: Expression, right: Expression) val inputTypeCheck = super.checkInputDataTypes() if (inputTypeCheck.isSuccess) { if (right.foldable) { - numberFormatter.check() + numberFormatter.checkInputDataTypes() } else { - TypeCheckResult.TypeCheckFailure(s"Format expression must be foldable, but got $right") + DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId(right.prettyName), + "inputType" -> toSQLType(right.dataType), + "inputExpr" -> toSQLExpr(right) + ) + ) } } else { inputTypeCheck @@ -222,9 +231,16 @@ case class ToCharacter(left: Expression, right: Expression) val inputTypeCheck = super.checkInputDataTypes() if (inputTypeCheck.isSuccess) { if (right.foldable) { - numberFormatter.check() + numberFormatter.checkInputDataTypes() } else { - TypeCheckResult.TypeCheckFailure(s"Format expression must be foldable, but got $right") + DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> toSQLId(right.prettyName), + "inputType" -> toSQLType(right.dataType), + "inputExpr" -> toSQLExpr(right) + ) + ) } } else { inputTypeCheck diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala index c54eeb1bf635f..4bc147f329772 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ToNumberParser.scala @@ -20,8 +20,10 @@ package org.apache.spark.sql.catalyst.util import scala.collection.mutable import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.expressions.Cast._ import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.types.{Decimal, DecimalType} +import org.apache.spark.sql.types.{Decimal, DecimalType, StringType} import org.apache.spark.unsafe.types.UTF8String // This object contains some definitions of characters and tokens for the parser below. @@ -253,22 +255,7 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali */ def parsedDecimalType: DecimalType = DecimalType(precision, scale) - /** - * Consumes the format string to check validity and computes an appropriate Decimal output type. - */ - def check(): TypeCheckResult = { - val validateResult: String = validateFormatString - if (validateResult.nonEmpty) { - TypeCheckResult.TypeCheckFailure(validateResult) - } else { - TypeCheckResult.TypeCheckSuccess - } - } - - /** - * This implementation of the [[check]] method returns any error, or the empty string on success. - */ - private def validateFormatString: String = { + def checkInputDataTypes(): TypeCheckResult = { val firstDollarSignIndex: Int = formatTokens.indexOf(DollarSign()) val firstDigitIndex: Int = formatTokens.indexWhere { case _: DigitGroups => true @@ -294,22 +281,37 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali // Make sure the format string contains at least one token. if (numberFormat.isEmpty) { - return "The format string cannot be empty" + return DataTypeMismatch( + errorSubClass = "FORMAT_EMPTY", + messageParameters = Map.empty + ) } // Make sure the format string contains at least one digit. if (!formatTokens.exists( token => token.isInstanceOf[DigitGroups])) { - return "The format string requires at least one number digit" + return DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_DIGIT", + messageParameters = Map.empty + ) } // Make sure that any dollar sign in the format string occurs before any digits. if (firstDigitIndex < firstDollarSignIndex) { - return s"Currency characters must appear before digits in the number format: '$numberFormat'" + return DataTypeMismatch( + errorSubClass = "FORMAT_CUR_MUST_BEFORE_DIGIT", + messageParameters = Map( + "format" -> toSQLValue(numberFormat, StringType) + ) + ) } // Make sure that any dollar sign in the format string occurs before any decimal point. if (firstDecimalPointIndex != -1 && firstDecimalPointIndex < firstDollarSignIndex) { - return "Currency characters must appear before any decimal point in the " + - s"number format: '$numberFormat'" + return DataTypeMismatch( + errorSubClass = "FORMAT_CUR_MUST_BEFORE_DEC", + messageParameters = Map( + "format" -> toSQLValue(numberFormat, StringType) + ) + ) } // Make sure that any thousands separators in the format string have digits before and after. if (digitGroupsBeforeDecimalPoint.exists { @@ -325,16 +327,24 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali false }) }) { - return "Thousands separators (, or G) must have digits in between them " + - s"in the number format: '$numberFormat'" + return DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map( + "format" -> toSQLValue(numberFormat, StringType) + ) + ) } // Make sure that thousands separators does not appear after the decimal point, if any. if (digitGroupsAfterDecimalPoint.exists { case DigitGroups(tokens, digits) => tokens.length > digits.length }) { - return "Thousands separators (, or G) may not appear after the decimal point " + - s"in the number format: '$numberFormat'" + return DataTypeMismatch( + errorSubClass = "FORMAT_THOUSANDS_SEPS_MUST_BEFORE_DEC", + messageParameters = Map( + "format" -> toSQLValue(numberFormat, StringType) + ) + ) } // Make sure that the format string does not contain any prohibited duplicate tokens. val inputTokenCounts = formatTokens.groupBy(identity).mapValues(_.size) @@ -344,7 +354,13 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali DollarSign(), ClosingAngleBracket()).foreach { token => if (inputTokenCounts.getOrElse(token, 0) > 1) { - return s"At most one ${token.toString} is allowed in the number format: '$numberFormat'" + return DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map( + "token" -> token.toString, + "format" -> toSQLValue(numberFormat, StringType) + ) + ) } } // Enforce the ordering of tokens in the format string according to this specification: @@ -377,12 +393,16 @@ class ToNumberParser(numberFormat: String, errorOnFail: Boolean) extends Seriali } } if (formatTokenIndex < formatTokens.length) { - return s"Unexpected ${formatTokens(formatTokenIndex).toString} found in the format string " + - s"'$numberFormat'; the structure of the format string must match: " + - "[MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]" + return DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> formatTokens(formatTokenIndex).toString, + "format" -> toSQLValue(numberFormat, StringType) + ) + ) } // Validation of the format string finished successfully. - "" + TypeCheckSuccess } /** 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 804d9351c7f9f..1ddaa777af82c 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 @@ -1022,76 +1022,175 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("ToNumber and ToCharacter: negative tests (the format string is invalid)") { - val unexpectedCharacter = "the structure of the format string must match: " + - "[MI|S] [$] [0|9|G|,]* [.|D] [0|9]* [$] [PR|MI|S]" - val thousandsSeparatorDigitsBetween = - "Thousands separators (, or G) must have digits in between them" - val mustBeAtEnd = "must be at the end of the number format" - val atMostOne = "At most one" Seq( // The format string must not be empty. - ("454", "") -> "The format string cannot be empty", + ("454", "") -> DataTypeMismatch( + errorSubClass = "FORMAT_EMPTY", + messageParameters = Map.empty), // Make sure the format string does not contain any unrecognized characters. - ("454", "999@") -> unexpectedCharacter, - ("454", "999M") -> unexpectedCharacter, - ("454", "999P") -> unexpectedCharacter, + ("454", "999@") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "character '@''", + "format" -> toSQLValue("999@", StringType)) + ), + ("454", "999M") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "character 'M''", + "format" -> toSQLValue("999M", StringType) + ) + ), + ("454", "999P") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "character 'P''", + "format" -> toSQLValue("999P", StringType)) + ), // Make sure the format string contains at least one digit. - ("454", "$") -> "The format string requires at least one number digit", + ("454", "$") -> DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_DIGIT", + messageParameters = Map.empty), // Make sure the format string contains at most one decimal point. - ("454", "99.99.99") -> atMostOne, + ("454", "99.99.99") -> + DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map( + "token" -> ". or D", + "format" -> toSQLValue("99.99.99", StringType)) + ), // Make sure the format string contains at most one dollar sign. - ("454", "$$99") -> atMostOne, + ("454", "$$99") -> + DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map( + "token" -> "$", "" + + "format" -> toSQLValue("$$99", StringType)) + ), // Make sure the format string contains at most one minus sign at the beginning or end. - ("$4-4", "$9MI9") -> unexpectedCharacter, - ("--4", "SMI9") -> unexpectedCharacter, - ("--$54", "SS$99") -> atMostOne, - ("-$54", "MI$99MI") -> atMostOne, - ("$4-4", "$9MI9MI") -> atMostOne, + ("$4-4", "$9MI9") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "digit sequence", + "format" -> toSQLValue("$9MI9", StringType)) + ), + ("--4", "SMI9") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "digit sequence", + "format" -> toSQLValue("SMI9", StringType)) + ), + ("--$54", "SS$99") -> + DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map( + "token" -> "S", + "format" -> toSQLValue("SS$99", StringType)) + ), + ("-$54", "MI$99MI") -> + DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map( + "token" -> "MI", + "format" -> toSQLValue("MI$99MI", StringType)) + ), + ("$4-4", "$9MI9MI") -> + DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map("token" -> "MI", + "format" -> toSQLValue("$9MI9MI", StringType)) + ), // Make sure the format string contains at most one closing angle bracket at the end. - ("<$45>", "PR$99") -> unexpectedCharacter, - ("$4<4>", "$9PR9") -> unexpectedCharacter, - ("<<454>>", "999PRPR") -> atMostOne, + ("<$45>", "PR$99") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "$", + "format" -> toSQLValue("PR$99", StringType)) + ), + ("$4<4>", "$9PR9") -> + DataTypeMismatch( + errorSubClass = "FORMAT_UNEXPECTED_TOKEN", + messageParameters = Map( + "token" -> "digit sequence", + "format" -> toSQLValue("$9PR9", StringType)) + ), + ("<<454>>", "999PRPR") -> + DataTypeMismatch( + errorSubClass = "FORMAT_WRONG_NUM_TOKEN", + messageParameters = Map( + "token" -> "PR", + "format" -> toSQLValue("999PRPR", StringType)) + ), // Make sure that any dollar sign in the format string occurs before any digits. - ("4$54", "9$99") -> "Currency characters must appear before digits", + ("4$54", "9$99") -> DataTypeMismatch( + errorSubClass = "FORMAT_CUR_MUST_BEFORE_DIGIT", + messageParameters = Map("format" -> toSQLValue("9$99", StringType))), // Make sure that any dollar sign in the format string occurs before any decimal point. - (".$99", ".$99") -> "Currency characters must appear before any decimal point", + (".$99", ".$99") -> DataTypeMismatch( + errorSubClass = "FORMAT_CUR_MUST_BEFORE_DEC", + messageParameters = Map("format" -> toSQLValue(".$99", StringType))), // Thousands separators must have digits in between them. - (",123", ",099") -> thousandsSeparatorDigitsBetween, - (",123,456", ",999,099") -> thousandsSeparatorDigitsBetween, - (",,345", "9,,09.99") -> thousandsSeparatorDigitsBetween, - (",,345", "9,99,.99") -> thousandsSeparatorDigitsBetween, - (",,345", "9,99,") -> thousandsSeparatorDigitsBetween, - (",,345", ",,999,099.99") -> thousandsSeparatorDigitsBetween, + (",123", ",099") -> DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map("format" -> toSQLValue(",099", StringType))), + (",123,456", ",999,099") -> DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map("format" -> toSQLValue(",999,099", StringType))), + (",,345", "9,,09.99") -> DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map("format" -> toSQLValue("9,,09.99", StringType))), + (",,345", "9,99,.99") -> DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map("format" -> toSQLValue("9,99,.99", StringType))), + (",,345", "9,99,") -> DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map("format" -> toSQLValue("9,99,", StringType))), + (",,345", ",,999,099.99") -> DataTypeMismatch( + errorSubClass = "FORMAT_CONT_THOUSANDS_SEPS", + messageParameters = Map("format" -> toSQLValue(",,999,099.99", StringType))), // Thousands separators must not appear after the decimal point. - ("123.45,6", "099.99,9") -> - "Thousands separators (, or G) may not appear after the decimal point" - ).foreach { case ((str: String, format: String), expectedErrMsg: String) => + ("123.45,6", "099.99,9") -> DataTypeMismatch( + errorSubClass = "FORMAT_THOUSANDS_SEPS_MUST_BEFORE_DEC", + messageParameters = Map("format" -> toSQLValue("099.99,9", StringType))) + ).foreach { case ((str: String, format: String), dataTypeMismatch: DataTypeMismatch) => val toNumberResult = ToNumber(Literal(str), Literal(format)).checkInputDataTypes() assert(toNumberResult != TypeCheckResult.TypeCheckSuccess, s"The format string should have been invalid: $format") - toNumberResult match { - case TypeCheckResult.TypeCheckFailure(message) => - assert(message.contains(expectedErrMsg)) - } + assert(toNumberResult == dataTypeMismatch) val tryToNumberResult = TryToNumber(Literal(str), Literal(format)).checkInputDataTypes() assert(tryToNumberResult != TypeCheckResult.TypeCheckSuccess, s"The format string should have been invalid: $format") - tryToNumberResult match { - case TypeCheckResult.TypeCheckFailure(message) => - assert(message.contains(expectedErrMsg)) - } + assert(tryToNumberResult == dataTypeMismatch) val toCharResult = ToCharacter(Decimal(456), Literal(format)).checkInputDataTypes() assert(toCharResult != TypeCheckResult.TypeCheckSuccess, s"The format string should have been invalid: $format") - toCharResult match { - case TypeCheckResult.TypeCheckFailure(message) => - assert(message.contains(expectedErrMsg)) - } + assert(toCharResult == dataTypeMismatch) } } + test("ToCharacter: fails analysis if numberFormat is not foldable") { + val right = AttributeReference("a", StringType)() + val toCharacterExpr = ToCharacter(Decimal(456), right) + assert(toCharacterExpr.checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> "`attributereference`", + "inputType" -> toSQLType(right.dataType), + "inputExpr" -> toSQLExpr(right) + ) + ) + ) + } + test("ToNumber: negative tests (the input string does not match the format string)") { Seq( // The input contained more thousands separators than the format string. @@ -1134,6 +1233,21 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("ToNumber: fails analysis if numberFormat is not foldable") { + val right = AttributeReference("a", StringType)() + val toNumberExpr = ToNumber(Literal("123456"), right) + assert(toNumberExpr.checkInputDataTypes() == + DataTypeMismatch( + errorSubClass = "NON_FOLDABLE_INPUT", + messageParameters = Map( + "inputName" -> "`attributereference`", + "inputType" -> toSQLType(right.dataType), + "inputExpr" -> toSQLExpr(right) + ) + ) + ) + } + test("ToCharacter: positive tests") { // Test '0' and '9' Seq( 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 4718100ffbc14..608c7ba21b743 100644 --- 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 @@ -4712,11 +4712,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.FORMAT_THOUSANDS_SEPS_MUST_BEFORE_DEC", "messageParameters" : { - "hint" : "", - "msg" : "Thousands separators (, or G) may not appear after the decimal point in the number format: '99G999G999D999G999'", - "sqlExpr" : "to_number('-34,338,492.654,878', '99G999G999D999G999')" + "format" : "'99G999G999D999G999'", + "sqlExpr" : "\"to_number(-34,338,492.654,878, 99G999G999D999G999)\"" }, "queryContext" : [ { "objectType" : "", @@ -4775,11 +4774,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "_LEGACY_ERROR_TEMP_2315", + "errorClass" : "DATATYPE_MISMATCH.FORMAT_CONT_THOUSANDS_SEPS", "messageParameters" : { - "hint" : "", - "msg" : "Thousands separators (, or G) must have digits in between them in the number format: '999G'", - "sqlExpr" : "to_number('123,000', '999G')" + "format" : "'999G'", + "sqlExpr" : "\"to_number(123,000, 999G)\"" }, "queryContext" : [ { "objectType" : "",