Skip to content
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

an example that shows the need for memory backpressure #2602

Closed
rabernat opened this issue Mar 18, 2019 · 106 comments
Closed

an example that shows the need for memory backpressure #2602

rabernat opened this issue Mar 18, 2019 · 106 comments

Comments

Labels
None yet
Projects
None yet
Linked pull requests

Successfully merging a pull request may close this issue.

None yet
@rabernat
Copy link

@rabernat rabernat commented Mar 18, 2019

In my work with large climate datasets, I often concoct calculations that cause my dask workers to run out of memory, start dumping to disk, and eventually grind my computation to a halt. There are many ways to mitigate this by e.g. using more workers, more memory, better disk-spilling settings, simpler jobs, etc. and these have all been tried over the years with some degree of success. But in this issue, I would like to address what I believe is the root of my problems within the dask scheduler algorithms.

The core problem is that the tasks early in my graph generate data faster than it can be consumed downstream, causing data to pile up, eventually overwhelming my workers. Here is a self contained example:

import dask.array as dsa

# create some random data
# assume chunk structure is not under my control, because it originates
# from the way the data is laid out in the underlying files
shape = (500000, 100, 500)
chunks = (100, 100, 500)
data = dsa.random.random(shape, chunks=chunks)

# now rechunk the data to permit me to do some computations along different axes
# this aggregates chunks along axis 0 and dis-aggregates along axis 1
data_rc = data.rechunk((1000, 1, 500))
FACTOR = 15


def my_custom_function(f):
    # a pretend custom function that would do a bunch of stuff along
    # axis 0 and 2 and then reduce the data heavily
    return f.ravel()[::15][None, :]

# apply that function to each chunk
c1 = math.ceil(data_rc.ravel()[::FACTOR].size / c0)
res = data_rc.map_blocks(my_custom_function, dtype=data.dtype,
                         drop_axis=[1, 2], new_axis=[1], chunks=(1, c1))

res.compute()

(Perhaps this could be simplified further, but I have done my best to preserve the basic structure of my real problem.)

When I watch this execute on my dashboard, I see the workers just keep generating data until they reach their memory thresholds, at which point they start writing data to disk, before my_custom_function ever gets called to relieve the memory buildup. Depending on the size of the problem and the speed of the disks where they are spilling, sometimes we can recover and manage to finish after a very long time. Usually the workers just stop working.

This fail case is frustrating, because often I can achieve a reasonable result by just doing the naive thing:

for n in range(500):
    res[n].compute()

and evaluating my computation in serial.

I wish the dask scheduler knew to stop generating new data before the downstream data could be consumed. I am not an expert, but I believe the term for this is backpressure. I see this term has come up in #641, and also in this blog post by @mrocklin regarding streaming data.

I have a hunch that resolving this problem would resolve many of the pervasive but hard-to-diagnose problems we have in the xarray / pangeo sphere. But I also suspect it is not easy and requires major changes to core algorithms.

Dask version 1.1.4

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Mar 18, 2019

Thanks for the writeup and the motivation @rabernat . In general I agree with everything that you've written.

I'll try to lay out the challenge from a scheduling perspective. The worker notices that it is running low on memory. The things that it can do are:

  1. Run one of the many tasks it has sitting around
  2. Stop running those tasks
  3. Write data to disk
  4. Kick tasks back to the scheduler for rescheduling

Probably it should run some task, but it doesn't know which tasks generate data, and which tasks allow it to eventually release data. In principle we know that operations like from_hdf5 are probably bad for memory and operations like sum are probably good, but we probably can't pin these names into the worker itself.

One option that I ran into recently is that we could slowly try to learn which tasks cause memory to arrive and which tasks cause memory to be released. This learning would happen on the worker. This isn't straightforward because there are many tasks running concurrently and their results on the system will be confused (there is no way to tie a system metric like CPU time or memory use to a particular Python function). Some simple model might give a decent idea over time though.

We do something similar (though simpler) with runtime. We maintain an exponentially weighted moving average of task run time, grouped by task prefix name (like from-hdf5), and use this for scheduling heuristics.

This approach would also be useful for other resource constraints, like network use (it'd be good to have a small number of network-heavy tasks like from-s3 running at once), and the use of accelerators like GPUs (the primary cause of my recent interest).

If someone wanted to try out the approach above my suggestion would be to ...

  1. Create a periodic callback on the worker that checked the memory usage of the process with some frequency
  2. Look at the tasks running (self.executing) and the memory growth since the last time and adjust some model for each of those tasks' prefixes (see key_split)
  3. That model might be very simple, like the number of times that memory has increased while seeing that function run. Greater than 0 means that memory increased more often than decreased and vice versa.
  4. Look at the policies in Worker.memory_monitor and maybe make a new one
    • Maybe we go through self.ready and reprioritize?
    • Maybe we set a flag so when we pop tasks from self.ready we only accept those that we think reduce memory use?
    • ...

There are likely other solutions to this whole problem. But this might be one.

Loading

@martindurant
Copy link
Member

@martindurant martindurant commented Mar 18, 2019

there is no way to tie a system metric like CPU time or memory use to a particular Python function

but we do measure the memory usage of the inputs and outputs of functions that have run (not the internal transient memory), and also know whether running of a function ought to free the memory used by its inputs. If measurements were done on the prefix basis mentioned, there could be a reasonable guess at the memory implications of running a given task.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented Mar 18, 2019

Thanks a lot for your quick response.

One quick clarification question... You say that a central challenge is that the scheduler

doesn't know which tasks generate data, and which tasks allow it to eventually release data

In my mental model, this is obvious from the input and output array shapes. If a task has an input which is 1000x1000 and an output which is 10x10, it is a net sink of memory. The initial nodes of the graph are always sources. I assumed that the graph must know about these input and output shapes, and the resulting memory footprint, even if it has no idea how long each task will take. But perhaps I misunderstand how much the scheduler knows about the tasks it is running.

Wouldn't it be easier to expose this information to the scheduler than it would be to rig up an adaptive monitoring solution?

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Mar 18, 2019

but we do measure the memory usage of the inputs and outputs of functions that have run (not the internal transient memory), and also know whether running of a function ought to free the memory used by its inputs. If measurements were done on the prefix basis mentioned, there could be a reasonable guess at the memory implications of running a given task.

That's a good point, and it's much easier to measure :)

In my mental model, this is obvious from the input and output array shapes. If a task has an input which is 1000x1000 and an output which is 10x10, it is a net sink of memory. The initial nodes of the graph are always sources. I assumed that the graph must know about these input and output shapes, and the resulting memory footprint, even if it has no idea how long each task will take. But perhaps I misunderstand how much the scheduler knows about the tasks it is running.

The scheduler doesn't know about shapes or dtypes of your arrays. It only knows that it is running a Python function, and that that function produces some outputs. You're thinking about Dask array, not Dask.

@martindurant 's suggestion is probably enough for your needs though.

Loading

@martindurant
Copy link
Member

@martindurant martindurant commented Mar 18, 2019

The scheduler doesn't know about shapes or dtypes of your arrays

I suppose the client does, at least for arrays, so there could be another route to pass along the expected output memory size of some tasks. In the dataframe case, the client might know enough, and in the general case, there could be a way for users to specify expected output size.

Loading

@guillaumeeb
Copy link
Member

@guillaumeeb guillaumeeb commented Mar 18, 2019

Hi everyone,

This is a subject of interest to me too. But I thought that Dask tried already to minimize memory footprint as explained here. Obviously, this is not what @rabernat is observing, and I've seen the same behavior as him on similar use cases.

Couldn't we just give an overall strategy for the Scheduler to use? Like work on the depth of the graph first?

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Mar 18, 2019

But I thought that Dask tried already to minimize memory footprint as explained here.
Couldn't we just give an overall strategy for the Scheduler to use? Like work on the depth of the graph first?

