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-41127][CONNECT][PYTHON] Implement DataFrame.CreateGlobalView in Python client #38642
Conversation
|
||
# Test when creating a view which is alreayd exists but | ||
self.assertTrue(self.spark.catalog.tableExists("global_temp.view_1")) | ||
with self.assertRaises(_MultiThreadedRendezvous): |
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 found this is what we have now when there is an exception happened during a RPC call.
I guess we need to improve it.
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.
_MultiThreadedRendezvous
is raised from gRPC channel when gRPC finds response state code is not OK
(thus some error has happened).
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.
Yes, right now, the client does not do any error handling, the exception simply bubbles up.
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.
you can just check for grpc.RpcError
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.
@grundprinzip yes! It is much better to use grpc.RpcError
for readability.
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.
Eventually we should also use error classes in client side (based on the grpc error to map to error classes by a way). Right now we only have this.
Can one of the admins verify this patch? |
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
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
Merged to master. |
…n Python client ### What changes were proposed in this pull request? This PR adds `CreateGlobalTempView` and `CreateOrReplaceGlobalTempView` to Python DataFrame API. Meanwhile, this PR extends `LogicalPlan` to let it have the ability to deal with `Command`. ### Why are the changes needed? Improve API coverage. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? UT Closes apache#38642 from amaliujia/create_temp_view_in_python. Authored-by: Rui Wang <rui.wang@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
What changes were proposed in this pull request?
This PR adds
CreateGlobalTempView
andCreateOrReplaceGlobalTempView
to Python DataFrame API.Meanwhile, this PR extends
LogicalPlan
to let it have the ability to deal withCommand
.Why are the changes needed?
Improve API coverage.
Does this PR introduce any user-facing change?
NO
How was this patch tested?
UT