-
Notifications
You must be signed in to change notification settings - Fork 17
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
Huge increase in average memory in test_spilling[release] #964
Comments
Will run an AB test to figure out whether or not the regression is real related: #966 |
A mild (10-20%) performance regression in spilling is expected as a result of merging dask/distributed#8083 (see dask/distributed#8083 (comment)). |
This looks potentially pretty bad. It seems like recursive reductions are now releasing memory later than they used to do. Next actions:
The code(runs on 5x m6i.large) a = da.random.random((94000, 94000))
a = a.persist()
distributed.wait(a)
# This is the midpoint of the plots below
b = a.sum().persist()
del a
b.compute() 2023.8.12023.9.0 |
Status update: some progress, but ultimately inconclusive. Lead 1: prioritydask/distributed#8083 changed the priority whenever you have a collection of embarassingly parallel tasks with no dependencies (e.g. Before the PR (priority, task name)
After the PR
Priority of the second stage, where all the tasks of the first stage (which are in memory) are recursively aggregated, does not change before and after the PR:
Every The order of execution of the import asyncio
from distributed import span
with span("A"):
a = da.random.random((94000, 94000))
a = a.persist()
distributed.wait(a)
async def spill_everything(dask_worker):
while dask_worker.data.fast:
dask_worker.data.evict()
await asyncio.sleep(0)
client.run(spill_everything)
input() # Pause until user hits resume
with span("B"):
b = a.sum().persist()
del a
b.compute() The two task plots remain substantially different: BeforeAfterIt's important to observe how the workers can contain exactly 170 tasks in memory before they're forced to spill. This number is clearly seen at all points while computing There is a round-trip to the scheduler:
As soon as 4 happens, the worker will start again with the next This is not the case without the Lead 2: round-trip durationI wanted to measure the round-trip described above. Here's the code that calculates the delta between the worker adding events = client.run(lambda dask_worker: list(dask_worker.state.stimulus_log))
from collections import defaultdict
from dask.utils import key_split
from distributed.worker_state_machine import ComputeTaskEvent, ExecuteSuccessEvent, FreeKeysEvent
deltas = []
for evs in events.values():
success = {}
for ev in evs:
if isinstance(ev, ExecuteSuccessEvent) and key_split(ev.key) == "sum":
key = ev.key if isinstance(ev.key, tuple) else eval(ev.key)
success[key[1:]] = ev
elif isinstance(ev, FreeKeysEvent):
assert len(ev.keys) == 1
key = next(iter(ev.keys))
if not isinstance(key, tuple):
key = eval(key)
if key_split(key) == "random_sample":
if key[1:] in success:
deltas.append(ev.handled - success[key[1:]].handled)
import pandas
pandas.Series(deltas).hist(bins=50) This plot looks roughly the same before and after dask/distributed#8083. However, disabling the memory monitor ( I still have no clue why the round-trip time looks so precisely centered around multiples of 1 second. Lead 3: work stealing / bad scheduler decisions99% of the work of a However I've measured that the number of such cases is negligible both before and after. Side by sideHere's how the second phase of the workload (starting from everything spilled out) looks like, side by side. left: immediately before dask/distributed#8083 Peek.2023-09-15.18-06.mp4As you can see, before the worker is just... faster. I still can't figure out why. Finally, Fine Performance Metrics are inconclusive: As expected, the vast majority of the time is between
Note how the total of time spent on a blocked event loop roughly matches the disk-read time. This is unsurprising, as the workers have 2 threads each - so while one is unspilling, the other is uncapable of operating. The time that can be spent while unspilling, the execution time, is too negligible in this case to matter. |
TL;DR; version of the above mega-post
Async spilling would drastically improve this, as it would drop the time during which a task remains in memory unnecessarily from 1~4 seconds to tens of milliseconds.
|
Workflow Run URL
The text was updated successfully, but these errors were encountered: