Active Memory Management Control System
See original GitHub issueSystem to Control Active Memory Management
There has been a lot of side conversations around active memory management, but I don’t think we’ve ever written this up properly (aside from maybe https://github.com/dask/distributed/issues/1002 , which I’m now closing)
Dask currently replicates data around the cluster whenever a dependency on one worker needs to be moved for a dependent on another worker. This passive approach is good because highly needed data tends to spread around, but bad because excessive copies may start to take up distributed memory.
Motivating use cases
There are many situations where we want to more actively control the management of data on the cluster. I’ll list a few below:
-
Data that we know will be in high demand should be replicated proactively.
Demand might be either because of many not-yet-run dependencies, or as insurance for workers that might go down
-
Data that has been replicated may no longer be necessary and it may be a good idea to clear it out
-
When we shut down a worker we want to offload its data before retiring it
-
If we’re able to reliably shut down workers then we would like to cycle their processes to avoid issues like memory leaks
-
When a worker is running low on space we may want to offload its data rather than sending it to disk
-
When new workers arrive they should probably proactively pull data away from some of their more saturated peers
-
Advanced users will want to define their own special policies
-
The user may specify replication requests to manually motivate one of the above reasons
-
…
Challenges
However, making a general purpose system that can solve all of these problems (and more) is hard for a few reasons. I’ll list a few below:
- Efficiency: We need to efficiently and quickly identify what data should move where, even in cases where we have millions of tasks and thousands of workers
- Consistency: Moving this data can be error prone during cluster activity. This requires a subtle dance between scheduler, possibly multiple workers, and maybe other active policies
- Customizability: We are not smart enough to predict the right approach here and implement a fully general solution on the first try. Instead we probably need to make a system that allows for a variety of overlapping user-defined policies.
Existing work on rebalance logic
There is currently some work from @crusaderky on a special but important case of this problem, making rebalance faster and more robust. The idea here being that this is a good and general first step towards a larger system.
This is good work because it solves a tricky problem of how to quickly identify data that should be moved, and good targets to accept this data. It also starts to think about robust movement of data.
This is one example of logic that folks might want to implement
Proposed Control System
At the same time, I’d like for us to start thinking about a control system for these policies. I’ll outline something below to serve as a first pass. This is the simplest system I could think of that can maybe solve the problems listed above.
We have a variety of memory management policies which are invoked periodically, they also optionally have a finished criterion, which will clear them from the system.
Example, retire workers method
So retire workers might look like the following:
async def retire_workers(self, workers):
# register several new policies with the memory manager
policies = [RetireWorkerPolicy(self, worker) for worker in workers]
# wait until they are finished
await asyncio.gather(*policies)
# close workers now that data has been moved
await asyncio.gather(*[self.close_worker(worker) for worker in workers])
These RetireWorkerPolicy objects would live in some plugin for a while until they finshed up. There might be a variety of other policies in there, some long-lived, some short-lived
>>> scheduler.memory_manager.policies
[<RetireWorkers(tcp://..., status: waiting, period: 100ms)>,
<RetireWorkers(tcp://..., status: waiting, period: 100ms)>,
<RetireWorkers(tcp://..., status: waiting, period: 100ms)>,
<RetireWorkers(tcp://..., status: waiting, period: 100ms)>,
<WatchForExcessMemory(period: 1s)>,
...
<Replicate(task: sklearn-model-123, replicas: 3, period: 500ms)>
...
]
Actions
Each policy, when invoked, emits a set of proposed actions.
Actions can be one of two forms:
- Ask a worker to acquire a piece of data
- Ask a worker to delete a piece of data, if it agrees
Note that we don’t merge these two, as we did in replicate/rebalance. This is error prone and requires careful attention. I think that by separating these two we can achieve what we want with more stability.
Acquire data
Workers already have a solid way to acquire a piece of data in a noisy cluster. They use this today to gather dependencies. Let’s reuse that logic.
However to avoid consistency issues I suspect that we’re going to need to keep track of what data each workers are currently trying to obtain. Otherwise, for data that takes a long time to acquire we might mistakenly ask many workers to get the same piece of data.
To address this I think that we could start tracking not only what data each
worker has, but what data it is trying to get. We would also have to track the
same information on the task. This would be a dynamic version of
TaskState.who_has
and WorkerState.has_what
. Something like
TaskState.who_is_acquiring
and WorkerState.collecting
but with better
names.
Remove data
Similarly with adding data …
- We already have a system for the scheduler to encourage workers to remove data
- Things may change in the meantime, and workers may disagree (maybe they were just recently assigned a task that requires the piece of data as a dependency.
Unlike with adding data, I suspect that we can do this without adding extra state to the workers or tasks. I suspect that we can ask the worker to remove a piece of data, and then immediately mark that data as removed so that future scheduling decisions won’t depend on it. If for some reason the worker disagrees then it can send back its usual “I now have this data” message and things may be ok.
Example, Replicate policy
Then, something like replicate might look like the following:
class Replicate(MemoryManagementPolicy):
period = "1s"
def __init__(self, task: TaskState, replicas: int):
self.task = task
self.replicas = replicas
async def update(self):
if self.task.name not in self.scheduler.tasks:
self.finished.set()
return []
count = len(self.task.who_has) + len(self.task.who_is_acquiring)
if count < self.replicas:
ws = self.scheduler.get_worker_with_lots_of_memory()
return [("get", ws, ts)] # some other return convention is probably better
elif count > self.replicas
ws = # find most saturated worker
return [("remove", ws, ts)] # some other return convention is probably better
Some things to note here:
- We’re not doing the work to move the data around. We’re going to trust the system to do this. I hope that the system can be relatively simple, given the simple operations that we have.
- This policy closes itself off once the task has left the scheduler.
- We don’t try to set every possible worker change in one go. We could if wanted to by returning multiple actions, but we don’t have to. We’re making the world a bit better. This system will be called again shortly, so we don’t need to be super-aggressive here. We just need to improve the world a bit.
Control loop
I am hopeful (probably naively so) that the control loop can be fairly simple. I hope that the following code gets my point across
class ActiveMemoryManager(SchedulerExtension):
def __init__(self, scheduler):
self.scheduler = scheduler
self.periodic_callbacks = {}
def add_policy(self, policy: MemoryManagementPolicy):
policy.scheduler = self.scheduler
pc = PeriodicCallback(functools.partial(self.run, policy=policy.update), interval=parse_timedelta(policy.period))
self.periodic_callbacks[policy] = pc
asyncio.ensure_future(self.cleanup(policy)) # TODO: avoid dangling futures
def run(self, policy):
recommendations = policy.update()
for action, ws, ts in recommendations:
if action == "get":
self.send_to_worker(ws.address, {"op": "gather_data", "keys": [ts.key]})
elif action == "remove":
self.send_to_worker(ws.address, {"op": "remove_data", "keys": [ts.key]})
async def cleanup(self, policy)
await policy.finished
pc = self.periodic_callbacks.pop(policy)
pc.stop()
Issue Analytics
- State:
- Created 2 years ago
- Comments:26 (20 by maintainers)
Top GitHub Comments
@mrocklin
@fjetter and I had a chat. The agreement is that, in a first iteration, the system that deletes unwanted replicas will use Scheduler.rpc (what rebalance and replicate use now). This has been called “option 2” above in the discussion. This means that, as of the initial iteration, the deletion won’t be safe to run in the middle of a computation. Also as of the initial iteration, the cost will be O(n) to the total number of tasks in the scheduler, as already discussed.
In the meantime, @fjetter will have the time to complete #5046. The key feature that’s not in the PR yet is that, whenever the scheduler sends a fire-and-forget request for the deletion of a key and the worker decides NOT to delete it, the worker must report back its decision. On the scheduler side, TaskState will keep track of the pending deletion requests. This extra state is necessary to avoid either accidentally deleting all copies of a key or accidentally ending up with 2 or more replicas of the same key in perpetuity.
Once both of the above are done, it should be pretty much trivial to switch system (literally delete the 2 lines of self.rpc and replace them with the enqueueing into the bulk comms). This will be simple because already as of the initial design the whole thing won’t need to wait for feedback from the workers. From a user’s perspective, this will enable safely discarding unwanted key replicas in the middle of a computation.
As a separate discussion, we are both very concerned about how the policy plugin system is going to handle conflicting decisions between policies, and we would really like to see some real-life use cases of what a user-defined policy could want to achieve (at functional analysis level) before a design is finalised.
Last note about the gathering: We do have a mechanism right now to gather keys on the worker side which circumvents the ordinary state machine and instead is an explicit RPC call, namely the
gather: Worker.gather
comm handler. This is much simpler and does not deal with business, retries, missing data, metrics collection, etc. For a simple mechanism to start with, this might be sufficient (That’s what the current rebalance/replicate is built on)