The scheduler does indeed run the graph depth first to the extent possible (actually, it's a bit more complex than this, but your depth-first intuition is correct). Things get odd though if

  1. Some tasks that we would want to run are blocked by other things, like data transfer times. Dask might move on to other tasks
  2. Some tasks that are available we might not want to run, even if we can. We'd prefer to sacrifice parallelism and wait rather than allocate more memory
  3. Some tasks in our graph may be much more expensive in terms of memory than others, and so depth first may not be enough of a constraint to choose the optimal path

Loading

@sjperkins
Copy link
Contributor

@sjperkins sjperkins commented Mar 19, 2019

These issues also impact my use cases.

One thing I've considered is rather than creating a graph purely composed of parallel reductions:

a     b  c   d   e  f    g  h
  \  /    \ /     \ /    \ /
    \     /         \    /
     \  /             \ /
       \              /
         \           /
           \        /
             \     /
               \ /       

is to define some degree of parallelism (2 in this example) and then create two "linked lists", where the results of a previous operation are aggregated with those of the current.

a    e
|    |
b    f
|    |
c    g
|    |
d    h
  \ /

I haven't tried this out yet, so not sure to what extent it would actually help, but its on my todo list to mock the above up with dummy ops.

I suppose the client does, at least for arrays, so there could be another route to pass along the expected output memory size of some tasks. In the dataframe case, the client might know enough, and in the general case, there could be a way for users to specify expected output size.

In this context it's probably worth re-raising Task Annotations again: dask/dask#3783, which in a complete form would allow annotating a task with an estimate of it's memory output. I started on it in #2180 but haven't found time to push it forward.

Loading

@abergou
Copy link

@abergou abergou commented Mar 19, 2019

This also affects my work-flow so I would be curious to see progress on it. On our end, writing out to disk doesn't help since disk is also a finite resource. We have a few workarounds such as adding artificial dependencies into a graph or submitting a large graph in pieces.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented Mar 22, 2019

Thanks everyone for the helpful discussion. It sounds like @mrocklin and @martindurant have identified a concrete idea to try which could improve the situation. This issue is a pretty high priority for us in Pangeo, so we would be very happy to test any prototype implementation of this idea.

Loading

@mrocklin mrocklin transferred this issue from dask/dask Apr 10, 2019
@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented Apr 17, 2019

I’ll be on paternity leave for at least the next two weeks (maybe longer, depending on how things are going). If this is still open then I’ll tackle it then.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented May 22, 2019

Hi folks...I'm just pinging this issue to remind us that it remains a crucial priority for Pangeo.

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 22, 2019

👍 it's on my list for tomorrow :)

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 23, 2019

https://nbviewer.jupyter.org/gist/TomAugspurger/2df7828c22882d336ad5a0722fbec842 has a few initial thoughts. The first problem is that the rechunking from along just axis 0 to just axis 1 cause a tough, global communication pattern. The output of my_custom_function will need input from every original chunk.

image

So for this specific problem it may be an option to preserve the chunks along axis 0, and only add additional chunks along axis 1. This leads to a better communication pattern.

image

But, a few issues with that

  1. It may not work for the real problem (@rabernat do you have a "real world" example available publicly somewhere?). We should still attempt a proper solution to the problem.
  2. It increases the number of tasks, which may bump up against other scheduler limitations.

I'll keep looking into this, assuming that a different chunking pattern isn't feasible.

(FYI @mrocklin the HTML array repr came in handy).

Loading

@martindurant
Copy link
Member

@martindurant martindurant commented May 23, 2019

@TomAugspurger , the general problem remains interesting, though, of using the expected output size and known set of things that could be dropped of a given tasks as an additional heuristic in determining when/where to schedule that task.

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 23, 2019

Yep.

I notice now that my notebook oversimplified things. The original example still has chunks along the first and second axis before the map_blocks (which means the communication shouldn't be entirely global). I'll update things.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented May 23, 2019

This is a "real world" example that can be run from ocean.pangeo.io which I believe illustrates the core issue. It is challenging because the chunks are big, leading to lots of memory pressure.

import intake
cat = intake.Catalog('https://raw.githubusercontent.com/pangeo-data/pangeo-datastore/master/intake-catalogs/ocean.yaml')
ds = cat.GODAS.to_dask()
print(ds)
salt_clim = ds.salt.groupby('time.month').mean(dim='time')
print(salt_clim)

# launch dask cluster
from dask.distributed import Client
from dask_kubernetes import KubeCluster
# using more workers might alleviate the problem, but that is not a satisfactory workaround
cluster = KubeCluster(n_workers=5)
client = Client(cluster)

# computation A
# compute just one month of the climatology
# it works fine, indicating that the computation could be done in serial
salt_clim[0].load()

# computation B
# now load the whole thing
# workers quickly run out of memory, spill to disk, and even get killed
salt_clim.load()

image

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 24, 2019

Thanks @rabernat. I'll try it out on the pangeo binder once I have something working locally.

I think (hope) I have a workable solution. My basic plan is

  1. Track the cumulative output .nbytes of tasks at the prefix-level (easy)
  2. When deciding how to schedule under load, look up / compute the relative size
    of the input to the output, again at the prefix level. Prioritize tasks that look
    like they'll reduce memory usage. (seems harder. I don't know if we use
    prefix-level relationships anywhere else in scheduling decisions).

So if we currently have high memory usage, and we see a set of tasks like

      a1   a2  # 5 bytes, 5 bytes   |   x1  x2   # 5 bytes, 5 bytes
        \ /                         |    \ /
         b     # 1 byte             |     y      # 20 bytes

We would (ideally) schedule b before y, since it tends to have a net
decrease in memory usage. Or if we have tasks running that we know will free up
some memory, we might wait to schedule y since it tends to increase memory.
I'm sure there are things I'm missing, but this should provide a good start.

Loading

@martindurant
Copy link
Member

@martindurant martindurant commented May 24, 2019

@TomAugspurger , that's exactly how I was picturing things. Additionally, you may need to know whether calculating b, for example, actually releases a1 and a2.

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 24, 2019

you may need to know whether calculating b, for example, actually releases a1 and a2.

Yeah, I've been looking through the recommendations -> release logic now. I'm wondering if we can also keep a counter of "completing this task recommended that we release this many bytes". That seems a bit easier to track than knowing whether a1 and a2 were actually released.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented May 24, 2019

As a user of dask.array, it confuses me that the graph itself doesn't have this information in it already. In my mind, all of my dask.array operations know about the shape and dtype of their inputs and outputs. Consequently, the memory impact of a task is known a-priori.

I do understand that this information is not encoded into the dask graph--all the graph knows about is tasks and their dependencies. But, to my naive interpretation, it seems harder to try to "learn" the memory impact of tasks based on past experience than it does to pass this information through the graph itself, in the form of task annotation.

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 24, 2019

As a user of dask.array, it confuses me that the graph itself doesn't have this information in it already. In my mind, all of my dask.array operations know about the shape and dtype of their inputs and outputs. Consequently, the memory impact of a task is known a-priori.

As you say, TaskAnnotations mentioned by @sjperkins in #2602 (comment) would let us pass that information through. I think that approach should be considered, but I have two reasons for pursing the "learning" approach within distributed for now

  1. I suspect it's easier to get a rough implementation of this in place. Task annotations will (I think) be a larger change to dask.
  2. There always be cases where the user doesn't or can't provide the expected memory usage when the graph is constructed. Say as a result of a boolean filter, or when using delayed on a source that can't easily provide the memory usage information.

I will take another look at the task annotation issue though.

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 24, 2019

A promising note, the additional LOC for the tracking seems to be ~3 :) I'm just keeping a task_net_nbytes that tracks the difference between task input and output at the prefix-level. With that, we correctly see that

  1. random sample is a net increaser of memory
  2. rechunking has no effect on memory (again, we aren't measuring transient memory here, just final).
  3. my_custom_function is a net reducer of memory
--------------------------------------------------------------------------------
# task_net_nbytes after all the computation is done.
{'my_custom_function': -18666656,
 'random_sample': 20000000,
 'rechunk-merge': 0,
 'rechunk-split-rechunk-merge': 0}

The scheduling changes will be harder, but this seems promising so far.

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented May 24, 2019

@TomAugspurger said

We would (ideally) schedule b before y, since it tends to have a net
decrease in memory usage

Sounds reasonable. As a warning, there might be other considerations fighting for attention here. Scheduling is an interesting balancing game.

@martindurant said

Additionally, you may need to know whether calculating b, for example, actually releases a1 and a2.

This is a nice point. One naive approach would be to divide the effect by the number of dependents of the dependency-to-be-released.

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented May 24, 2019

Also, I'm very glad to have more people thinking about the internal scheduling logic!

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 24, 2019

@mrocklin do you know off hand whether the changes to the scheduling logic are likely to be on the worker, scheduler, or both? I can imagine both situations occurring, so I suspect the answer is "both".

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented May 24, 2019

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented May 24, 2019

I could also imagine us not asking the client to run dask.order at all, and doing all task prioritization on the workers. This would allow us to integrate the byte sizes into the dask.order computation (and remove it from the single-threaded client-scheduler processing). This is probably something to explore as future work though.

Loading

@TomAugspurger
Copy link
Member

@TomAugspurger TomAugspurger commented May 28, 2019

I could also imagine us not asking the client to run dask.order at all, and doing all task prioritization on the workers.

FWIW, dask.order does correctly order this workload. The final task of each parallel branch comes before the first task of the next branch.

Unknown

So this looks more like a problem with reordering / waiting to execute some tasks, if we notice that we

  1. are at high memory
  2. Have a set of ready tasks that are likely to reduce memory usage.

Loading

@apatlpo
Copy link
Contributor

@apatlpo apatlpo commented Jun 4, 2021

Hi all, thank you very much for exposing this challenge @rabernat !
The issue was very helpful to our team.

I am not sure this is the right place for it but don't know where to push it otherwise.
I deeply apologize if this is not the place to report on this.

We have been using for some time an helper method in order to sequentially process large datasets along embarrassingly parallel dimensions and I'd like to share it: here

The method has proven to be very valuable with time and the hope is that it can help other people.
There are most likely other approaches to do this and I would be very much looking forward to hear about it.

Loading

@gjoseph92
Copy link
Contributor

@gjoseph92 gjoseph92 commented Jun 26, 2021

So #4892 helps with this a lot. On my laptop (8-core intel MPB, 32GB memory), @rabernat's original example at the top of of this issue takes:

  • main: 43 minutes, spills ~130GB data to disk, makes my system largely unresponsive
  • #4892: 3.3 minutes, no spilling, reasonable memory usage

main (performance report):
Screen Shot 2021-06-25 at 6 31 53 PM
#4892 (performance report):
Screen Shot 2021-06-25 at 6 32 06 PM

I want to give a big shout-out to @JSKenyon, who so clearly identified the problem in #4864, and @fjetter who started us towards a solution. Dask had been scheduling the initial data-creation tasks in a very bad way, leading to huge amounts of unnecessary data transfer, and therefore data duplication, and therefore ballooning memory usage.

I made two changes before running these trials:

  1. Added a .copy() at the end of the "pretend memory-reducing function":

    def my_custom_function(f):
        # a pretend custom function that would do a bunch of stuff along
        # axis 0 and 2 and then reduce the data heavily
        return f.ravel()[::15][None, :].copy()  # <-- key: copy the slice to release `f`

    This is essential, because the small slice of f we were returning was just a view of f's memory. And that meant all of f's memory had to stick around—so the memory-reducing function was not reducing memory at all! The NumPy docs even mention this:

    Care must be taken when extracting a small portion from a large array which becomes useless after the extraction, because the small portion extracted contains a reference to the large original array whose memory will not be released until all arrays derived from it are garbage-collected. In such cases an explicit copy() is recommended.

    This is unintuitive behavior, but I wonder how often it happens unknowingly with Dask, and if it plays a role in real-world cases. I wonder if we could add logic to Dask to warn you of this situation, where the memory backing an array result is, say, 2x larger than memory needed for the number of elements in that array, and suggest you make a copy.

  2. Ran under jemalloc to avoid the unreleased memory issue (@crusaderky has explained this clearly in the Dask docs):

    DYLD_INSERT_LIBRARIES=$(brew --prefix jemalloc)/lib/libjemalloc.dylib python backpressure.py

    It probably would have still worked without this, but would have spilled more and been messier.

It's also interesting that memory backpressure wouldn't have helped with this exact example snippet. Because of the view-vs-copy issue, that code was essentially trying to compute all 93 GiB of the original data into memory. No matter your order through the graph, that end result's just not possible.

Not to say that we won't ever need memory backpressure (xref #4891). But I wonder how many other cases that feel like Dask is producing more data than it can consume actually involve other sly bugs like the ones here. (Both the view-vs-copy and the poor ordering of root tasks were sly bugs, on behalf of a user and of dask.) And if there are tools/visualizations to help identify these issues that would serve us better than adding more complexity (memory backpressure) to try to work around them.

I'd love to hear if folks who have also run into these memory problems could retry their workloads with this update and report back how things change?

Loading

@itamarst
Copy link

@itamarst itamarst commented Jun 26, 2021

But I wonder how many other cases that feel like Dask is producing more data than it can consume actually involve other sly bugs like the ones here. ... And if there are tools/visualizations to help identify these issues that would serve us better than adding more complexity (memory backpressure) to try to work around them.

For issues that involve allocating too much memory or holding on to too much memory, you can figure out the sources of peak memory usage using the Fil memory profiler. Though for now you'll need to use the threaded or single-threaded Dask schedulers, it doesn't support subprocesses yet.

Loading

@gjoseph92
Copy link
Contributor

@gjoseph92 gjoseph92 commented Jun 29, 2021

I've tried rerunning a couple examples from this thread under the new PR. Performance is greatly improved.

@rabernat's canonical anomaly-mean example from #2602 (comment)

20-worker cluster with 2-CPU, 10GiB workers

@mrocklin originally said about this example:

I don't think that there is a way to compute that example in small memory without loading data twice. You have to load all of the data in order to compute clim. And then you need that data again to compute anom_mean.

The scheduling ask here is "be willing to delete and recompute data" while the scheduling ask before was "prefer tasks that release memory, and be willing to hold off a bit on tasks that increase net data".

This is still true. However in this example, data was only 200GiB, which is a reasonable size for a cluster (mine had exactly 200GiB total memory). So this still won't work on your laptop. But on a cluster, when we place the initial chunks of data well so they don't have to be shuffled around later, we can actually compute this quite efficiently.

@dougiesquire's climactic mean example from #2602 (comment)

20-worker cluster with 2-CPU, 20GiB workers

Screen Shot 2021-06-29 at 3 04 01 PM

FWIW, even with the colocation PR, this struggled when workers only had 8GiB of memory. Bumping to 20GiB smoothed things out (well, not for main). It would still be really cool if dask/distributed could tell you what amount of memory per worker was necessary to compute a given graph.

Loading

@dougiesquire
Copy link

@dougiesquire dougiesquire commented Jun 29, 2021

Wow, this is really terrific @gjoseph92 ! Thanks heaps. Looking forward to giving it a spin.

Loading

@Thomas-Moore-Creative
Copy link

@Thomas-Moore-Creative Thomas-Moore-Creative commented Jun 29, 2021

@gjoseph92 - that left-hand dashboard looks sadly familiar but the one on the right looks really exciting! Fantastic.

It would still be really cool if dask/distributed could tell you what amount of memory per worker was necessary to compute a given graph.

+1
@mrocklin how much crowd-sourcing do you think we need to make that happen? ;-)

Loading

@RichardScottOZ
Copy link

@RichardScottOZ RichardScottOZ commented Jun 29, 2021

So 200GiB problem required 400GiB cluster memory to run smoothly? Seems like what happens generally from experience?

Be interesting to try that same test in the single large machine case - e.g. run the above ona 512GB machine what happens?

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Jun 29, 2021

@mrocklin how much crowd-sourcing do you think we need to make that happen? ;-)

Have you filled out the dask survey yet? dask.org/survey

A lot of what we're working on now is motivated by responses there. Although this work itself comes from generous sponsorship from @rabernat and crew.

Loading

@Thomas-Moore-Creative
Copy link

@Thomas-Moore-Creative Thomas-Moore-Creative commented Jun 29, 2021

(hopefully) obviously it was a cheeky comment @mrocklin - but your response got me thinking about how other orgs ( beyond @rabernat et al ) might try to help resource these efforts.

Thanks for all the hard work from all contributors. Going to hit that survey now.

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Jun 29, 2021

Tell all your friends!

Loading

@sjperkins
Copy link
Contributor

@sjperkins sjperkins commented Jun 30, 2021

Really well done by all involved!

Loading

@JSKenyon
Copy link

@JSKenyon JSKenyon commented Jun 30, 2021

This does look like a huge improvement! I look forward to trying it out on my problems.

Loading

@dcherian
Copy link

@dcherian dcherian commented Jun 30, 2021

EXciting!

This is essential, because the small slice of f we were returning was just a view of f's memory. And that meant all of f's memory had to stick around—so the memory-reducing function was not reducing memory at all!
This is unintuitive behavior, but I wonder how often it happens unknowingly with Dask, and if it plays a role in real-world cases. I wonder if we could add logic to Dask to warn you of this situation, where the memory backing an array result is, say, 2x larger than memory needed for the number of elements in that array, and suggest you make a copy.

@gjoseph92 This happens all the time! See dask/dask#3595 (especially dask/dask#3595 (comment)) @mrocklin proposed a solution here: dask/dask#3595 . I use the map_blocks(np.copy) trick frequently

Loading

@gjoseph92
Copy link
Contributor

@gjoseph92 gjoseph92 commented Jun 30, 2021

@dcherian thanks for the xref! dask/dask#3595 (comment) is exactly what I was thinking. (It wouldn't help with user code in map_blocks like this case specifically, but since that's advanced, maybe that's okay.) I might like to take this on.

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Jun 30, 2021

Loading

@mrocklin mrocklin closed this in fc47318 Jun 30, 2021
@mrocklin
Copy link
Member

@mrocklin mrocklin commented Jun 30, 2021

I'm closing this out. If folks have other issues here though then I'm happy to reopen.

@rabernat , thank you for opening this originally. My apologies that it took so long to come to a good solution.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented Jul 30, 2021

Which dask release do users need to be running to get this feature? My impression is 2021.07.0, correct?

Loading

@mrocklin
Copy link
Member

@mrocklin mrocklin commented Jul 30, 2021

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented Aug 2, 2021

I am working on a blog post to advertise the exciting progress we have made on this problem. As part of this, I have created an example notebook that can be run in Pangeo Binder (including with Dask Gateway) with two different versions of Dask which attempts to reproduce the issue and its resolution.

New Dask version (2021.07.1): https://binder.pangeo.io/v2/gh/pangeo-gallery/default-binder/39f7202?urlpath=git-pull%3Frepo%3Dhttps%253A%252F%252Fgist.github.com%252Frabernat%252F39d8b6a396e076d168c24167b8871c4b%26urlpath%3Dtree%252F39d8b6a396e076d168c24167b8871c4b%252Fanomaly_std.ipynb%26branch%3Dmaster

Older Dask version (2020.12.1): https://binder.pangeo.io/v2/gh/pangeo-gallery/default-binder/b8d1c53?urlpath=git-pull%3Frepo%3Dhttps%253A%252F%252Fgist.github.com%252Frabernat%252F39d8b6a396e076d168c24167b8871c4b%26urlpath%3Dtree%252F39d8b6a396e076d168c24167b8871c4b%252Fanomaly_std.ipynb%26branch%3Dmaster

The notebook uses the following cluster settings, which is the biggest cluster we can reasonably share publicly on Pangeo Binder:

nworkers = 30
worker_memory = 8
worker_cores = 1

It includes both the "canonical anomaly-mean example example" with synthetic data from #2602 (comment) (referenced by @gjoseph92 in #2602 (comment)) as well as a similar real-world example that uses cloud data.

I have found that the critical performance issues are largely resolved even in Dask 2020.12.1 when I use options.environment = {"MALLOC_TRIM_THRESHOLD_": "0"}. For the synthetic data example, the older Dask version (pre #4967) is just a few s slower. For the real-world data example, the new version is significantly faster (2 min vs. 3 min), but nowhere close to the 6x increases reported above. But if I don't set MALLOC_TRIM_THRESHOLD_, both clusters crash. This leads me to conclude that, for these workloads, MALLOC_TRIM_THRESHOLD_ is much more important than #4967.

My questions for the group are:

  1. Is this interpretation correct? Or am I missing something?
  2. If it is correct that MALLOC_TRIM_THRESHOLD_ makes the biggest differences for these climatology-anomaly-style workloads, is there an alternative workflow that would better highlight the improvements of #4967 for the blog post?
  3. Is there a downside to setting MALLOC_TRIM_THRESHOLD_=0 on all of our clusters?

(edit below)

@dougiesquire's climactic mean example from #2602 (comment)

20-worker cluster with 2-CPU, 20GiB workers

I am now trying to run this example, which should presumably reproduce the issue better.

Loading

@rabernat
Copy link
Author

@rabernat rabernat commented Aug 2, 2021

Latest edit:

I ran a slightly modified version of the "dougiesquire's climactic mean example" and added it to the gist. The only real change I made was to also reduce over the ensemble dimension in order to reduce the total size of the final result--otherwise you end up with a 20GB array that can't fit into the notebook memory.

climactic mean example code
import pandas as pd

size = (28, 237, 48, 21, 90, 144)
chunks = (1, 1, 48, 21, 90, 144)
arr = dsa.random.random(size, chunks=chunks)
arr

items = dict(
    ensemble = np.arange(size[0]),
    init_date = pd.date_range(start='1960', periods=size[1]),
    lat = np.arange(size[2]).astype(float),
    lead_time = np.arange(size[3]),
    level = np.arange(size[4]).astype(float),
    lon = np.arange(size[5]).astype(float),
)
dims, coords = zip(*list(items.items()))

array = xr.DataArray(arr, coords=coords, dims=dims)
dset = xr.Dataset({'data': array})
display(dset)

# reduce of "init_date" and "ensemble" to reduce final memory requirements.
result = dset['data'].groupby("init_date.month").mean(dim=["init_date", "ensemble"])
%time result.compute()

Using MALLOC_TRIM_THRESHOLD_=0 and comparing Dask 2020.12.0 vs. 2021.07.1, I found 6min 49s vs. 4min 13s. This is definitely an improvement, but very different from @gjoseph92's results ("main + MALLOC_TRIM_THRESHOLD_=0: gave up after 30min and 1.25TiB spilled to disk (!!)")

So I can definitely see evidence of an incremental improvement, but I feel like I'm still missing something.

Loading

@RichardScottOZ
Copy link

@RichardScottOZ RichardScottOZ commented Aug 2, 2021

Thanks Ryan. Have not tested this on anything seriously large, but hopefully will soon.

Loading

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment