-
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
Changes from all commits
07eba7d
81f9713
b904a99
7e6774c
462ad9f
88b9ba2
4dcf54f
5861fcc
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -172,6 +172,7 @@ private[connect] object ErrorUtils extends Logging { | |
"classes", | ||
JsonMethods.compact(JsonMethods.render(allClasses(st.getClass).map(_.getName)))) | ||
|
||
val maxMetadataSize = SparkEnv.get.conf.get(Connect.CONNECT_GRPC_MAX_METADATA_SIZE) | ||
// Add the SQL State and Error Class to the response metadata of the ErrorInfoObject. | ||
st match { | ||
case e: SparkThrowable => | ||
|
@@ -181,7 +182,12 @@ private[connect] object ErrorUtils extends Logging { | |
} | ||
val errorClass = e.getErrorClass | ||
if (errorClass != null && errorClass.nonEmpty) { | ||
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 commentThe 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). |
||
errorInfo.putMetadata("errorClass", errorClass) | ||
errorInfo.putMetadata("messageParameters", messageParameters) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe 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 commentThe 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 commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more.
This only means that we should keep the python client in sync. Not more.
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 commentThe 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 errorConstructor(params =>
new SparkNumberFormatException(
params.errorClass.orNull,
params.messageParameters,
params.queryContext)),
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For example https://github.com/MaxGekk/spark/runs/19914247939, the failed test
How should I create There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
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 commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. |
||
} | ||
} | ||
case _ => | ||
} | ||
|
@@ -200,8 +206,10 @@ private[connect] object ErrorUtils extends Logging { | |
val withStackTrace = | ||
if (sessionHolderOpt.exists( | ||
_.session.conf.get(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED) && stackTrace.nonEmpty)) { | ||
val maxSize = SparkEnv.get.conf.get(Connect.CONNECT_JVM_STACK_TRACE_MAX_SIZE) | ||
errorInfo.putMetadata("stackTrace", StringUtils.abbreviate(stackTrace.get, maxSize)) | ||
val maxSize = Math.min( | ||
SparkEnv.get.conf.get(Connect.CONNECT_JVM_STACK_TRACE_MAX_SIZE), | ||
maxMetadataSize) | ||
errorInfo.putMetadata("stackTrace", StringUtils.abbreviate(stackTrace.get, maxSize.toInt)) | ||
} else { | ||
errorInfo | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 commentThe 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. |
||
.remote("local[4]") | ||
.getOrCreate() | ||
) | ||
|
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