For some workflows we don't know the extent of the computation at the outset. We need to do some computation in order to figure out the rest of the computation that we need to do. The computation grows and evolves as we do more work.
As an example, consider a situation where you need to read many files and then based on the contents of those files, fire off additional work. You would like to read the files in parallel, and then within each file expose more parallelism.
This example goes through three ways to handle this situation using Dask Futures
as_completed
async/await
But first, lets run our code sequentially.
filenames = ["file.{}.txt".format(i) for i in range(10)]
filenames[:3]
import random, time
def parse_file(fn: str) -> list:
""" Returns a list work items of unknown length """
time.sleep(random.random())
return [random.random() for _ in range(random.randint(1, 10))]
def process_item(x: float):
""" Process each work item """
time.sleep(random.random() / 4)
return x + 1
%%time
# This takes around 10-20s
results = []
for fn in filenames:
L = parse_file(fn)
for x in L:
out = process_item(x)
results.append(out)
We'll need a Dask client in order to manage dynamic workloads
from dask.distributed import Client
client = Client(processes=False, n_workers=1, threads_per_worker=6)
client
The as_completed iterator lets us handle futures as they complete. We can then submit more data on the fly.
%%time
from dask.distributed import as_completed
import operator
lists = client.map(parse_file, filenames, pure=False)
lengths = client.map(len, lists)
mapping = dict(zip(lengths, lists))
futures = []
for future in as_completed(lengths):
n = future.result()
L = mapping[future]
for i in range(n):
new = client.submit(operator.getitem, L, i, priority=1)
new = client.submit(process_item, new, priority=1)
futures.append(new)
client.gather(futures)
We can also handle the concurrency here within our local process. This requires you to understand async/await syntax, but is generally powerful and arguably simpler than the as_completed
approach above.
import asyncio
async def f(fn):
""" Handle the lifecycle of a single file """
future = client.submit(parse_file, fn, pure=False)
length_future = client.submit(len, future)
length = await length_future
futures = [client.submit(operator.getitem, future, i, priority=10)
for i in range(length)]
futures = client.map(process_item, futures, priority=10)
return futures
async def run_all(filenames):
list_of_list_of_futures = await asyncio.gather(*[f(fn) for fn in filenames])
futures = sum(list_of_list_of_futures, [])
return await client.gather(futures)
We now need to run this function in the same event loop as our client is running. If we had started our client asynchronously, then we could have done this:
client = await Client(asynchronous=True)
await run_all(filenames)
However, because we started our client without the asynchronous=True
flag the event loop is actually running in a separate thread, so we'll have to ask the client to run this for us.
client.sync(run_all, filenames)
We can also submit tasks that themselves submit more tasks. See documentation here.
%%time
from dask.distributed import get_client, secede, rejoin
def f(fn):
L = parse_file(fn)
client = get_client()
futures = client.map(process_item, L, priority=10)
secede()
results = client.gather(futures)
rejoin()
return results
futures = client.map(f, filenames, pure=False)
results = client.gather(futures)