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

[Core] Restore worker silently fails and the program is stuck #24248

Open
frank-lsf opened this issue Apr 27, 2022 · 13 comments
Open

[Core] Restore worker silently fails and the program is stuck #24248

frank-lsf opened this issue Apr 27, 2022 · 13 comments
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core P3 Issue moderate in impact or severity
Milestone

Comments

@frank-lsf
Copy link
Contributor

What happened + What you expected to happen

When the restore worker encounters an error (e.g. in #24196, or in general for example the spilled file is removed), it silently fails (only prints an error /tmp/ray/session_latest/logs/io-*.err). It looks like the system will schedule a few retries to restore the object, but eventually give up.

The behavior on the application side is that the program is stuck, and no progress is made, and no error is thrown, nor is there any error message. It would be nice if the error can be popped back to the application. For example, when the spill worker errors due to disk space or other reason, the application receives a Python error. Can we do the same for restore workers?

Versions / Dependencies

master

Reproduction script

You can manually make the restore worker throw in external_storage.py, then test its behavior.

Issue Severity

Low: It annoys or frustrates me.

@frank-lsf frank-lsf added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Apr 27, 2022
@stephanie-wang stephanie-wang added the P2 Important issue, but not time-critical label Apr 27, 2022
@stephanie-wang
Copy link
Contributor

@rkooo567 do you think you could take this one since you worked on the spill error?

@stephanie-wang stephanie-wang removed the triage Needs triage (eg: priority, bug/not-bug, and owning component) label Apr 27, 2022
@rkooo567
Copy link
Contributor

Yeah I'd love to. It seems to be a part of better error message as well (which I have worked a bit and stopped for a while). Is P2 the correct priority?

@rkooo567 rkooo567 self-assigned this May 2, 2022
@stephanie-wang stephanie-wang added P1 Issue that should be fixed within a few weeks and removed P2 Important issue, but not time-critical labels May 12, 2022
@stephanie-wang
Copy link
Contributor

I'm upgrading this to P1 since it is actually pretty bad for debugging if we're silently dropping errors.

@rkooo567 do you think you have bandwidth to take this in the next week or two?

@rkooo567
Copy link
Contributor

Hmm I will be a bit busy until June 21st I think. I can share some thoughts if there's anybody who has bandwidth on Shuffle project's side instead?

@stephanie-wang
Copy link
Contributor

I think we can find someone to work on it. But please share what you know about the issue so far!

@kennethlien
Copy link
Contributor

kennethlien commented May 17, 2022

Yeah I'd love to. It seems to be a part of better error message as well (which I have worked a bit and stopped for a while).

Could you point us to the related PRs / any examples of bubbling up errors from workers? @rkooo567

@rkooo567
Copy link
Contributor

@kennethlien
There are two paths where things can hang forever.

Spilling: needed when ray.put is called
Restoration: needed when ray.get is called

Right now, if spilling is failed, it is retried later when another spilling happens. (Ideally, we'd like to fail ray.put if this happens a lot, but I think this is P2).

For restoration, we have the hanging issue described in this issue.

Basically, this is how the problem happens;

  • When the object is not available from the object store, and we need to restore, pull_manager pulls them. It uses the restore worker to restore the object (Look external_stroage.py to see how the restoration works).
    restore_spilled_object_(object_id,
  • If restoration failed, the restore worker just return ok status. Since this won't actually restore the object, pull manager retries to restore the object (by default, it retries pulling 30 seconds + exponential backoff if the object is not available in the next retry timer).
    except Exception:
  • It means that if it keeps failing to restore objects, it will be retried forever with 30 seconds * exponential backoff delay (which will eventually become like hanging).

Ideally, we'd like to catch failures and raise an exception for users when restoration keeps failing.

what you need to do is;

  1. When the restoration request is failed from restore worker, we'd like to return failed status + exception message to the raylet instead of okay status.
    except Exception:
  2. If the restoration request is failed,
    if (!status.ok()) {
    , instead of doing nothing, we should record # of retries (it could be 0 if we'd like to simplify the implementation). If retries == max retries, we just need to mark the object as failed using MarkObjectsAsFailed from node_manager.cc (you should pass this method as a callback to pull manager). Example:
    MarkObjectsAsFailed(error_type, {ref}, JobID::Nil());
    . This creates a Ray object on the corresponding object id, so that the caller can raise an exception.
    elif error_type == ErrorType.Value("ACTOR_DIED"):
  3. Note that you can provide additional data to the error object (the metadata == error type, and the data can contain actual exception messages so that users can see why things are failing). This is how we create a ray object from the core worker to provide actual data (by creating a ray object with metadata as specified here
    RAY_UNUSED(in_memory_store_->Put(RayObject(error_type, ray_error_info), object_id));
    ). Note that providing data to the error object is not supported by Raylet yet. You can talk to me to figure out how to support it, but it is also okay to not include the data in the short term (I will work on improving this path anyway). If you can provide the data, you can include the exception message here.
  4. Now ray.get will raise an exception using this error object.

@kennethlien
Copy link
Contributor

Working on this at #25000

@kennethlien
Copy link
Contributor

kennethlien commented May 24, 2022

When the restoration request is failed from restore worker, we'd like to return failed status + exception message to the raylet instead of okay status.

Is there a way to do this without changing the return value of

cdef int64_t restore_spilled_objects_handler(

@frank-lsf
Copy link
Contributor Author

@kennethlien I think here you can just throw the exception like we do in the spill_objects_handler function above.

@kennethlien
Copy link
Contributor

@rkooo567 We're not sure how to get restore_spilled_objects_handler to return the error without large changes. Would it be reasonable to mark the object as failed after a certain number of retries?

@stephanie-wang
Copy link
Contributor

stephanie-wang commented May 27, 2022

How about we do both? Log the error on the restore handler, and then on the raylet side we can mark the object as failed with a generic "OBJECT_RESTORATION_FAILED" error. That way we shouldn't have endless restore errors.

@rkooo567
Copy link
Contributor

rkooo567 commented Jun 1, 2022

How about we do both? Log the error on the restore handler, and then on the raylet side we can mark the object as failed with a generic "OBJECT_RESTORATION_FAILED" error. That way we shouldn't have endless restore errors.

I think if we can raise an exception with the exception in the error message, it is not necessary to stream logs? (I am okay with this approach if adding an exception message to the error is really difficult).

@kennethlien how big is the change? I am also down to have 1:1 to discuss more options to proceed this faster.

@richardliaw richardliaw added the core Issues that should be addressed in Ray Core label Oct 7, 2022
@hora-anyscale hora-anyscale added P2 Important issue, but not time-critical and removed P1 Issue that should be fixed within a few weeks labels Oct 13, 2022
@rkooo567 rkooo567 removed their assignment Dec 8, 2022
@jjyao jjyao added the P3 Issue moderate in impact or severity label Oct 30, 2024
@jjyao jjyao removed the P2 Important issue, but not time-critical label Oct 30, 2024
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 core Issues that should be addressed in Ray Core P3 Issue moderate in impact or severity
Projects
None yet
Development

No branches or pull requests

7 participants