Skip to content

Conversation

@dayshah
Copy link
Contributor

@dayshah dayshah commented Jun 10, 2025

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

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())

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 :(

dayshah added 2 commits June 10, 2025 12:41
…asma

Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
Copilot AI review requested due to automatic review settings June 10, 2025 20:19
@dayshah dayshah added the go add ONLY when ready to merge, run all tests label Jun 10, 2025
Copy link
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

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

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();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

returning out here so the rest of the code is not nested. All changes are only in the if (task_spec.MaxActorRestarts() != 0) block

}
if (log_message) {
RAY_LOG(ERROR)
<< "Actor " << task_spec.ActorCreationId()
Copy link
Collaborator

Choose a reason for hiding this comment

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

what does the creation ID look like? Is it possible to use the class name for a better user-facing error?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yup, added class name.
actor creation id is the actor id full message in the other comment

<< "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.";
Copy link
Collaborator

Choose a reason for hiding this comment

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

considering linking to GH issue with more context (and we need to file one!)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah
Copy link
Contributor Author

dayshah commented Jun 10, 2025

New error message @edoakes

[2025-06-10 16:49:35,865 E 19136 41860503] core_worker.cc:2726: Actor with class name: Actor and ID: 
f8c40b73b071c277d8f081ae01000000 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. See 
https://github.com/ray-project/ray/issues/53727 for more details.

@dayshah dayshah requested a review from edoakes June 10, 2025 23:52
@edoakes
Copy link
Collaborator

edoakes commented Jun 11, 2025

@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>
@edoakes
Copy link
Collaborator

edoakes commented Jun 11, 2025

I updated to make the above change

@edoakes edoakes enabled auto-merge (squash) June 11, 2025 15:14
@edoakes edoakes merged commit 3173e30 into ray-project:master Jun 11, 2025
6 checks passed
@dayshah dayshah deleted the actor-restart-error branch June 11, 2025 16:54
elliot-barn pushed a commit that referenced this pull request Jun 18, 2025
…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>
elliot-barn pushed a commit that referenced this pull request Jul 2, 2025
…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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants