Add initial draft of large scale computation class#4972
Add initial draft of large scale computation class#4972
Conversation
distributed/scheduler.py
Outdated
| dependencies = dependencies or {} | ||
|
|
||
| if len(tasks) > 1 or Computation._recent is None: | ||
| # TODO: maybe reuse old computation based on time interval? |
There was a problem hiding this comment.
I would propose adding a ctx manager here instead of trying to be smart. something like
with computation(id="my-favourite-computation"):
while True:
client.submit(foo)Most users wouldn't need to use this since we'd assign every compute its own computation. The ctx manager would work similar to annotations, maybe the computation ID is an annotation? idk.
We can iterate on this, of course. I would like to avoid any fragile timing based heuristics if possible
There was a problem hiding this comment.
Not applying heuristics here would also allow us to not store state. I believe there wouldn't be a reason for the _recent if we made it explicit
There was a problem hiding this comment.
My current thinking is that we continue to use the previous computation until it finishes. Once it finishes then we close it out and start a new one.
So the following would probably result in a single computation
x = x.persist()
y = y.persist()
z = (x + y).sum().compute()There was a problem hiding this comment.
We now use the previous computation if Scheduler.total_occupancy > epsilon, so there is no longer a magic time interval.
I do think that a magic time interval might at some point be welcome, but I'm happy to wait to feel the need for that.
|
the one thing I would add to get started is a unique ID for every computation. probably a UUID or a token (hash) with timestamp. |
We currently store a timestamp. The scheduler has a UUID. Maybe that's enough? I thought about adding a UUID but I didn't need it for anything, and so I stopped. I'm inclined to find a use case before we add this. |
Should be enough. I wasn't aware. |
This avoids the use of a magic time interval
| def __repr__(self): | ||
| return ( | ||
| f"Computation: {format_time(time() - self._start)} ago with {len(self.groups)} groups\n\n" | ||
| + "\n---------------------------\n".join(self.code) |
There was a problem hiding this comment.
I don't think the plain repr should be that long. The code could be thousands of lines long and the repr is something we might see in a console, an IDE, a log message, CI, etc.
There was a problem hiding this comment.
Yeah, I don't disagree. This was useful during debugging / writing this PR. I'm happy to strip out the code. We could also consider only collecting +-50 lines of code around the relevant line.
| self._tasks = tasks | ||
| else: | ||
| self._tasks = dict() | ||
| self._computations = deque(maxlen=100) |
There was a problem hiding this comment.
I believe, this should be configurable from the start. We've seen in the past that too long deques can cause problems for long running clusters and reducing the deque length is often the only way to deal with this problem.
We do not know where people are issuing computations from or how large those modules are. With large modules, these Computation objects might easily reach MB size (for reference, our scheduler.py has ~260KiB) since we might be storing multiple code snapshots per Computation
There was a problem hiding this comment.
No disagreement from me
|
Also TODO: the |
|
@fjetter any interest in taking this over? |
How important is this feature to us? I remember us having this argument before and you mentioned it was useful. I have a different perspective here but this is obviously very biased and depends on the way one is using dask In the past years, most of the code I was using with dask for regularly was in libraries. The compute would be the most outer shell of a service and would not reveal any useful logic. from my_library.dask_stuff import construct_dask_graph
def endpoint(foo, bar, cluster_addr):
# Instead of a webservice endpoint, this could be a jupyter notebook or smth else
params = parse_params(foo, bar)
with Client(cluster_addr):
graph = construct_dask_graph(**params)
graph.compute()For these situations the frame inspection doesn't work at all or rather it is not very informative since we can only capture the immediate context of the compute. I would expect the pattern to put dask code into libraries not than uncommon.
sure |
Maybe we can solve this problem by making this list of known libraries configurable. We could also include a brief stack. To demonstrate value I'll point to the first page of performance reports, which includes code similarly. As someone debugging someone else's code I find that looking at the context of the compute call is pretty helpful. Often I look at the task stream and say "hrm, that's odd, why so much white space" then I look at their code and I say "Oh! Your chunks are way too small". Getting insight into user code has been, for me, quite helpful. |
|
Closed by #5001 |
Fixes #4613
This is very much an early work in progress / proof of concept. Mostly this is here to generate discussion
cc @fjetter