[core] Warning when creating actor with restarts and arguments in plasma#53713
[core] Warning when creating actor with restarts and arguments in plasma#53713edoakes merged 4 commits intoray-project:masterfrom
Conversation
…asma Signed-off-by: dayshah <dhyey2019@gmail.com>
There was a problem hiding this comment.
Pull Request Overview
This PR adds a warning when creating actors that have restarts and include arguments stored in plasma, indicating that if those arguments go out of scope the actor restart may fail. Key changes include:
- Returning early in local mode by replacing ExecuteTaskLocalMode with a return.
- Adding a loop to inspect actor creation arguments and logging a warning if any argument is stored in the object store.
- Minor formatting adjustments in the failure handling for actor creation.
Comments suppressed due to low confidence (1)
src/ray/core_worker/core_worker.cc:2718
- [nitpick] Consider renaming 'log_message' to something more descriptive (e.g., 'hasObjectStoreArgs') to clarify its purpose.
bool log_message = false;
| auto status = actor_creator_->RegisterActor(task_spec); | ||
| if (!status.ok()) { | ||
| return status; | ||
| return Status::OK(); |
There was a problem hiding this comment.
returning out here so the rest of the code is not nested. All changes are only in the if (task_spec.MaxActorRestarts() != 0) block
src/ray/core_worker/core_worker.cc
Outdated
| } | ||
| if (log_message) { | ||
| RAY_LOG(ERROR) | ||
| << "Actor " << task_spec.ActorCreationId() |
There was a problem hiding this comment.
what does the creation ID look like? Is it possible to use the class name for a better user-facing error?
There was a problem hiding this comment.
yup, added class name.
actor creation id is the actor id full message in the other comment
src/ray/core_worker/core_worker.cc
Outdated
| << "Actor " << task_spec.ActorCreationId() | ||
| << " has arguments in the object store and max_restarts is not equal to 0. If " | ||
| "the arguments in the object store go out of scope or are lost, the " | ||
| "actor restart will fail."; |
There was a problem hiding this comment.
considering linking to GH issue with more context (and we need to file one!)
|
New error message @edoakes |
|
@dayshah sorry, one more wording change we should make to clarify: "constructor arguments" rather than "arguments", else people might think they're regular method args |
Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
|
I updated to make the above change |
…sma (#53713) Currently actor restarts won't because lineage ref counting doesn't work for actors with restarts. See description here #51653 (comment). Minimal repro ``` cluster = ray_start_cluster cluster.add_node(num_cpus=0) # head ray.init(address=cluster.address) worker1 = cluster.add_node(num_cpus=1) @ray.remote(num_cpus=1, max_restarts=1) class Actor: def __init__(self, config): self.config = config def ping(self): return self.config # Arg is >100kb so will go in the object store actor = Actor.remote(np.zeros(100 * 1024 * 1024, dtype=np.uint8)) ray.get(actor.ping.remote()) worker2 = cluster.add_node(num_cpus=1) cluster.remove_node(worker1, allow_graceful=True) # This line will break ray.get(actor.ping.remote()) ``` --------- Signed-off-by: dayshah <dhyey2019@gmail.com> Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com> Co-authored-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
…sma (#53713) Currently actor restarts won't because lineage ref counting doesn't work for actors with restarts. See description here #51653 (comment). Minimal repro ``` cluster = ray_start_cluster cluster.add_node(num_cpus=0) # head ray.init(address=cluster.address) worker1 = cluster.add_node(num_cpus=1) @ray.remote(num_cpus=1, max_restarts=1) class Actor: def __init__(self, config): self.config = config def ping(self): return self.config # Arg is >100kb so will go in the object store actor = Actor.remote(np.zeros(100 * 1024 * 1024, dtype=np.uint8)) ray.get(actor.ping.remote()) worker2 = cluster.add_node(num_cpus=1) cluster.remove_node(worker1, allow_graceful=True) # This line will break ray.get(actor.ping.remote()) ``` --------- Signed-off-by: dayshah <dhyey2019@gmail.com> Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com> Co-authored-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
Why are these changes needed?
Currently actor restarts won't because lineage ref counting doesn't work for actors with restarts. See description here #51653 (comment).
Minimal repro
Related issue number
Checks
git commit -s) in this PR.scripts/format.shto lint the changes in this PR.method in Tune, I've added it in
doc/source/tune/api/under thecorresponding
.rstfile.