-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
[SPARK-46532][CONNECT] Pass message parameters in metadata of ErrorInfo
#44468
Conversation
ErrorInfo
@dongjoon-hyun @HyukjinKwon @cloud-fan Could you review this PR, please. I need this changes to migrate the rest Spark exceptions on error classes in the PR: #44464 |
@@ -3452,7 +3452,7 @@ def test_error_stack_trace(self): | |||
self.spark.stop() | |||
spark = ( | |||
PySparkSession.builder.config(conf=self.conf()) | |||
.config("spark.connect.jvmStacktrace.maxSize", 128) | |||
.config("spark.connect.grpc.maxMetadataSize", 128) |
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 had to restrict amount of metadata much more otherwise the test fails w/ a gRPC failure: client cannot receive big message.
errorInfo.putMetadata("errorClass", errorClass) | ||
val messageParameters = JsonMethods.compact( | ||
JsonMethods.render(map2jvalue(e.getMessageParameters.asScala.toMap))) | ||
if (messageParameters.length <= maxMetadataSize) { |
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 restricted the size because some errors includes huge parameters like entire string of nested exceptions (especially in legacy error classes).
It looks reasonable to me, @MaxGekk . |
ex.getErrorClass === | ||
"INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER") | ||
assert( | ||
ex.getMessageParameters.asScala == Map( |
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.
Before the changes, getMessageParameters()
returned empty Map.
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 partially true, getMessageParameters()
only returned empty Map when error enrichment is disabled or doesn't work
Merging to master. Thank you, @dongjoon-hyun for review. |
JsonMethods.render(map2jvalue(e.getMessageParameters.asScala.toMap))) | ||
if (messageParameters.length <= maxMetadataSize) { | ||
errorInfo.putMetadata("errorClass", errorClass) | ||
errorInfo.putMetadata("messageParameters", messageParameters) |
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.
@HyukjinKwon please review this. I'm not sure if this is correct, as I don't see we pass the error context here (e.g. SQL string index).
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.
Without the error context you can restore an error but without message parameter it is impossible, and just passing the error class is useless.
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.
Constructing exception messages based on error class and message parameters only works well for Scala client if I understand correctly? For Python client, it only recognizes a very limited set of error classes. https://github.com/databricks/runtime/blob/master/python/pyspark/errors/error_classes.py#L1114
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.
Another issue for constructing exception messages based on error class and message parameters on the client side is the multi version support because for the same error classes in different versions of clients, constructing error messages may require different message parameters
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.
only works well for Scala client if I understand correctly? For Python client, it only recognizes a very limited set of error classes.
This only means that we should keep the python client in sync. Not more.
constructing error messages may require different message parameters
The issue might be solved by sending old and new parameters together but if you send nothing, you don't have any chance.
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.
In the PR #44464, I require all spark exceptions must have an error class (cannot build from just a text message). Found a few test failures at the client connect while creating a SparkThrowable
in GrpcExceptionConverter
like at:
errorConstructor(params =>
new SparkNumberFormatException(
params.errorClass.orNull,
params.messageParameters,
params.queryContext)),
SparkNumberFormatException
's constructor fails because it cannot fill in message parameters since we don't transfer them (the map is empty).
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.
For example https://github.com/MaxGekk/spark/runs/19914247939, the failed test ClientE2ETestSuite.cause exception - false
:
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: Expected exception org.apache.spark.SparkUpgradeException to be thrown, but org.apache.spark.SparkException was thrown
...
Caused by: sbt.ForkMain$ForkError: org.apache.spark.SparkException: [INTERNAL_ERROR] Undefined error message parameter for error class: 'INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER'. Parameters: Map() SQLSTATE: XX000
at org.apache.spark.SparkException$.internalError(SparkException.scala:92)
at org.apache.spark.SparkException$.internalError(SparkException.scala:96)
How should I create SparkUpgradeException
if its constructor requires an error class and correct message parameters (otherwise it fails with the internal error)?
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.
The issue might be solved by sending old and new parameters together but if you send nothing, you don't have any chance.
BTW, maintaining backward compatibility of a ~7k LOC error-classes.json that is changing constantly can be a challenge...
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.
Let me make a followup to match Python side for now.
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.
…Python Spark Connect client ### What changes were proposed in this pull request? This PR is a followup of #44468 that addresses the additional metadata in Python Spark Connect client. ### Why are the changes needed? For feature parity. ### Does this PR introduce _any_ user-facing change? Yes, when `spark.sql.connect.enrichError.enabled` is disabled, users are sill able to get the message parameters. ### How was this patch tested? Unittest was added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44528 from HyukjinKwon/SPARK-46532-followup. Authored-by: Hyukjin Kwon <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
…Python Spark Connect client ### What changes were proposed in this pull request? This PR is a followup of apache/spark#44468 that addresses the additional metadata in Python Spark Connect client. ### Why are the changes needed? For feature parity. ### Does this PR introduce _any_ user-facing change? Yes, when `spark.sql.connect.enrichError.enabled` is disabled, users are sill able to get the message parameters. ### How was this patch tested? Unittest was added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44528 from HyukjinKwon/SPARK-46532-followup. Authored-by: Hyukjin Kwon <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
…Python Spark Connect client ### What changes were proposed in this pull request? This PR is a followup of apache/spark#44468 that addresses the additional metadata in Python Spark Connect client. ### Why are the changes needed? For feature parity. ### Does this PR introduce _any_ user-facing change? Yes, when `spark.sql.connect.enrichError.enabled` is disabled, users are sill able to get the message parameters. ### How was this patch tested? Unittest was added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44528 from HyukjinKwon/SPARK-46532-followup. Authored-by: Hyukjin Kwon <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
What changes were proposed in this pull request?
In the PR, I propose to put message parameters together with an error class in the
messageParameter
field in metadata ofErrorInfo
.Why are the changes needed?
To be able to create an error from an error class and message parameters. Before the changes, it is not possible to re-construct an error having only an error class.
Does this PR introduce any user-facing change?
No.
How was this patch tested?
By running the modified test:
Was this patch authored or co-authored using generative AI tooling?
No.