-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-38929][SQL] Improve error messages for cast failures in ANSI #36241
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
The tests running: https://github.com/anchovYu/spark/actions/runs/2182280611 |
| } catch { | ||
| case e: NumberFormatException => | ||
| throw QueryExecutionErrors.invalidInputSyntaxForNumericError(e, errorContext) | ||
| throw QueryExecutionErrors.invalidInputSyntaxForNumericError(to, s, errorContext) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here it doesn't use the error message in e coming from toLongExact, but uses the error message in error-classes.json for better error message organization and grouping.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But the side effect is, it throws the org.apache.spark.SparkNumberFormatException instead of the java.lang.NumberFormatException. The former is a subclass of the latter, so the code catching the latter will still work with the updated version. Will it be a problem?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is fine
| } | ||
|
|
||
| def invalidInputSyntaxForNumericError( | ||
| to: AbstractDataType, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't know whether there is a better way than this AbstractDataType and corresponding tosimpleString method, considering the DecimalType. Since the cast fails, we can't get a concrete DecimalType class with scale and precision, so I just can't use the DataType here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Problem solved if we use the DecimalType from the caller of fromStringANSI:
https://github.com/apache/spark/pull/36244/files#diff-393a0ad195d66fb094f2dca3669c1317b7f57a6637f6e95241b57bc6550ce49aR819
|
Hi @cloud-fan and @MaxGekk , could you take a look at this one? Thank you! |
| "message" : [ "Input schema %s can only contain StringType as a key type for a MapType." ] | ||
| }, | ||
| "INVALID_LITERAL_FORMAT_FOR_CAST" : { | ||
| "message" : [ "Invalid %s literal: %s. To return NULL instead, use 'try_cast'. If necessary set %s to false to bypass this error.%s" ], |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is cast input always a literal? I think Invalid input value for type %s: %s. To return NULL ... is better
| } catch { | ||
| case _: NumberFormatException => | ||
| throw QueryExecutionErrors.invalidInputSyntaxForNumericError(str, errorContext) | ||
| throw QueryExecutionErrors.invalidInputSyntaxForNumericError(DecimalType, str, errorContext) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Where do we call fromStringANSI? There should be a concrete DecimalType available
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we can get that concrete Decimal type because the cast fails: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala#L629
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I didn't do it initially, because I think fromStringANSI and changePrecision are two separate processes and has its corresponding exceptions. The former has an exception to cast from a string to a BigDecimal, and at this step, the target DecimalType doesn't appear. While the latter has exception to change precision, where target starts to take place.
But maybe even though these are two steps, they are always called together serially. Thus, having the target information in the first stage seems fine. It also saves a lot of trouble in AbstractDataType etc. Will update the PR.
| case StringType => | ||
| val doubleStr = ctx.freshVariable("doubleStr", StringType) | ||
| (c, evPrim, evNull) => | ||
| val dt = ctx.addReferenceObj("doubleType", DoubleType, DoubleType.getClass.getName) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is overkill. Let's simply hardcode the data type name in the method invalidInputSyntaxForNumericError like #36244
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry I forget to remove this line .. the dt is not used.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BTW according to the comment from Maxim below, I believe we still need to accept a DataType or AbstractDataType instead of a pure string as a parameter to invalidInputSyntaxForNumericError, so that we can utilize the toSQLType to format
|
@anchovYu I am sorry that I didn't notice you start this one. I realize it after I created #36244. It would be great if you ping me in ANSI-related PRs, thanks! |
| formatter.parse("x123") | ||
| }.getMessage | ||
| assert(errMsg.contains("Cannot cast x123 to DateType")) | ||
| assert(errMsg.contains("Invalid `date` literal: 'x123'")) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please, use toSQLType to output types, see #36233
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will do, thanks for the source!
|
Can one of the admins verify this patch? |
|
The latest tests running: https://github.com/anchovYu/spark/actions/runs/2186916595 |
| "sqlState" : "42000" | ||
| }, | ||
| "INVALID_FORMAT_FOR_CAST" : { | ||
| "message" : [ "Invalid input value for type %s: %s. To return NULL instead, use 'try_cast'. If necessary set %s to false to bypass this error.%s" ], |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we keep the previous wording
Invalid input syntax for type ...
It is from PostgreSQL
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK
| "message" : [ "Field name %s is invalid: %s is not a struct." ], | ||
| "sqlState" : "42000" | ||
| }, | ||
| "INVALID_FORMAT_FOR_CAST" : { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about INVALID_SYNTAX_FOR_CAST
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala
Outdated
Show resolved
Hide resolved
| new DateTimeException(s"Cannot cast $value to $to. To return NULL instead, use 'try_cast'. " + | ||
| s"If necessary set ${SQLConf.ANSI_ENABLED.key} to false to bypass this error." + errorContext) | ||
| new DateTimeException(s"Invalid input syntax for type ${toSQLType(to)}: " + | ||
| s"${if (value.isInstanceOf[UTF8String]) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: let's save the string value to a val before line 1016
| } | ||
|
|
||
| def fromStringANSI(str: UTF8String, errorContext: String = ""): Decimal = { | ||
| def fromStringANSI(str: UTF8String, to: DecimalType = DecimalType.USER_DEFAULT, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| def fromStringANSI(str: UTF8String, to: DecimalType = DecimalType.USER_DEFAULT, | |
| def fromStringANSI( | |
| str: UTF8String, | |
| to: DecimalType = DecimalType.USER_DEFAULT, |
gengliangwang
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM pending tests
| @@ -1,5 +1,5 @@ | |||
| -- Automatically generated by SQLQueryTestSuite | |||
| -- Number of queries: 142 | |||
| -- Number of queries: 143 | |||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hmm, only increased the number?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is the generated results, and did a check that 143 is the right number
|
+1, LGTM. Merging to master. |
|
@anchovYu Could you backport the changes to branch-3.3, please. |
### What changes were proposed in this pull request? Improve the error messages for cast failures in ANSI. As mentioned in https://issues.apache.org/jira/browse/SPARK-38929, this PR targets two cast-to types: numeric types and date types. * For numeric(`int`, `smallint`, `double`, `float`, `decimal` ..) types, it embeds the cast-to types in the error message. For example, ``` Invalid input value for type INT: '1.0'. To return NULL instead, use 'try_cast'. If necessary set %s to false to bypass this error. ``` It uses the `toSQLType` and `toSQLValue` to wrap the corresponding types and literals. * For date types, it does similarly as above. For example, ``` Invalid input value for type TIMESTAMP: 'a'. To return NULL instead, use 'try_cast'. If necessary set spark.sql.ansi.enabled to false to bypass this error. ``` ### Why are the changes needed? To improve the error message in general. ### Does this PR introduce _any_ user-facing change? It changes the error messages. ### How was this patch tested? The related unit tests are updated. Closes apache#36241 from anchovYu/ansi-error-improve. Authored-by: Xinyi Yu <xinyi.yu@databricks.com> Signed-off-by: Max Gekk <max.gekk@gmail.com> (cherry picked from commit f76b3e7)
|
The cherrypick PR to 3.3: #36275 |
What changes were proposed in this pull request?
Improve the error messages for cast failures in ANSI.
As mentioned in https://issues.apache.org/jira/browse/SPARK-38929, this PR targets two cast-to types: numeric types and date types.
int,smallint,double,float,decimal..) types, it embeds the cast-to types in the error message. For example,toSQLTypeandtoSQLValueto wrap the corresponding types and literals.Why are the changes needed?
To improve the error message in general.
Does this PR introduce any user-facing change?
It changes the error messages.
How was this patch tested?
The related unit tests are updated.