Dask provides multi-core execution on larger-than-memory datasets using blocked algorithms and task scheduling. It maps high-level NumPy and list operations on large datasets on to graphs of many operations on small in-memory datasets. It then executes these graphs in parallel on a single machine. Dask lets us use traditional NumPy and list programming while operating on inconveniently large data in a small amount of space.
dask
is a specification to describe task dependency graphs.dask.array
is a drop-in NumPy replacement (for a subset of NumPy) that encodes blocked algorithms indask
dependency graphs.dask.bag
encodes blocked algorithms on Python lists of arbitrary Python objects.dask.async
is a shared-memory asynchronous scheduler efficiently executedask
dependency graphs on multiple cores.
Dask does not currently have a distributed memory scheduler.
See full documentation at https://dask.pydata.org or read developer-focused blogposts about dask's development.
Dask is easily installable through your favorite Python package manager:
conda install dask or pip install dask[array] or pip install dask[bag] or pip install dask[complete]
Consider the following simple program:
def inc(i):
return i + 1
def add(a, b):
return a + b
x = 1
y = inc(x)
z = add(y, 10)
We encode this as a dictionary in the following way:
d = {'x': 1,
'y': (inc, 'x'),
'z': (add, 'y', 10)}
While less aesthetically pleasing this dictionary may now be analyzed, optimized, and computed on by other Python code, not just the Python interpreter.
The dask.array
module creates these graphs from NumPy-like operations
>>> import dask.array as da
>>> x = da.random.random((4, 4), blockshape=(2, 2))
>>> x.T[0, 3].dask
{('x', 0, 0): (np.random.random, (2, 2)),
('x', 0, 1): (np.random.random, (2, 2)),
('x', 1, 0): (np.random.random, (2, 2)),
('x', 1, 1): (np.random.random, (2, 2)),
('y', 0, 0): (np.transpose, ('x', 0, 0)),
('y', 0, 1): (np.transpose, ('x', 1, 0)),
('y', 1, 0): (np.transpose, ('x', 0, 1)),
('y', 1, 1): (np.transpose, ('x', 1, 1)),
('z',): (getitem, ('y', 0, 1), (0, 1))}
Finally, a scheduler executes these graphs to achieve the intended result. The
dask.async
module contains a shared memory scheduler that efficiently
leverages multiple cores.
dask.core
supports Python 2.6+ and Python 3.3+ with a common codebase. It
is pure Python and requires no dependencies beyond the standard library. It is
a light weight dependency.
dask.array
depends on numpy
.
dask.bag
depends on toolz
and dill
.
New BSD. See License File.
One might ask why we didn't use one of these other fine libraries:
- Luigi
- Joblib
- mrjob
- Any of the fine schedulers in numeric analysis (DAGue, ...)
- Any of the fine high-throughput schedulers (Condor, Pegasus, Swiftlang, ...)
The answer is because we wanted all of the following:
- Fine-ish grained parallelism (latencies around 1ms)
- In-memory communication of intermediate results
- Dependency structures more complex than
map
- Good support for numeric data
- First class Python support
- Trivial installation
Most task schedulers in the Python ecosystem target long-running batch jobs, often for processing large amounts of text and aren't appropriate for executing multi-core numerics.
There are many "Big NumPy Array" or general distributed array solutions all with fine characteristics. Some projects in the Python ecosystem include the following:
There is a rich history of distributed array computing. An incomplete sampling includes the following projects: