Unhandled exception handler based on local ref counting#14049
Unhandled exception handler based on local ref counting#14049ericl merged 19 commits intoray-project:masterfrom
Conversation
python/ray/_raylet.pyx
Outdated
| metadata = Buffer.make(error.GetMetadata()).to_pybytes() | ||
| results = worker.deserialize_objects( | ||
| [(data, metadata)], [ray.ObjectRef.nil()]) | ||
| print("DESERIALIZATION DONE") |
There was a problem hiding this comment.
@edoakes any idea why deserialization would succeed (deserialize_objects prints "DESER RESULT..." ), but then we never get to this second print?
The output I get is
DESERIALIZATION RESULT [RayTaskError('__main__.f', 'Traceback (most recent call last):\n File "python/ray/_raylet.pyx", line 487, in ray._raylet.execute_task\n File "test.py", line 7, in f\n raise ValueError("hi")\nValueError: hi\n', <class 'ValueError'>)]
terminate called without an active exception
zsh: abort python test.py
There was a problem hiding this comment.
I have no idea why you're not hitting the 2nd print, but the only time I've seen this error message is when a c++ thread goes out of scope without joining it or detaching it. If this is being called on a callback from the core worker there might be something weird happening with the lifetime management in c++
| auto iter = objects_.find(object_id); | ||
| if (iter != objects_.end()) { | ||
| auto obj = iter->second; | ||
| obj->SetAccessed(); |
There was a problem hiding this comment.
Why do we need the accessed flag? I thought it would be enough to just throw the error below, if we Put the value but there's no ref to the object.
There was a problem hiding this comment.
Oh I see, for cases where the ref is still in scope but never used?
There was a problem hiding this comment.
Not sure I understand your comment, this is just recording the access.
stephanie-wang
left a comment
There was a problem hiding this comment.
Can we add a unit test for the memory store? This is long overdue but for now we could just test that the callback gets triggered correctly.
| check_signals_(check_signals), | ||
| unhandled_exception_handler_(unhandled_exception_handler) {} | ||
|
|
||
| void CoreWorkerMemoryStore::GetAsync( |
There was a problem hiding this comment.
Should we also SetAccessed here?
edoakes
left a comment
There was a problem hiding this comment.
Nice, really good change both from UX and design standpoint
| # We need to release the gil since object destruction may call the | ||
| # unhandled exception handler. |
There was a problem hiding this comment.
If this is the only time that the unhandled exception handler is called, should we just hold the gil through the whole call to RemoveLocalReference and not acquire it in the handler?
There was a problem hiding this comment.
Sadly we need to hold it during the call to .native().
| num_exceptions = 0 | ||
|
|
||
| def interceptor(e): | ||
| nonlocal num_exceptions |
There was a problem hiding this comment.
i've legitimately never seen nonlocal before.... interesting
| logger.error("Unhandled error (suppress with " | ||
| "RAY_IGNORE_UNHANDLED_ERRORS=1): {}".format(e)) |
There was a problem hiding this comment.
Instead of unhandled error it might be clearer if we explicitly say something like:
Exception from ObjectRef that was never retrieved with ray.get()
There was a problem hiding this comment.
Yeah it's kind of long though... I don't suppose "ungetted error" is a real word?
| # TODO(ekl) remove task push errors entirely now that we have | ||
| # the separate unhandled exception handler. |
|
Comments addressed. |
…project#14049)" This reverts commit 9dc671a.
…ng (ray-project#14049)" (ray-project#14099)" This reverts commit b45ae76.
…project#14049)" (ray-project#14099) This reverts commit 9dc671a.
ray-project#14113) * Revert "Revert "Unhandled exception handler based on local ref counting (ray-project#14049)" (ray-project#14099)" This reverts commit b45ae76. * reomve test * fix * fix
…ng (ray-project#14049)" (ray-project#14099)" This reverts commit 31231f1.
…project#14049)" This reverts commit 64783d3.
Why are these changes needed?
Instead of suppressing remote task errors based on a leaky heuristic, use ref counting to report only the errors that are truly unhandled.
This also adds an env var RAY_IGNORE_UNHANDLED_ERRORS that can be used to suppress the messages.
Related issue number
Closes #12018
Checks
scripts/format.shto lint the changes in this PR.