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] Allow to specify application-level error to retry for actor task #42492

Merged
merged 4 commits into from Jan 23, 2024

Conversation

c21
Copy link
Contributor

@c21 c21 commented Jan 18, 2024

Why are these changes needed?

User reported issue that they cannot specify application-level exception retry for actor task (retry_exceptions). This is due to our actor pool operator does not allow specify ray remote arguments for actor task. This PR adds a config as DataContext.actor_task_retry_on_errors, so users can control application-level exceptions retry.

Related issue number

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: Cheng Su <scnju13@gmail.com>
ray for each map worker. This applies to Ray tasks and actors.
To request resource requirements for tasks launched by Ray actor,
specify ``ray_actor_task_remote_args={...}`` inside
``ray_remote_args``.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIUC, the ray_remote_args is already used for creating the actors. but it also seems unintuitive to use a nested parameter. putting the parameter in the top tier seems better. WDYT?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes agree with the unintuitive behavior. right now we do not mention actor / task in top-level parameter list for map_batches. Seems not good to add an ray_actor_task_remote_args, that brings actor / task back. Another option is to use a data config to specify what's application level error to retry for actor task. WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if currently the requirement is only for enabling application level error retry, I'd vote for adding a specific configure in DataContext. This is easier for Data users to understand.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated, thanks.

Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: Cheng Su <scnju13@gmail.com>
@c21 c21 changed the title [Data] Allow to specify ray remote args for actor task [Data] Allow to specify application-level error to retry for actor task Jan 22, 2024
self._ray_actor_task_remote_args = {}
actor_task_errors = DataContext.get_current().actor_task_retry_on_errors
if len(actor_task_errors) > 0:
self._ray_actor_task_remote_args["retry_exceptions"] = actor_task_errors
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIRC retry_exceptions can either be True/False or a list of exception types in ray core. and defaults to false.
Maybe let's keep the behavior same in data.
also remember to update the comments in DataContext.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, updated.

Signed-off-by: Cheng Su <scnju13@gmail.com>
@c21 c21 merged commit bddcaad into ray-project:master Jan 23, 2024
9 checks passed
@c21 c21 deleted the actor-task-arg branch January 23, 2024 20:29
khluu pushed a commit to khluu/ray that referenced this pull request Jan 24, 2024
…sk (ray-project#42492)

User reported issue that they cannot specify application-level exception retry for actor task (`retry_exceptions`). This is due to our actor pool operator does not allow specify ray remote arguments for actor task. This PR adds a config as `DataContext.actor_task_retry_on_errors`, so users can control application-level exceptions retry.

Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: khluu <khluu000@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants