Test that numpy literals in graph don't get inlined#2080
Test that numpy literals in graph don't get inlined#2080mrocklin merged 3 commits intodask:masterfrom
Conversation
|
Include the tasks you don't want fused in the |
|
Can do.
General question about fuse though, do we want to fuse keys that point to
literals rather than tasks?
…On Tue, Mar 14, 2017 at 1:01 PM, Erik Welch ***@***.***> wrote:
Include the tasks you don't want fused in the keys= keyword argument to
fuse. All or most optimization functions accept keys=.
—
You are receiving this because you authored the thread.
Reply to this email directly, view it on GitHub
<#2080 (comment)>, or mute
the thread
<https://github.com/notifications/unsubscribe-auth/AASszH80ZZU-n24jghy42-M8RhxkyEfhks5rlsf2gaJpZM4Mcyt1>
.
|
|
OK, expanding on this a bit I actually also don't want to fuse the last task of a getitem chain. When thinking about distributed computing we want to quickly split up large numpy arrays with getitem calls, and then move them to other workers as necessary before fusing with any other computation. We want to avoid moving large intermediates. So in regards to fusion this is a bit odd. We want to fuse long getitem chains (this is very important for single-machine xarray workloads). But we don't want to fuse a getitem chain to anything else that comes afterwards (like a +100 operation), this helps keep data movement easier in a distributed system. My current approach uses the suggestion from @eriknw to specify a set of keys to specifically avoid fusing. We start at the bottom of the graph from all data nodes (anything that isn't a task) and move up while there are getitem chains. We claim the topmost element of that chain as a key that we don't want to lose/fuse away. This costs two linear scans, one to find data and then one call to |
fe9c701 to
f1537d8
Compare
f1537d8 to
915c6bd
Compare
|
I would like to merge this soon. @shoyer I've run this against the xarray test suite and didn't run into any problems. Still you might want to look at this if you get a chance. This stops Dask from fusing getitem calls into other, non-getitem calls. |
|
OK, I'm going ahead with this. |
In dask#2080 a bug was introduced that prevented fusing slices across aliases in the graph. These would show up when several dask arrays were concatenated together. This fixes that bug, and adds a test that fusing works across aliases.
* Fuse slices works with alias in graph In #2080 a bug was introduced that prevented fusing slices across aliases in the graph. These would show up when several dask arrays were concatenated together. This fixes that bug, and adds a test that fusing works across aliases. * Fuse slices works with locks
OK, so it's important that numpy arrays stay where they are in the graph and don't get fused. This is important for two reasons:
cc @eriknw any thoughts on how to address this cheaply?