Skip to content
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

[Data] map_batches doesn't allow for custom errors to be caught #46962

Closed
Takadimi opened this issue Aug 3, 2024 · 6 comments · Fixed by #47339
Closed

[Data] map_batches doesn't allow for custom errors to be caught #46962

Takadimi opened this issue Aug 3, 2024 · 6 comments · Fixed by #47339
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues good first issue Great starter issue for someone just starting to contribute to Ray P1 Issue that should be fixed within a few weeks

Comments

@Takadimi
Copy link

Takadimi commented Aug 3, 2024

What happened + What you expected to happen

As far as I can tell, when wrapping a call to map_batches in a try/except block any errors thrown inside the batch function are "swallowed" and cannot be accessed through typical means.

The error that gets thrown is a RayTaskError(UserCodeException). The docs suggest that we should be able to extract the original error from a RayTaskError by accessing its cause attribute, but that's set to the UserCodeException error (a Ray library error).

The expected behavior is that we can catch user-defined exceptions when they're thrown from the batch function of a map_batches call.

Versions / Dependencies

Ray v2.34.0 (seen in Ray v2.31.0 as well)
Python 3.12.2

Reproduction script

Script

import ray

class MyError(Exception):
    def __init__(self, foo: str):
        super().__init__(f"foo={foo}")

        self.foo=foo

ray.init()

ds = ray.data.from_items([
    {"food": "spam", "price": 9.34},
    {"food": "ham", "price": 5.37},
    {"food": "eggs", "price": 0.94}
])
# ds.show(limit=1)

@ray.remote
def my_remote():
    def transform_batch(batch):
        raise MyError("My custom error")

    try:
        # NOTE: If we just raise the error directly it gets caught correctly
        # raise MyError("My custom error raised directly")

        transformed_ds = ds.map_batches(transform_batch)
        return transformed_ds.materialize()
    except MyError as e:
        print("WORKING: We're able to catch MyError explicitly")
        raise e
    except Exception as e:
        print("NOT WORKING: MyError was skipped...")
        print("Exception is instance of: ", type(e))
        if isinstance(e, ray.exceptions.RayTaskError):
            print("RayTaskError.__cause__ is instance of: ", type(e.__cause__))
            cause = e.cause
            print("RayTaskError.cause is instance of: ", type(cause))
            if isinstance(cause, ray.exceptions.UserCodeException):
                print("UserCodeException.__cause__ is instance of: ", type(cause.__cause__))
        raise e

ref = my_remote.remote()
result = ray.get(ref)

ray.shutdown()

Current output

Here's the most relevant, abbreviated, output from the script above:

(my_remote pid=59182) NOT WORKING: MyError was skipped...
(my_remote pid=59182) Exception is instance of:  <class 'ray.exceptions.RayTaskError(UserCodeException)'>
(my_remote pid=59182) RayTaskError.__cause__ is instance of:  <class 'NoneType'>
(my_remote pid=59182) RayTaskError.cause is instance of:  <class 'ray.exceptions.UserCodeException'>
(my_remote pid=59182) UserCodeException.__cause__ is instance of:  <class 'NoneType'>

Full logs with exception trace

Issue Severity

High: It blocks me from completing my task.

@Takadimi Takadimi added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Aug 3, 2024
@anyscalesam anyscalesam added the data Ray Data-related issues label Aug 5, 2024
@Bye-legumes
Copy link
Contributor

Bye-legumes commented Aug 19, 2024

This problem can be solve with debug mode in ray data.

if _is_ray_debugger_enabled():

When you set RAY_DEBUG, it will raise udf exception

@Takadimi
Copy link
Author

This problem can be solve with debug mode in ray data.

if _is_ray_debugger_enabled():

When you set RAY_DEBUG, it will raise udf exception

That does seem to work, but is that really the suggested path? Why is debug mode required to be able to surface user defined errors?

Are there any other implications (performance or otherwise) to consider when enabling debug mode?

@scottjlee
Copy link
Contributor

For this, you'll want to set the DataContext.log_internal_stack_trace_to_stdout attribute. There are two ways to do it:

  • Set the env var RAY_DATA_LOG_INTERNAL_STACK_TRACE_TO_STDOUT=1
  • Set DataContext: ray.data.DataContext.get_current().log_internal_stack_trace_to_stdout = True

In later versions of Ray (2.33+), we changed this behavior so that we always show the full internal stack trace: #46647

Feel free to followup with any other questions.

@scottjlee scottjlee removed the triage Needs triage (eg: priority, bug/not-bug, and owning component) label Aug 21, 2024
@Takadimi
Copy link
Author

@scottjlee Thanks for the follow up, but I'm not sure I understand. The example reproduction script is from running Ray 2.34.0. The PR you linked seems to be about seeing the full traceback in logs, but I'm talking about being able to catch the custom error (MyError from the example).

The current behavior is that I can only catch a RayTaskError and get its cause which is a UserCodeException and not MyError. @Bye-legumes suggestion of setting RAY_DEBUG=1 does fix the issue, but I'm hesitant to add that to my production system in case there are other side-effects (e.g., performance degradations in debug mode) now or in the future.

@scottjlee
Copy link
Contributor

scottjlee commented Aug 22, 2024

Ah i see, I apologize I misunderstood upon initial read. I will mark this as a bug, as we should add another way to expose the full exception. Thanks for reporting

I would say enabling the Ray debugger has some negative performance impact (although I am not exactly sure what the impact would be if it is enabled, and no debugger breakpoints are added). Probably not a good idea to enable in production, as you said.

@scottjlee scottjlee added good first issue Great starter issue for someone just starting to contribute to Ray P1 Issue that should be fixed within a few weeks labels Aug 22, 2024
@Takadimi
Copy link
Author

Ah i see, I apologize I misunderstood upon initial read. I will mark this as a bug, as we should add another way to expose the full exception. Thanks for reporting

I would say enabling the Ray debugger has some negative performance impact (although I am not exactly sure what the impact would be if it is enabled, and no debugger breakpoints are added). Probably not a good idea to enable in production, as you said.

Sounds good. And thanks for confirming turning on RAY_DEBUG would be likely a bad call in production!

scottjlee pushed a commit that referenced this issue Aug 30, 2024
## Why are these changes needed?

close #46962
## Related issue number

#46962
## Checks

- [√] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [√] I've run `scripts/format.sh` to lint the changes in this PR.
- [√] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [ ] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [√] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
ujjawal-khare pushed a commit to ujjawal-khare-27/ray that referenced this issue Oct 12, 2024
## Why are these changes needed?

close ray-project#46962
## Related issue number

ray-project#46962
## Checks

- [√] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [√] I've run `scripts/format.sh` to lint the changes in this PR.
- [√] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [ ] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [√] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: ujjawal-khare <ujjawal.khare@dream11.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues good first issue Great starter issue for someone just starting to contribute to Ray P1 Issue that should be fixed within a few weeks
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants