-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Actor: don't hold key references on workers #4937
Changes from all commits
97f9d60
c3539c2
e786f39
7d780cd
0edc1a5
904f0f6
aa87db8
4176294
4dac642
cb8ca76
1cb1066
e2cee3f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -3,11 +3,11 @@ | |||
import threading | ||||
from queue import Queue | ||||
|
||||
from .client import Future, default_client | ||||
from .client import Future | ||||
from .protocol import to_serialize | ||||
from .utils import iscoroutinefunction, sync, thread_state | ||||
from .utils_comm import WrappedKey | ||||
from .worker import get_worker | ||||
from .worker import get_client, get_worker | ||||
|
||||
|
||||
class Actor(WrappedKey): | ||||
|
@@ -59,12 +59,15 @@ def __init__(self, cls, address, key, worker=None): | |||
self._client = None | ||||
else: | ||||
try: | ||||
# TODO: `get_worker` may return the wrong worker instance for async local clusters (most tests) | ||||
# when run outside of a task (when deserializing a key pointing to an Actor, etc.) | ||||
self._worker = get_worker() | ||||
except ValueError: | ||||
self._worker = None | ||||
try: | ||||
self._client = default_client() | ||||
self._future = Future(key) | ||||
self._client = get_client() | ||||
self._future = Future(key, inform=self._worker is None) | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why is the inform not always necessary? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is the key change in this PR. See #4936 (particularly the "complex case") for a description of the case this is solving. Basically, when an Actor handle got serialized and transferred to another worker, that new worker was taking out a lease on the Actor's key, which prevented that key from ever being released because the scheduler saw that some client (that worker) wanted it, which meant the scheduler never told that worker to release the key. In the end, the Actor would continue running even when nothing depended on it. But also as I mentioned in the description, I don't think this is a good implementation. I just wanted to post the problem and a quick fix, and mostly hear from @mrocklin on what the I've been thinking something like diff --git a/distributed/actor.py b/distributed/actor.py
index 77b2cda6..79e96f5b 100644
--- a/distributed/actor.py
+++ b/distributed/actor.py
@@ -49,24 +49,22 @@ class Actor(WrappedKey):
2
"""
- def __init__(self, cls, address, key, worker=None):
+ def __init__(self, cls, address, key):
self._cls = cls
self._address = address
self.key = key
self._future = None
- if worker:
- self._worker = worker
- self._client = None
- else:
- try:
- self._worker = get_worker()
- except ValueError:
- self._worker = None
+ self._client = None
+ self._worker = None
+
+ try:
+ self._worker = get_worker()
+ except ValueError:
try:
- self._client = default_client()
+ self._client = get_client()
self._future = Future(key)
except ValueError:
- self._client = None
+ pass
def __repr__(self):
return "<Actor: %s, key=%s>" % (self._cls.__name__, self.key) where we remove the If the Actor handle was sent to another worker because a task depends on that Actor's key, holding the Future is unnecessary—the fact that a task depends on that key means the scheduler won't release that key yet. The only issue could be running a task where the function happens to connect to an Actor internally (but the task has no explicit dependency on that Actor)—in that case, if the client released the Actor's key while that task was running, in theory the Actor could be cancelled while that task needed it, since the task doesn't itself hold a Future to the key (since There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That sounds a bit like the worker where the actor is deserialized is not doing a good job of cleaning up after itself. In particular, once the key/actor is removed on Worker B (think release_key), the future should be released. Once the future is released, the scheduler should trigger the proper forget lifecycle for the actor. Inspecting the actor class, we do not have a dedicated release mechanism. Therefore, the future is coupled to the lifetime of the client the future is attached to. The client will probably live as long as the worker lives and therefore will only be cleaned up once the worker closes. Even worse in same-process worker situations where clients may be shared between workers, the future may even outlive the worker which cause the reference. I argue that something like the following should do the trick diff --git a/distributed/actor.py b/distributed/actor.py
index 231cc8b3..b40e4c1a 100644
--- a/distributed/actor.py
+++ b/distributed/actor.py
@@ -64,10 +64,18 @@ class Actor(WrappedKey):
self._worker = None
try:
self._client = get_client()
- self._future = Future(key, inform=self._worker is None)
+ self._future = Future(key)
except ValueError:
self._client = None
+ def release(self):
+ if self._future:
+ self._future.release()
+ self._future = None
+
+ def __del__(self):
+ self.release()
+
def __repr__(self):
return "<Actor: %s, key=%s>" % (self._cls.__name__, self.key)
diff --git a/distributed/worker.py b/distributed/worker.py
index 44e05f05..27c9de97 100644
--- a/distributed/worker.py
+++ b/distributed/worker.py
@@ -2643,11 +2643,13 @@ class Worker(ServerNode):
self.log.append((key, "release-key", {"cause": cause}, reason))
else:
self.log.append((key, "release-key", reason))
+
if key in self.data and not ts.dependents:
- try:
- del self.data[key]
- except FileNotFoundError:
- logger.error("Tried to delete %s but no file found", exc_info=True)
+ data = self.data.pop(key)
+ from distributed.actor import Actor
+ if isinstance(data, Actor):
+ data.release()
+ del data
if key in self.actors and not ts.dependents:
del self.actors[key] however, this also deadlocks and doesn't release. this deadlock seems to connect to the recent worker state machine issues and loosely connects to #4918 I'm not entirely sure if the fixes over there would resolve that deadlock, though. To get the tests unstuck I needed to add one more patch. FWIW, I think your approach is fine. For ordinary data, workers are not able to hold references to remote data and block their release. why should it be any different for actors? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
The issue is that there may be nothing to cause the key/actor to be removed on Worker B— Let's call the key distributed/distributed/scheduler.py Line 7315 in bb991d1
Who wants it? actor-abcde 's own client. Therefore, the scheduler doesn't recommend transitioning actor-abcde to forgotten, so the scheduler will never tell the workers to release_key('actor-abcde') , so the client preventing that key from being released will never be released.
Good point. That makes me feel comfortable going to go with the approach I showed above. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a test showing that an actor is not currently cleaned up? |
||||
# ^ When running on a worker, only hold a weak reference to the key, otherwise the key could become unreleasable. | ||||
except ValueError: | ||||
self._client = None | ||||
|
||||
|
@@ -109,7 +112,8 @@ def _sync(self, func, *args, **kwargs): | |||
if self._client: | ||||
return self._client.sync(func, *args, **kwargs) | ||||
else: | ||||
# TODO support sync operation by checking against thread ident of loop | ||||
if self._asynchronous: | ||||
return func(*args, **kwargs) | ||||
return sync(self._worker.loop, func, *args, **kwargs) | ||||
|
||||
def __dir__(self): | ||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If this is running in a task (normal or actor), then the thread-local execution state must contain the right worker, no await would have happened, no? You're thinking of cases where the task itself is async?
See this line in my related unmerged PR.