diff -pruN 2022.01.0+dfsg-1/continuous_integration/environment-3.7.yaml 2022.02.0+dfsg-1/continuous_integration/environment-3.7.yaml
--- 2022.01.0+dfsg-1/continuous_integration/environment-3.7.yaml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/continuous_integration/environment-3.7.yaml	2022-02-11 16:21:12.000000000 +0000
@@ -28,11 +28,9 @@ dependencies:
   - tiledb>=2.5.0
   - xarray
   - fsspec
-  # sqlalchemy 1.4.0 causes deprecation warnings to be raised from pandas
-  # along with other issues https://github.com/pandas-dev/pandas/issues/40467
-  - sqlalchemy<1.4.0
+  - sqlalchemy>=1.4.0
   - pyarrow=1.0
-  - coverage
+  - coverage<6.3 # https://github.com/nedbat/coveragepy/issues/1310
   - jsonschema
   # other -- IO
   - bcolz
@@ -58,7 +56,7 @@ dependencies:
   - requests
   - scikit-image
   - scikit-learn
-  - scipy
+  - scipy<1.8.0 # See https://github.com/dask/dask/issues/8682
   - toolz
   - python-snappy
   - sparse
diff -pruN 2022.01.0+dfsg-1/continuous_integration/environment-3.8.yaml 2022.02.0+dfsg-1/continuous_integration/environment-3.8.yaml
--- 2022.01.0+dfsg-1/continuous_integration/environment-3.8.yaml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/continuous_integration/environment-3.8.yaml	2022-02-11 16:21:12.000000000 +0000
@@ -28,11 +28,9 @@ dependencies:
   - tiledb>=2.5.0
   - xarray
   - fsspec
-  # sqlalchemy 1.4.0 causes deprecation warnings to be raised from pandas
-  # along with other issues https://github.com/pandas-dev/pandas/issues/40467
-  - sqlalchemy<1.4.0
+  - sqlalchemy>=1.4.0
   - pyarrow=4.0
-  - coverage
+  - coverage<6.3 # https://github.com/nedbat/coveragepy/issues/1310
   - jsonschema
   # other -- IO
   - bcolz
@@ -57,7 +55,7 @@ dependencies:
   - requests
   - scikit-image
   - scikit-learn
-  - scipy
+  - scipy<1.8.0 # See https://github.com/dask/dask/issues/8682
   - toolz
   - python-snappy
   - sparse
diff -pruN 2022.01.0+dfsg-1/continuous_integration/environment-3.9.yaml 2022.02.0+dfsg-1/continuous_integration/environment-3.9.yaml
--- 2022.01.0+dfsg-1/continuous_integration/environment-3.9.yaml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/continuous_integration/environment-3.9.yaml	2022-02-11 16:21:12.000000000 +0000
@@ -28,11 +28,9 @@ dependencies:
   - tiledb>=2.5.0
   - xarray
   - fsspec
-  # sqlalchemy 1.4.0 causes deprecation warnings to be raised from pandas
-  # along with other issues https://github.com/pandas-dev/pandas/issues/40467
-  - sqlalchemy<1.4.0
+  - sqlalchemy>=1.4.0
   - pyarrow
-  - coverage
+  - coverage<6.3 # https://github.com/nedbat/coveragepy/issues/1310
   - jsonschema
   # other -- IO
   # Not available for Python 3.9 on conda-forge
@@ -58,7 +56,7 @@ dependencies:
   - requests
   - scikit-image
   - scikit-learn
-  - scipy
+  - scipy<1.8.0 # See https://github.com/dask/dask/issues/8682
   - toolz
   - python-snappy
   - sparse
diff -pruN 2022.01.0+dfsg-1/continuous_integration/gpuci/axis.yaml 2022.02.0+dfsg-1/continuous_integration/gpuci/axis.yaml
--- 2022.01.0+dfsg-1/continuous_integration/gpuci/axis.yaml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/continuous_integration/gpuci/axis.yaml	2022-02-11 16:21:12.000000000 +0000
@@ -1,5 +1,5 @@
 PYTHON_VER:
-- "3.8"
+- "3.9"
 
 CUDA_VER:
 - "11.5"
@@ -8,6 +8,6 @@ LINUX_VER:
 - ubuntu18.04
 
 RAPIDS_VER:
-- "22.02"
+- "22.04"
 
 excludes:
diff -pruN 2022.01.0+dfsg-1/continuous_integration/gpuci/build.sh 2022.02.0+dfsg-1/continuous_integration/gpuci/build.sh
--- 2022.01.0+dfsg-1/continuous_integration/gpuci/build.sh	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/continuous_integration/gpuci/build.sh	2022-02-11 16:21:12.000000000 +0000
@@ -43,6 +43,9 @@ python -m pip install git+https://github
 gpuci_logger "Install dask"
 python setup.py install
 
+gpuci_logger "Pin scipy"
+conda install -c conda-forge "scipy<1.8.0" -y
+
 gpuci_logger "Check Python version"
 python --version
 
diff -pruN 2022.01.0+dfsg-1/continuous_integration/scripts/install.sh 2022.02.0+dfsg-1/continuous_integration/scripts/install.sh
--- 2022.01.0+dfsg-1/continuous_integration/scripts/install.sh	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/continuous_integration/scripts/install.sh	2022-02-11 16:21:12.000000000 +0000
@@ -10,13 +10,14 @@ if [[ ${UPSTREAM_DEV} ]]; then
     mamba install -y -c arrow-nightlies "pyarrow>5.0"
 
     # FIXME https://github.com/mamba-org/mamba/issues/412
-    # mamba uninstall --force numpy pandas
-    conda uninstall --force numpy pandas fastparquet
+    # mamba uninstall --force numpy pandas scipy fastparquet
+    conda uninstall --force numpy pandas scipy fastparquet
 
     python -m pip install --no-deps --pre \
         -i https://pypi.anaconda.org/scipy-wheels-nightly/simple \
         numpy \
-        pandas
+        pandas \
+        scipy
 
     python -m pip install \
         --upgrade \
diff -pruN 2022.01.0+dfsg-1/dask/array/chunk.py 2022.02.0+dfsg-1/dask/array/chunk.py
--- 2022.01.0+dfsg-1/dask/array/chunk.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/chunk.py	2022-02-11 16:21:12.000000000 +0000
@@ -8,12 +8,6 @@ import numpy as np
 from tlz import concat
 
 from ..core import flatten
-from . import numpy_compat as npcompat
-
-try:
-    from numpy import take_along_axis
-except ImportError:  # pragma: no cover
-    take_along_axis = npcompat.take_along_axis
 
 
 def keepdims_wrapper(a_callable):
@@ -234,7 +228,7 @@ def argtopk(a_plus_idx, k, axis, keepdim
     idx2 = np.argpartition(a, -k, axis=axis)
     k_slice = slice(-k, None) if k > 0 else slice(-k)
     idx2 = idx2[tuple(k_slice if i == axis else slice(None) for i in range(a.ndim))]
-    return take_along_axis(a, idx2, axis), take_along_axis(idx, idx2, axis)
+    return np.take_along_axis(a, idx2, axis), np.take_along_axis(idx, idx2, axis)
 
 
 def argtopk_aggregate(a_plus_idx, k, axis, keepdims):
@@ -244,11 +238,12 @@ def argtopk_aggregate(a_plus_idx, k, axi
     and return the index only.
     """
     assert keepdims is True
+    a_plus_idx = a_plus_idx if len(a_plus_idx) > 1 else a_plus_idx[0]
     a, idx = argtopk(a_plus_idx, k, axis, keepdims)
     axis = axis[0]
 
     idx2 = np.argsort(a, axis=axis)
-    idx = take_along_axis(idx, idx2, axis)
+    idx = np.take_along_axis(idx, idx2, axis)
     if k < 0:
         return idx
     return idx[
diff -pruN 2022.01.0+dfsg-1/dask/array/core.py 2022.02.0+dfsg-1/dask/array/core.py
--- 2022.01.0+dfsg-1/dask/array/core.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/core.py	2022-02-11 16:21:12.000000000 +0000
@@ -18,13 +18,14 @@ from collections.abc import (
     Iterator,
     Mapping,
     MutableMapping,
+    Sequence,
 )
 from functools import partial, reduce, wraps
 from itertools import product, zip_longest
 from numbers import Integral, Number
 from operator import add, mul
 from threading import Lock
-from typing import Any, Sequence
+from typing import Any
 
 import numpy as np
 from fsspec import get_mapper
@@ -40,17 +41,19 @@ from ..base import (
     persist,
     tokenize,
 )
+from ..blockwise import blockwise as core_blockwise
 from ..blockwise import broadcast_dimensions
 from ..context import globalmethod
 from ..core import quote
 from ..delayed import Delayed, delayed
-from ..highlevelgraph import HighLevelGraph
-from ..layers import reshapelist
+from ..highlevelgraph import HighLevelGraph, MaterializedLayer
+from ..layers import ArraySliceDep, reshapelist
 from ..sizeof import sizeof
 from ..utils import (
     IndexCallable,
     M,
     SerializableLock,
+    cached_cumsum,
     cached_property,
     concrete,
     derived_from,
@@ -76,7 +79,7 @@ from .chunk_types import is_valid_array_
 # Keep einsum_lookup and tensordot_lookup here for backwards compatibility
 from .dispatch import concatenate_lookup, einsum_lookup, tensordot_lookup  # noqa: F401
 from .numpy_compat import _numpy_120, _Recurser
-from .slicing import cached_cumsum, replace_ellipsis, setitem_array, slice_array
+from .slicing import replace_ellipsis, setitem_array, slice_array
 
 config.update_defaults({"array": {"chunk-size": "128MiB", "rechunk-threshold": 4}})
 
@@ -232,46 +235,88 @@ def slices_from_chunks(chunks):
     return list(product(*slices))
 
 
-def getem(
-    arr,
+def graph_from_arraylike(
+    arr,  # Any array-like which supports slicing
     chunks,
+    shape,
+    name,
     getitem=getter,
-    shape=None,
-    out_name=None,
     lock=False,
     asarray=True,
     dtype=None,
-):
-    """Dask getting various chunks from an array-like
+    inline_array=False,
+) -> HighLevelGraph:
+    """
+    HighLevelGraph for slicing chunks from an array-like according to a chunk pattern.
+
+    If ``inline_array`` is True, this make a Blockwise layer of slicing tasks where the
+    array-like is embedded into every task.,
 
-    >>> getem('X', chunks=(2, 3), shape=(4, 6))  # doctest: +SKIP
-    {('X', 0, 0): (getter, 'X', (slice(0, 2), slice(0, 3))),
-     ('X', 1, 0): (getter, 'X', (slice(2, 4), slice(0, 3))),
-     ('X', 1, 1): (getter, 'X', (slice(2, 4), slice(3, 6))),
-     ('X', 0, 1): (getter, 'X', (slice(0, 2), slice(3, 6)))}
-
-    >>> getem('X', chunks=((2, 2), (3, 3)))  # doctest: +SKIP
-    {('X', 0, 0): (getter, 'X', (slice(0, 2), slice(0, 3))),
-     ('X', 1, 0): (getter, 'X', (slice(2, 4), slice(0, 3))),
-     ('X', 1, 1): (getter, 'X', (slice(2, 4), slice(3, 6))),
-     ('X', 0, 1): (getter, 'X', (slice(0, 2), slice(3, 6)))}
+    If ``inline_array`` is False, this inserts the array-like as a standalone value in
+    a MaterializedLayer, then generates a Blockwise layer of slicing tasks that refer
+    to it.
+
+    >>> dict(graph_from_arraylike(arr, chunks=(2, 3), shape=(4, 6), name="X", inline_array=True))  # doctest: +SKIP
+    {(arr, 0, 0): (getter, arr, (slice(0, 2), slice(0, 3))),
+     (arr, 1, 0): (getter, arr, (slice(2, 4), slice(0, 3))),
+     (arr, 1, 1): (getter, arr, (slice(2, 4), slice(3, 6))),
+     (arr, 0, 1): (getter, arr, (slice(0, 2), slice(3, 6)))}
+
+    >>> dict(  # doctest: +SKIP
+            graph_from_arraylike(arr, chunks=((2, 2), (3, 3)), shape=(4,6), name="X", inline_array=False)
+        )
+    {"original-X": arr,
+     ('X', 0, 0): (getter, 'original-X', (slice(0, 2), slice(0, 3))),
+     ('X', 1, 0): (getter, 'original-X', (slice(2, 4), slice(0, 3))),
+     ('X', 1, 1): (getter, 'original-X', (slice(2, 4), slice(3, 6))),
+     ('X', 0, 1): (getter, 'original-X', (slice(0, 2), slice(3, 6)))}
     """
-    out_name = out_name or arr
     chunks = normalize_chunks(chunks, shape, dtype=dtype)
-    keys = product([out_name], *(range(len(bds)) for bds in chunks))
-    slices = slices_from_chunks(chunks)
+    out_ind = tuple(range(len(shape)))
 
     if (
         has_keyword(getitem, "asarray")
         and has_keyword(getitem, "lock")
         and (not asarray or lock)
     ):
-        values = [(getitem, arr, x, asarray, lock) for x in slices]
+        getter = partial(getitem, asarray=asarray, lock=lock)
     else:
         # Common case, drop extra parameters
-        values = [(getitem, arr, x) for x in slices]
+        getter = getitem
 
-    return dict(zip(keys, values))
+    if inline_array:
+        layer = core_blockwise(
+            getter,
+            name,
+            out_ind,
+            arr,
+            None,
+            ArraySliceDep(chunks),
+            out_ind,
+            numblocks={},
+        )
+        return HighLevelGraph.from_collections(name, layer)
+    else:
+        original_name = "original-" + name
+
+        layers = {}
+        layers[original_name] = MaterializedLayer({original_name: arr})
+        layers[name] = core_blockwise(
+            getter,
+            name,
+            out_ind,
+            original_name,
+            None,
+            ArraySliceDep(chunks),
+            out_ind,
+            numblocks={},
+        )
+
+        deps = {
+            original_name: set(),
+            name: {original_name},
+        }
+        return HighLevelGraph(layers, deps)
 
 
 def dotmany(A, B, leftfunc=None, rightfunc=None, **kwargs):
@@ -3233,13 +3278,10 @@ def from_array(
     )
 
     if name in (None, True):
-        token = tokenize(x, chunks)
-        original_name = "array-original-" + token
+        token = tokenize(x, chunks, lock, asarray, fancy, getitem, inline_array)
         name = name or "array-" + token
     elif name is False:
-        original_name = name = "array-" + str(uuid.uuid1())
-    else:
-        original_name = name
+        name = "array-" + str(uuid.uuid1())
 
     if lock is True:
         lock = SerializableLock()
@@ -3266,23 +3308,17 @@ def from_array(
             else:
                 getitem = getter_nofancy
 
-        if inline_array:
-            get_from = x
-        else:
-            get_from = original_name
-
-        dsk = getem(
-            get_from,
+        dsk = graph_from_arraylike(
+            x,
             chunks,
+            x.shape,
+            name,
             getitem=getitem,
-            shape=x.shape,
-            out_name=name,
             lock=lock,
             asarray=asarray,
             dtype=x.dtype,
+            inline_array=inline_array,
         )
-        if not inline_array:
-            dsk[original_name] = x
 
     # Workaround for TileDB, its indexing is 1-based,
     # and doesn't seems to support 0-length slicing
@@ -3361,6 +3397,7 @@ def to_zarr(
     component=None,
     storage_options=None,
     overwrite=False,
+    region=None,
     compute=True,
     return_stored=False,
     **kwargs,
@@ -3386,6 +3423,9 @@ def to_zarr(
     overwrite: bool
         If given array already exists, overwrite=False will cause an error,
         where overwrite=True will replace the existing data.
+    region: tuple of slices or None
+        The region of data that should be written if ``url`` is a zarr.Array.
+        Not to be used with other types of ``url``.
     compute: bool
         See :func:`~dask.array.store` for more details.
     return_stored: bool
@@ -3397,6 +3437,7 @@ def to_zarr(
     ------
     ValueError
         If ``arr`` has unknown chunk sizes, which is not supported by Zarr.
+        If ``region`` is specified and ``url`` is not a zarr.Array
 
     See Also
     --------
@@ -3421,8 +3462,27 @@ def to_zarr(
                 "Cannot store into in memory Zarr Array using "
                 "the Distributed Scheduler."
             )
-        arr = arr.rechunk(z.chunks)
-        return arr.store(z, lock=False, compute=compute, return_stored=return_stored)
+
+        if region is None:
+            arr = arr.rechunk(z.chunks)
+            regions = None
+        else:
+            from .slicing import new_blockdim, normalize_index
+
+            old_chunks = normalize_chunks(z.chunks, z.shape)
+            index = normalize_index(region, z.shape)
+            chunks = tuple(
+                tuple(new_blockdim(s, c, r))
+                for s, c, r in zip(z.shape, old_chunks, index)
+            )
+            arr = arr.rechunk(chunks)
+            regions = [region]
+        return arr.store(
+            z, lock=False, regions=regions, compute=compute, return_stored=return_stored
+        )
+
+    if region is not None:
+        raise ValueError("Cannot use `region` keyword when url is not a `zarr.Array`.")
 
     if not _check_regular_chunks(arr.chunks):
         raise ValueError(
@@ -3918,7 +3978,8 @@ def concatenate(seq, axis=0, allow_unkno
     ----------
     seq: list of dask.arrays
     axis: int
-        Dimension along which to align all of the arrays
+        Dimension along which to align all of the arrays. If axis is None,
+        arrays are flattened before use.
     allow_unknown_chunksizes: bool
         Allow unknown chunksizes, such as come from converting from dask
         dataframes.  Dask.array is unable to verify that chunks line up.  If
@@ -3956,6 +4017,10 @@ def concatenate(seq, axis=0, allow_unkno
     if not seq:
         raise ValueError("Need array(s) to concatenate")
 
+    if axis is None:
+        seq = [a.flatten() for a in seq]
+        axis = 0
+
     seq_metas = [meta_from_array(s) for s in seq]
     _concatenate = concatenate_lookup.dispatch(
         type(max(seq_metas, key=lambda x: getattr(x, "__array_priority__", 0)))
@@ -4278,7 +4343,7 @@ def asarray(
     return from_array(a, getitem=getter_inline, **kwargs)
 
 
-def asanyarray(a, dtype=None, order=None, *, like=None):
+def asanyarray(a, dtype=None, order=None, *, like=None, inline_array=False):
     """Convert the input to a dask array.
 
     Subclasses of ``np.ndarray`` will be passed through as chunks unchanged.
@@ -4305,6 +4370,9 @@ def asanyarray(a, dtype=None, order=None
         argument. If ``like`` is a Dask array, the chunk type of the
         resulting array will be defined by the chunk type of ``like``.
         Requires NumPy 1.20.0 or higher.
+    inline_array:
+        Whether to inline the array in the resulting dask graph. For more information,
+        see the documentation for ``dask.array.from_array()``.
 
     Returns
     -------
@@ -4343,7 +4411,13 @@ def asanyarray(a, dtype=None, order=None
             return a.map_blocks(np.asanyarray, like=like_meta, dtype=dtype, order=order)
         else:
             a = np.asanyarray(a, like=like_meta, dtype=dtype, order=order)
-    return from_array(a, chunks=a.shape, getitem=getter_inline, asarray=False)
+    return from_array(
+        a,
+        chunks=a.shape,
+        getitem=getter_inline,
+        asarray=False,
+        inline_array=inline_array,
+    )
 
 
 def is_scalar_for_elemwise(arg):
diff -pruN 2022.01.0+dfsg-1/dask/array/creation.py 2022.02.0+dfsg-1/dask/array/creation.py
--- 2022.01.0+dfsg-1/dask/array/creation.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/creation.py	2022-02-11 16:21:12.000000000 +0000
@@ -10,7 +10,7 @@ from tlz import sliding_window
 
 from ..base import tokenize
 from ..highlevelgraph import HighLevelGraph
-from ..utils import derived_from
+from ..utils import cached_cumsum, derived_from
 from . import chunk
 from .core import (
     Array,
@@ -19,7 +19,6 @@ from .core import (
     blockwise,
     broadcast_arrays,
     broadcast_to,
-    cached_cumsum,
     concatenate,
     normalize_chunks,
     stack,
@@ -540,6 +539,8 @@ def eye(N, chunks="auto", M=None, k=0, d
     eye = {}
     if M is None:
         M = N
+    if dtype is None:
+        dtype = float
 
     if not isinstance(chunks, (int, str)):
         raise ValueError("chunks must be an int or string")
diff -pruN 2022.01.0+dfsg-1/dask/array/fft.py 2022.02.0+dfsg-1/dask/array/fft.py
--- 2022.01.0+dfsg-1/dask/array/fft.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/fft.py	2022-02-11 16:21:12.000000000 +0000
@@ -193,7 +193,7 @@ def fft_wrap(fft_func, kind=None, dtype=
     if kind.endswith("fft"):
         _func = func
 
-        def func(a, n=None, axis=None):
+        def func(a, n=None, axis=None):  # type: ignore
             s = None
             if n is not None:
                 s = (n,)
diff -pruN 2022.01.0+dfsg-1/dask/array/gufunc.py 2022.02.0+dfsg-1/dask/array/gufunc.py
--- 2022.01.0+dfsg-1/dask/array/gufunc.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/gufunc.py	2022-02-11 16:21:12.000000000 +0000
@@ -14,7 +14,7 @@ from .utils import meta_from_array
 # See https://docs.scipy.org/doc/numpy/reference/c-api/generalized-ufuncs.html
 _DIMENSION_NAME = r"\w+"
 _CORE_DIMENSION_LIST = "(?:{0:}(?:,{0:})*,?)?".format(_DIMENSION_NAME)
-_ARGUMENT = fr"\({_CORE_DIMENSION_LIST}\)"
+_ARGUMENT = rf"\({_CORE_DIMENSION_LIST}\)"
 _INPUT_ARGUMENTS = "(?:{0:}(?:,{0:})*,?)?".format(_ARGUMENT)
 _OUTPUT_ARGUMENTS = "{0:}(?:,{0:})*".format(
     _ARGUMENT
diff -pruN 2022.01.0+dfsg-1/dask/array/__init__.py 2022.02.0+dfsg-1/dask/array/__init__.py
--- 2022.01.0+dfsg-1/dask/array/__init__.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/__init__.py	2022-02-11 16:21:12.000000000 +0000
@@ -108,6 +108,7 @@ try:
         dstack,
         ediff1d,
         einsum,
+        expand_dims,
         extract,
         flatnonzero,
         flip,
diff -pruN 2022.01.0+dfsg-1/dask/array/linalg.py 2022.02.0+dfsg-1/dask/array/linalg.py
--- 2022.01.0+dfsg-1/dask/array/linalg.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/linalg.py	2022-02-11 16:21:12.000000000 +0000
@@ -1399,7 +1399,7 @@ def lstsq(a, b):
     q, r = qr(a)
     x = solve_triangular(r, q.T.conj().dot(b))
     residuals = b - a.dot(x)
-    residuals = abs(residuals ** 2).sum(axis=0, keepdims=b.ndim == 1)
+    residuals = abs(residuals**2).sum(axis=0, keepdims=b.ndim == 1)
 
     token = tokenize(a, b)
 
diff -pruN 2022.01.0+dfsg-1/dask/array/random.py 2022.02.0+dfsg-1/dask/array/random.py
--- 2022.01.0+dfsg-1/dask/array/random.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/random.py	2022-02-11 16:21:12.000000000 +0000
@@ -8,7 +8,7 @@ import numpy as np
 
 from ..base import tokenize
 from ..highlevelgraph import HighLevelGraph
-from ..utils import _deprecated, derived_from, random_state_data, skip_doctest
+from ..utils import derived_from, random_state_data
 from .core import (
     Array,
     asarray,
@@ -20,18 +20,6 @@ from .core import (
 from .creation import arange
 
 
-@_deprecated()
-def doc_wraps(func):
-    """Copy docstring from one function to another"""
-
-    def _(func2):
-        if func.__doc__ is not None:
-            func2.__doc__ = skip_doctest(func.__doc__)
-        return func2
-
-    return _
-
-
 class RandomState:
     """
     Mersenne Twister pseudo-random number generator
diff -pruN 2022.01.0+dfsg-1/dask/array/rechunk.py 2022.02.0+dfsg-1/dask/array/rechunk.py
--- 2022.01.0+dfsg-1/dask/array/rechunk.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/rechunk.py	2022-02-11 16:21:12.000000000 +0000
@@ -5,12 +5,13 @@ The rechunk module defines:
     rechunk: a function to convert the blocks
         of an existing dask array to new chunks or blockshape
 """
+from __future__ import annotations
+
 import heapq
 import math
 from functools import reduce
 from itertools import chain, count, product
 from operator import add, itemgetter, mul
-from typing import Tuple
 from warnings import warn
 
 import numpy as np
@@ -698,18 +699,18 @@ def _get_chunks(n, chunksize):
     return tuple(chunks)
 
 
-def _balance_chunksizes(chunks: Tuple[int, ...]) -> Tuple[int, ...]:
+def _balance_chunksizes(chunks: tuple[int, ...]) -> tuple[int, ...]:
     """
     Balance the chunk sizes
 
     Parameters
     ----------
-    chunks : Tuple[int, ...]
+    chunks : tuple[int, ...]
         Chunk sizes for Dask array.
 
     Returns
     -------
-    new_chunks : Tuple[int, ...]
+    new_chunks : tuple[int, ...]
         New chunks for Dask array with balanced sizes.
     """
     median_len = np.median(chunks).astype(int)
diff -pruN 2022.01.0+dfsg-1/dask/array/reductions.py 2022.02.0+dfsg-1/dask/array/reductions.py
--- 2022.01.0+dfsg-1/dask/array/reductions.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/reductions.py	2022-02-11 16:21:12.000000000 +0000
@@ -230,7 +230,7 @@ def _tree_reduce(
             depth = int(builtins.max(depth, ceil(log(n, split_every[i]))))
     func = partial(combine or aggregate, axis=axis, keepdims=True)
     if concatenate:
-        func = compose(func, partial(_concatenate2, axes=axis))
+        func = compose(func, partial(_concatenate2, axes=sorted(axis)))
     for i in range(depth - 1):
         x = partial_reduce(
             func,
@@ -243,7 +243,7 @@ def _tree_reduce(
         )
     func = partial(aggregate, axis=axis, keepdims=keepdims)
     if concatenate:
-        func = compose(func, partial(_concatenate2, axes=axis))
+        func = compose(func, partial(_concatenate2, axes=sorted(axis)))
     return partial_reduce(
         func,
         x,
@@ -293,9 +293,11 @@ def partial_reduce(
         out_chunks = list(getter(out_chunks))
     dsk = {}
     for k, p in zip(keys, product(*parts)):
-        decided = {i: j[0] for (i, j) in enumerate(p) if len(j) == 1}
-        dummy = dict(i for i in enumerate(p) if i[0] not in decided)
-        g = lol_tuples((x.name,), range(x.ndim), decided, dummy)
+        free = {
+            i: j[0] for (i, j) in enumerate(p) if len(j) == 1 and i not in split_every
+        }
+        dummy = dict(i for i in enumerate(p) if i[0] in split_every)
+        g = lol_tuples((x.name,), range(x.ndim), free, dummy)
         dsk[(name,) + k] = (func, g)
     graph = HighLevelGraph.from_collections(name, dsk, dependencies=[x])
 
@@ -719,11 +721,11 @@ def moment_chunk(
 
 def _moment_helper(Ms, ns, inner_term, order, sum, axis, kwargs):
     M = Ms[..., order - 2].sum(axis=axis, **kwargs) + sum(
-        ns * inner_term ** order, axis=axis, **kwargs
+        ns * inner_term**order, axis=axis, **kwargs
     )
     for k in range(1, order - 1):
         coeff = factorial(order) / (factorial(k) * factorial(order - k))
-        M += coeff * sum(Ms[..., order - k - 2] * inner_term ** k, axis=axis, **kwargs)
+        M += coeff * sum(Ms[..., order - k - 2] * inner_term**k, axis=axis, **kwargs)
     return M
 
 
@@ -1358,7 +1360,7 @@ def cumreduction(
           This method may be faster or more memory efficient depending on workload,
           scheduler, and hardware.  More benchmarking is necessary.
     preop: callable, optional
-        Function used by 'blelloch' method like `np.cumsum->np.sum`` or ``np.cumprod->np.prod``
+        Function used by 'blelloch' method like ``np.cumsum->np.sum`` or ``np.cumprod->np.prod``
 
     Returns
     -------
diff -pruN 2022.01.0+dfsg-1/dask/array/routines.py 2022.02.0+dfsg-1/dask/array/routines.py
--- 2022.01.0+dfsg-1/dask/array/routines.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/routines.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,9 +1,10 @@
+from __future__ import annotations
+
 import math
 import warnings
 from collections.abc import Iterable
 from functools import partial, reduce, wraps
 from numbers import Integral, Real
-from typing import List, Tuple
 
 import numpy as np
 from tlz import concat, interleave, sliding_window
@@ -1884,6 +1885,20 @@ def ravel(array_like):
 
 
 @derived_from(np)
+def expand_dims(a, axis):
+    if type(axis) not in (tuple, list):
+        axis = (axis,)
+
+    out_ndim = len(axis) + a.ndim
+    axis = validate_axis(axis, out_ndim)
+
+    shape_it = iter(a.shape)
+    shape = [1 if ax in axis else next(shape_it) for ax in range(out_ndim)]
+
+    return a.reshape(shape)
+
+
+@derived_from(np)
 def squeeze(a, axis=None):
     if axis is None:
         axis = tuple(i for i, d in enumerate(a.shape) if d == 1)
@@ -2205,21 +2220,18 @@ def select(condlist, choicelist, default
     )
 
 
-def _partition(total: int, divisor: int) -> Tuple[Tuple[int, ...], Tuple[int, ...]]:
-    """
-    Given a total and a divisor, return two tuples: A tuple containing `divisor` repeated
-    the number of times it divides `total`, and length-1 or empty tuple containing the remainder when
-    `total` is divided by `divisor`. If `divisor` factors `total`, i.e. if the remainder is 0, then
-    `remainder` is empty.
+def _partition(total: int, divisor: int) -> tuple[tuple[int, ...], tuple[int, ...]]:
+    """Given a total and a divisor, return two tuples: A tuple containing `divisor`
+    repeated the number of times it divides `total`, and length-1 or empty tuple
+    containing the remainder when `total` is divided by `divisor`. If `divisor` factors
+    `total`, i.e. if the remainder is 0, then `remainder` is empty.
     """
     multiples = (divisor,) * (total // divisor)
-    remainder = ()
-    if (total % divisor) > 0:
-        remainder = (total % divisor,)
-    return (multiples, remainder)
+    remainder = (total % divisor,) if total % divisor else ()
+    return multiples, remainder
 
 
-def aligned_coarsen_chunks(chunks: List[int], multiple: int) -> Tuple[int]:
+def aligned_coarsen_chunks(chunks: list[int], multiple: int) -> tuple[int, ...]:
     """
     Returns a new chunking aligned with the coarsening multiple.
     Any excess is at the end of the array.
diff -pruN 2022.01.0+dfsg-1/dask/array/slicing.py 2022.02.0+dfsg-1/dask/array/slicing.py
--- 2022.01.0+dfsg-1/dask/array/slicing.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/slicing.py	2022-02-11 16:21:12.000000000 +0000
@@ -4,15 +4,15 @@ import math
 import warnings
 from itertools import product
 from numbers import Integral, Number
-from operator import add, itemgetter
+from operator import itemgetter
 
 import numpy as np
-from tlz import accumulate, concat, memoize, merge, pluck
+from tlz import concat, memoize, merge, pluck
 
 from .. import config, core, utils
 from ..base import is_dask_collection, tokenize
 from ..highlevelgraph import HighLevelGraph
-from ..utils import is_arraylike
+from ..utils import cached_cumsum, is_arraylike
 from .chunk import getitem
 
 colon = slice(None, None, None)
@@ -641,7 +641,7 @@ def take(outname, inname, chunks, index,
     other_chunks = [chunks[i] for i in range(len(chunks)) if i != axis]
     other_numel = np.prod([sum(x) for x in other_chunks])
 
-    if math.isnan(other_numel):
+    if math.isnan(other_numel) or other_numel == 0:
         warnsize = maxsize = math.inf
     else:
         maxsize = math.ceil(nbytes / (other_numel * itemsize))
@@ -1288,65 +1288,6 @@ def shuffle_slice(x, index):
         return x[index2].rechunk(chunks2)[index3]
 
 
-class _HashIdWrapper:
-    """Hash and compare a wrapped object by identity instead of value"""
-
-    def __init__(self, wrapped):
-        self.wrapped = wrapped
-
-    def __eq__(self, other):
-        if not isinstance(other, _HashIdWrapper):
-            return NotImplemented
-        return self.wrapped is other.wrapped
-
-    def __ne__(self, other):
-        if not isinstance(other, _HashIdWrapper):
-            return NotImplemented
-        return self.wrapped is not other.wrapped
-
-    def __hash__(self):
-        return id(self.wrapped)
-
-
-@functools.lru_cache()
-def _cumsum(seq, initial_zero):
-    if isinstance(seq, _HashIdWrapper):
-        seq = seq.wrapped
-    if initial_zero:
-        return tuple(accumulate(add, seq, 0))
-    else:
-        return tuple(accumulate(add, seq))
-
-
-def cached_cumsum(seq, initial_zero=False):
-    """Compute :meth:`toolz.accumulate` with caching.
-
-    Caching is by the identify of `seq` rather than the value. It is thus
-    important that `seq` is a tuple of immutable objects, and this function
-    is intended for use where `seq` is a value that will persist (generally
-    block sizes).
-
-    Parameters
-    ----------
-    seq : tuple
-        Values to cumulatively sum.
-    initial_zero : bool, optional
-        If true, the return value is prefixed with a zero.
-
-    Returns
-    -------
-    tuple
-    """
-    if isinstance(seq, tuple):
-        # Look up by identity first, to avoid a linear-time __hash__
-        # if we've seen this tuple object before.
-        result = _cumsum(_HashIdWrapper(seq), initial_zero)
-    else:
-        # Construct a temporary tuple, and look up by value.
-        result = _cumsum(tuple(seq), initial_zero)
-    return result
-
-
 def parse_assignment_indices(indices, shape):
     """Reformat the indices for assignment.
 
@@ -1402,6 +1343,12 @@ def parse_assignment_indices(indices, sh
     >>> parse_assignment_indices((slice(-1, 2, -1), 3, [1, 2]), (7, 8, 9))
     ([slice(3, 7, 1), 3, array([1, 2])], [4, 2], [0], [0, 2])
 
+    >>> parse_assignment_indices((slice(0, 5), slice(3, None, 2)), (5, 4))
+    ([slice(0, 5, 1), slice(3, 4, 2)], [5, 1], [], [0, 1])
+
+    >>> parse_assignment_indices((slice(0, 5), slice(3, 3, 2)), (5, 4))
+    ([slice(0, 5, 1), slice(3, 3, 2)], [5, 0], [], [0])
+
     """
     if not isinstance(indices, tuple):
         indices = (indices,)
@@ -1469,8 +1416,13 @@ def parse_assignment_indices(indices, sh
                 reverse.append(i)
 
             start, stop, step = index.indices(size)
+
+            # Note: We now have stop >= start and step >= 0
+
             div, mod = divmod(stop - start, step)
-            if div <= 0:
+            if not div and not mod:
+                # stop equals start => zero-sized slice for this
+                # dimension
                 implied_shape.append(0)
             else:
                 if mod != 0:
@@ -1970,11 +1922,7 @@ def setitem_array(out_name, array, indic
         dim_1d_int_index = None
 
         for dim, (index, full_size, (loc0, loc1)) in enumerate(
-            zip(
-                indices,
-                array_shape,
-                locations,
-            )
+            zip(indices, array_shape, locations)
         ):
 
             integer_index = isinstance(index, int)
@@ -2078,9 +2026,7 @@ def setitem_array(out_name, array, indic
                 index = indices[j]
 
                 value_indices[i] = value_indices_from_1d_int_index(
-                    dim_1d_int_index,
-                    value_shape[i + value_offset],
-                    *loc0_loc1,
+                    dim_1d_int_index, value_shape[i + value_offset], *loc0_loc1
                 )
             else:
                 # Index is a slice or 1-d boolean array
diff -pruN 2022.01.0+dfsg-1/dask/array/stats.py 2022.02.0+dfsg-1/dask/array/stats.py
--- 2022.01.0+dfsg-1/dask/array/stats.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/stats.py	2022-02-11 16:21:12.000000000 +0000
@@ -194,7 +194,7 @@ def skew(a, axis=0, bias=True, nan_polic
     m2 = moment(a, 2, axis)
     m3 = moment(a, 3, axis)
     zero = m2 == 0
-    vals = da.where(~zero, m3 / m2 ** 1.5, 0.0)
+    vals = da.where(~zero, m3 / m2**1.5, 0.0)
     # vals = da.where(~zero, (m2, m3),
     #                 lambda m2, m3: m3 / m2**1.5,
     #                 0.)
@@ -226,7 +226,7 @@ def skewtest(a, axis=0, nan_policy="prop
     y = b2 * math.sqrt(((n + 1) * (n + 3)) / (6.0 * (n - 2)))
     beta2 = (
         3.0
-        * (n ** 2 + 27 * n - 70)
+        * (n**2 + 27 * n - 70)
         * (n + 1)
         * (n + 3)
         / ((n - 2.0) * (n + 5) * (n + 7) * (n + 9))
@@ -252,7 +252,7 @@ def kurtosis(a, axis=0, fisher=True, bia
     zero = m2 == 0
     olderr = np.seterr(all="ignore")
     try:
-        vals = da.where(zero, 0, m4 / m2 ** 2.0)
+        vals = da.where(zero, 0, m4 / m2**2.0)
     finally:
         np.seterr(**olderr)
 
@@ -293,7 +293,7 @@ def kurtosistest(a, axis=0, nan_policy="
         * np.sqrt((6.0 * (n + 3) * (n + 5)) / (n * (n - 2) * (n - 3)))
     )
     # [1]_ Eq. 3:
-    A = 6.0 + 8.0 / sqrtbeta1 * (2.0 / sqrtbeta1 + np.sqrt(1 + 4.0 / (sqrtbeta1 ** 2)))
+    A = 6.0 + 8.0 / sqrtbeta1 * (2.0 / sqrtbeta1 + np.sqrt(1 + 4.0 / (sqrtbeta1**2)))
     term1 = 1 - 2 / (9.0 * A)
     denom = 1 + x * np.sqrt(2 / (A - 4.0))
     denom = np.where(denom < 0, 99, denom)
@@ -384,7 +384,7 @@ def _unequal_var_ttest_denom(v1, n1, v2,
     vn1 = v1 / n1
     vn2 = v2 / n2
     with np.errstate(divide="ignore", invalid="ignore"):
-        df = (vn1 + vn2) ** 2 / (vn1 ** 2 / (n1 - 1) + vn2 ** 2 / (n2 - 1))
+        df = (vn1 + vn2) ** 2 / (vn1**2 / (n1 - 1) + vn2**2 / (n2 - 1))
 
     # If df is undefined, variances are zero (assumes n1 > 0 & n2 > 0).
     # Hence it doesn't matter what df is as long as it's not NaN.
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_array_core.py 2022.02.0+dfsg-1/dask/array/tests/test_array_core.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_array_core.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_array_core.py	2022-02-11 16:21:12.000000000 +0000
@@ -39,8 +39,8 @@ from dask.array.core import (
     from_array,
     from_delayed,
     from_func,
-    getem,
     getter,
+    graph_from_arraylike,
     normalize_chunks,
     optimize,
     stack,
@@ -52,21 +52,38 @@ from dask.blockwise import broadcast_dim
 from dask.blockwise import make_blockwise_graph as top
 from dask.blockwise import optimize_blockwise
 from dask.delayed import Delayed, delayed
-from dask.utils import apply, key_split, parse_bytes, tmpdir, tmpfile
-from dask.utils_test import dec, inc
+from dask.highlevelgraph import HighLevelGraph, MaterializedLayer
+from dask.layers import Blockwise
+from dask.utils import SerializableLock, apply, key_split, parse_bytes, tmpdir, tmpfile
+from dask.utils_test import dec, hlg_layer_topological, inc
 
 from ..chunk import getitem
 from .test_dispatch import EncapsulateNDArray
 
 
-def test_getem():
-    sol = {
-        ("X", 0, 0): (getter, "X", (slice(0, 2), slice(0, 3))),
-        ("X", 1, 0): (getter, "X", (slice(2, 4), slice(0, 3))),
-        ("X", 1, 1): (getter, "X", (slice(2, 4), slice(3, 6))),
-        ("X", 0, 1): (getter, "X", (slice(0, 2), slice(3, 6))),
-    }
-    assert getem("X", (2, 3), shape=(4, 6)) == sol
+@pytest.mark.parametrize("inline_array", [True, False])
+def test_graph_from_arraylike(inline_array):
+    d = 2
+    chunk = (2, 3)
+    shape = tuple(d * n for n in chunk)
+    arr = np.ones(shape)
+
+    dsk = graph_from_arraylike(
+        arr, chunk, shape=shape, name="X", inline_array=inline_array
+    )
+
+    assert isinstance(dsk, HighLevelGraph)
+    if inline_array:
+        assert len(dsk.layers) == 1
+        assert isinstance(hlg_layer_topological(dsk, 0), Blockwise)
+    else:
+        assert len(dsk.layers) == 2
+        assert isinstance(hlg_layer_topological(dsk, 0), MaterializedLayer)
+        assert isinstance(hlg_layer_topological(dsk, 1), Blockwise)
+    dsk = dict(dsk)
+
+    # Somewhat odd membership check to avoid numpy elemwise __in__ overload
+    assert any(arr is v for v in dsk.values()) is not inline_array
 
 
 def test_top():
@@ -200,16 +217,14 @@ def test_chunked_dot_product():
     x = np.arange(400).reshape((20, 20))
     o = np.ones((20, 20))
 
-    d = {"x": x, "o": o}
-
-    getx = getem("x", (5, 5), shape=(20, 20))
-    geto = getem("o", (5, 5), shape=(20, 20))
+    getx = graph_from_arraylike(x, (5, 5), shape=(20, 20), name="x")
+    geto = graph_from_arraylike(o, (5, 5), shape=(20, 20), name="o")
 
     result = top(
         dotmany, "out", "ik", "x", "ij", "o", "jk", numblocks={"x": (4, 4), "o": (4, 4)}
     )
 
-    dsk = merge(d, getx, geto, result)
+    dsk = merge(getx, geto, result)
     out = dask.get(dsk, [[("out", i, j) for j in range(4)] for i in range(4)])
 
     assert_eq(np.dot(x, o), concatenate3(out))
@@ -218,14 +233,12 @@ def test_chunked_dot_product():
 def test_chunked_transpose_plus_one():
     x = np.arange(400).reshape((20, 20))
 
-    d = {"x": x}
-
-    getx = getem("x", (5, 5), shape=(20, 20))
+    getx = graph_from_arraylike(x, (5, 5), shape=(20, 20), name="x")
 
     f = lambda x: x.T + 1
     comp = top(f, "out", "ij", "x", "ji", numblocks={"x": (4, 4)})
 
-    dsk = merge(d, getx, comp)
+    dsk = merge(getx, comp)
     out = dask.get(dsk, [[("out", i, j) for j in range(4)] for i in range(4)])
 
     assert_eq(concatenate3(out), x.T + 1)
@@ -244,10 +257,11 @@ def test_broadcast_dimensions():
 
 
 def test_Array():
+    arr = object()  # arraylike is unimportant since we never compute
     shape = (1000, 1000)
     chunks = (100, 100)
     name = "x"
-    dsk = merge({name: "some-array"}, getem(name, chunks, shape=shape))
+    dsk = graph_from_arraylike(arr, chunks, shape, name)
     a = Array(dsk, name, chunks, shape=shape, dtype="f8")
 
     assert a.numblocks == (10, 10)
@@ -272,10 +286,11 @@ def test_uneven_chunks():
 
 
 def test_numblocks_suppoorts_singleton_block_dims():
+    arr = object()  # arraylike is unimportant since we never compute
     shape = (100, 10)
     chunks = (10, 10)
     name = "x"
-    dsk = merge({name: "some-array"}, getem(name, shape=shape, chunks=chunks))
+    dsk = graph_from_arraylike(arr, chunks, shape, name)
     a = Array(dsk, name, chunks, shape=shape, dtype="f8")
 
     assert set(concat(a.__dask_keys__())) == {("x", i, 0) for i in range(10)}
@@ -324,7 +339,7 @@ def test_Array_numpy_gufunc_call__array_
 def test_stack():
     a, b, c = (
         Array(
-            getem(name, chunks=(2, 3), shape=(4, 6)),
+            graph_from_arraylike(object(), chunks=(2, 3), shape=(4, 6), name=name),
             name,
             chunks=(2, 3),
             dtype="f8",
@@ -477,7 +492,7 @@ def test_stack_unknown_chunksizes():
 def test_concatenate():
     a, b, c = (
         Array(
-            getem(name, chunks=(2, 3), shape=(4, 6)),
+            graph_from_arraylike(object(), chunks=(2, 3), shape=(4, 6), name=name),
             name,
             chunks=(2, 3),
             dtype="f8",
@@ -562,6 +577,16 @@ def test_concatenate_unknown_axes():
     assert_eq(c_x, np.concatenate([a_df.values, b_df.values], axis=1))
 
 
+def test_concatenate_flatten():
+    x = np.array([1, 2])
+    y = np.array([[3, 4], [5, 6]])
+
+    a = da.from_array(x, chunks=(2,))
+    b = da.from_array(y, chunks=(2, 1))
+
+    assert_eq(np.concatenate([x, y], axis=None), da.concatenate([a, b], axis=None))
+
+
 def test_concatenate_rechunk():
     x = da.random.random((6, 6), chunks=(3, 3))
     y = da.random.random((6, 6), chunks=(2, 2))
@@ -2279,7 +2304,7 @@ def test_arithmetic():
     assert_eq(b | b, y | y)
     assert_eq(b ^ b, y ^ y)
     assert_eq(a // b, x // y)
-    assert_eq(a ** b, x ** y)
+    assert_eq(a**b, x**y)
     assert_eq(a % b, x % y)
     assert_eq(a > b, x > y)
     assert_eq(a < b, x < y)
@@ -2296,7 +2321,7 @@ def test_arithmetic():
     assert_eq(b | True, y | True)
     assert_eq(b ^ True, y ^ True)
     assert_eq(a // 2, x // 2)
-    assert_eq(a ** 2, x ** 2)
+    assert_eq(a**2, x**2)
     assert_eq(a % 2, x % 2)
     assert_eq(a > 2, x > 2)
     assert_eq(a < 2, x < 2)
@@ -2313,7 +2338,7 @@ def test_arithmetic():
     assert_eq(True | b, True | y)
     assert_eq(True ^ b, True ^ y)
     assert_eq(2 // b, 2 // y)
-    assert_eq(2 ** b, 2 ** y)
+    assert_eq(2**b, 2**y)
     assert_eq(2 % b, 2 % y)
     assert_eq(2 > b, 2 > y)
     assert_eq(2 < b, 2 < y)
@@ -2495,22 +2520,36 @@ def test_Array_normalizes_dtype():
     assert isinstance(x.dtype, np.dtype)
 
 
-def test_from_array_with_lock():
+@pytest.mark.parametrize("inline_array", [True, False])
+def test_from_array_with_lock(inline_array):
     x = np.arange(10)
-    d = da.from_array(x, chunks=5, lock=True)
-
-    tasks = [v for k, v in d.dask.items() if k[0] == d.name]
 
-    assert hasattr(tasks[0][4], "acquire")
-    assert len({task[4] for task in tasks}) == 1
+    class FussyLock(SerializableLock):
+        def acquire(self, blocking=True, timeout=-1):
+            if self.locked():
+                raise RuntimeError("I am locked")
+            return super().acquire(blocking, timeout)
+
+    lock = FussyLock()
+    d = da.from_array(x, chunks=5, lock=lock, inline_array=inline_array)
+
+    lock.acquire()
+    with pytest.raises(RuntimeError):
+        d.compute()
 
+    lock.release()
     assert_eq(d, x)
 
-    lock = Lock()
-    e = da.from_array(x, chunks=5, lock=lock)
-    f = da.from_array(x, chunks=5, lock=lock)
+    lock = CounterLock()
+    e = da.from_array(x, chunks=5, lock=lock, inline_array=inline_array)
 
-    assert_eq(e + f, x + x)
+    assert_eq(e, x)
+    # Note: the specific counts for composite arithmetic operations can vary
+    # significantly based on the complexity of the computation, whether we are inlining,
+    # and optimization fusion settings. But for this simple comparison it seems pretty
+    # stable.
+    assert lock.release_count == 2
+    assert lock.acquire_count == 2
 
 
 class MyArray:
@@ -2533,8 +2572,11 @@ class MyArray:
         (np.array(1), 1),
     ],
 )
-def test_from_array_tasks_always_call_getter(x, chunks):
-    dx = da.from_array(MyArray(x), chunks=chunks, asarray=False)
+@pytest.mark.parametrize("inline_array", [True, False])
+def test_from_array_tasks_always_call_getter(x, chunks, inline_array):
+    dx = da.from_array(
+        MyArray(x), chunks=chunks, asarray=False, inline_array=inline_array
+    )
     assert_eq(x, dx)
 
 
@@ -2592,15 +2634,18 @@ def test_from_array_scalar(type_):
 
 
 @pytest.mark.parametrize("asarray,cls", [(True, np.ndarray), (False, np.matrix)])
+@pytest.mark.parametrize("inline_array", [True, False])
 @pytest.mark.filterwarnings("ignore:the matrix subclass")
-def test_from_array_no_asarray(asarray, cls):
+def test_from_array_no_asarray(asarray, cls, inline_array):
     def assert_chunks_are_of_type(x):
         chunks = compute_as_if_collection(Array, x.dask, x.__dask_keys__())
-        for c in concat(chunks):
+        # If it's a tuple of tuples we want to concat, but if it's a tuple
+        # of 1d arrays, we just want to iterate directly
+        for c in concat(chunks) if isinstance(chunks[0], tuple) else chunks:
             assert type(c) is cls
 
     x = np.matrix(np.arange(100).reshape((10, 10)))
-    dx = da.from_array(x, chunks=(5, 5), asarray=asarray)
+    dx = da.from_array(x, chunks=(5, 5), asarray=asarray, inline_array=inline_array)
     assert_chunks_are_of_type(dx)
     assert_chunks_are_of_type(dx[0:5])
     assert_chunks_are_of_type(dx[0:5][:, 0])
@@ -2664,12 +2709,11 @@ def test_from_array_inline():
         pass
 
     a = np.array([1, 2, 3]).view(MyArray)
-    dsk = dict(da.from_array(a, name="my-array").dask)
-    assert dsk["my-array"] is a
+    dsk = dict(da.from_array(a, name="my-array", inline_array=False).dask)
+    assert dsk["original-my-array"] is a
 
     dsk = dict(da.from_array(a, name="my-array", inline_array=True).dask)
-    assert "my-array" not in dsk
-    assert a is dsk[("my-array", 0)][1]
+    assert "original-my-array" not in dsk
 
 
 @pytest.mark.parametrize("asarray", [da.asarray, da.asanyarray])
@@ -2701,15 +2745,19 @@ def test_asarray_dask_dataframe(asarray)
 
 
 @pytest.mark.parametrize("asarray", [da.asarray, da.asanyarray])
-def test_asarray_h5py(asarray):
+@pytest.mark.parametrize("inline_array", [True, False])
+def test_asarray_h5py(asarray, inline_array):
     h5py = pytest.importorskip("h5py")
 
     with tmpfile(".hdf5") as fn:
         with h5py.File(fn, mode="a") as f:
             d = f.create_dataset("/x", shape=(2, 2), dtype=float)
-            x = asarray(d)
-            assert d in x.dask.values()
-            assert not any(isinstance(v, np.ndarray) for v in x.dask.values())
+            x = asarray(d, inline_array=inline_array)
+
+            # Check for the array in the dsk
+            dsk = dict(x.dask)
+            assert (d in dsk.values()) is not inline_array
+            assert not any(isinstance(v, np.ndarray) for v in dsk.values())
 
 
 def test_asarray_chunks():
@@ -3542,11 +3590,7 @@ def test_from_array_names():
 
 
 @pytest.mark.parametrize(
-    "array",
-    [
-        da.arange(100, chunks=25),
-        da.ones((10, 10), chunks=25),
-    ],
+    "array", [da.arange(100, chunks=25), da.ones((10, 10), chunks=25)]
 )
 def test_array_picklable(array):
     from pickle import dumps, loads
@@ -3804,11 +3848,11 @@ def test_index_array_with_array_2d():
 
 @pytest.mark.xfail(reason="Chunking does not align well")
 def test_index_array_with_array_3d_2d():
-    x = np.arange(4 ** 3).reshape((4, 4, 4))
+    x = np.arange(4**3).reshape((4, 4, 4))
     dx = da.from_array(x, chunks=(2, 2, 2))
 
     ind = np.random.random((4, 4)) > 0.5
-    ind = np.arange(4 ** 2).reshape((4, 4)) % 2 == 0
+    ind = np.arange(4**2).reshape((4, 4)) % 2 == 0
     dind = da.from_array(ind, (2, 2))
 
     assert_eq(x[ind], dx[dind])
@@ -3855,17 +3899,29 @@ def test_setitem_extended_API_0d():
     assert_eq(x, dx.compute())
 
 
-def test_setitem_extended_API_1d():
+@pytest.mark.parametrize(
+    "index, value",
+    [
+        [Ellipsis, -1],
+        [slice(2, 8, 2), -2],
+        [slice(8, None, 2), -3],
+        [slice(8, None, 2), [-30]],
+        [slice(1, None, -2), -4],
+        [slice(1, None, -2), [-40]],
+        [slice(3, None, 2), -5],
+        [slice(-3, None, -2), -6],
+        [slice(1, None, -2), -4],
+        [slice(3, None, 2), -5],
+        [slice(3, None, 2), [10, 11, 12, 13]],
+        [slice(-4, None, -2), [14, 15, 16, 17]],
+    ],
+)
+def test_setitem_extended_API_1d(index, value):
     # 1-d array
     x = np.arange(10)
-    dx = da.from_array(x.copy(), chunks=(4, 6))
-
-    x[2:8:2] = -1
-    dx[2:8:2] = -1
-    assert_eq(x, dx.compute())
-
-    x[...] = -11
-    dx[...] = -11
+    dx = da.from_array(x, chunks=(4, 6))
+    dx[index] = value
+    x[index] = value
     assert_eq(x, dx.compute())
 
 
@@ -3907,6 +3963,10 @@ def test_setitem_extended_API_1d():
             ),
             [[-100, -101, -102, -103], [-200, -201, -202, -203]],
         ],
+        [slice(5, None, 2), -99],
+        [slice(5, None, 2), range(1, 11)],
+        [slice(1, None, -2), -98],
+        [slice(1, None, -2), range(11, 21)],
     ],
 )
 def test_setitem_extended_API_2d(index, value):
@@ -4463,12 +4523,13 @@ def test_zarr_return_stored(compute):
         assert a2.chunks == a.chunks
 
 
-def test_zarr_inline_array():
+@pytest.mark.parametrize("inline_array", [True, False])
+def test_zarr_inline_array(inline_array):
     zarr = pytest.importorskip("zarr")
     a = zarr.array([1, 2, 3])
-    dsk = dict(da.from_zarr(a, inline_array=True).dask)
-    assert len(dsk) == 1
-    assert a in list(dsk.values())[0]
+    dsk = dict(da.from_zarr(a, inline_array=inline_array).dask)
+    assert len(dsk) == (0 if inline_array else 1) + 1
+    assert (a in dsk.values()) is not inline_array
 
 
 def test_zarr_existing_array():
@@ -4563,6 +4624,35 @@ def test_zarr_nocompute():
         assert a2.chunks == a.chunks
 
 
+def test_zarr_regions():
+    zarr = pytest.importorskip("zarr")
+
+    a = da.arange(16).reshape((4, 4)).rechunk(2)
+    z = zarr.zeros_like(a, chunks=2)
+
+    a[:2, :2].to_zarr(z, region=(slice(2), slice(2)))
+    a2 = da.from_zarr(z)
+    expected = [[0, 1, 0, 0], [4, 5, 0, 0], [0, 0, 0, 0], [0, 0, 0, 0]]
+    assert_eq(a2, expected)
+    assert a2.chunks == a.chunks
+
+    a[:3, 3:4].to_zarr(z, region=(slice(1, 4), slice(2, 3)))
+    a2 = da.from_zarr(z)
+    expected = [[0, 1, 0, 0], [4, 5, 3, 0], [0, 0, 7, 0], [0, 0, 11, 0]]
+    assert_eq(a2, expected)
+    assert a2.chunks == a.chunks
+
+    a[3:, 3:].to_zarr(z, region=(slice(2, 3), slice(1, 2)))
+    a2 = da.from_zarr(z)
+    expected = [[0, 1, 0, 0], [4, 5, 3, 0], [0, 15, 7, 0], [0, 0, 11, 0]]
+    assert_eq(a2, expected)
+    assert a2.chunks == a.chunks
+
+    with pytest.raises(ValueError):
+        with tmpdir() as d:
+            a.to_zarr(d, region=(slice(2), slice(2)))
+
+
 def test_tiledb_roundtrip():
     tiledb = pytest.importorskip("tiledb")
     # 1) load with default chunking
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_array_utils.py 2022.02.0+dfsg-1/dask/array/tests/test_array_utils.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_array_utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_array_utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -2,7 +2,9 @@ import numpy as np
 import pytest
 
 import dask.array as da
+from dask.array.core import Array
 from dask.array.utils import assert_eq, meta_from_array
+from dask.local import get_sync
 
 asarrays = [np.asarray]
 
@@ -87,3 +89,28 @@ def test_meta_from_array_type_inputs():
 def test_assert_eq_checks_dtype(a, b):
     with pytest.raises(AssertionError, match="a and b have different dtypes"):
         assert_eq(a, b)
+
+
+@pytest.mark.parametrize(
+    "a,b",
+    [
+        (1.0, 1.0),
+        ([1, 2], [1, 2]),
+        (da.array([1, 2]), da.array([1, 2])),
+    ],
+)
+def test_assert_eq_scheduler(a, b):
+    counter = 0  # Counts how many times `custom_scheduler` is executed.
+
+    def custom_scheduler(*args, **kwargs):
+        nonlocal counter
+        counter += 1
+        return get_sync(*args, **kwargs)
+
+    assert_eq(a, b)
+    assert counter == 0
+
+    assert_eq(a, b, scheduler=custom_scheduler)
+    # `custom_scheduler` should be executed twice if `a` and `b` are Arrays.
+    n_da_arrays = len([x for x in [a, b] if isinstance(x, Array)])
+    assert counter == n_da_arrays
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_atop.py 2022.02.0+dfsg-1/dask/array/tests/test_atop.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_atop.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_atop.py	2022-02-11 16:21:12.000000000 +0000
@@ -18,11 +18,11 @@ from dask.blockwise import (
 from dask.highlevelgraph import HighLevelGraph
 from dask.utils_test import dec, inc
 
-a, b, c, d, e, f, g = "abcdefg"
+a, b, c, d, e, f, g = "a", "b", "c", "d", "e", "f", "g"
 _0, _1, _2, _3, _4, _5, _6, _7, _8, _9 = (
-    _BLOCKWISE_DEFAULT_PREFIX + "%d" % i for i in range(10)
+    f"{_BLOCKWISE_DEFAULT_PREFIX}{i}" for i in range(10)
 )
-i, j, k = "ijk"
+i, j, k = "i", "j", "k"
 
 
 @pytest.mark.parametrize(
@@ -683,7 +683,7 @@ def test_args_delayed():
 
 
 @pytest.mark.parametrize(
-    "tup", [(1, 2), collections.namedtuple("foo", ["a", "b"])(1, 2)]
+    "tup", [(1, 2), collections.namedtuple("foo", ["a", "b"])(1, 2)]  # type: ignore
 )
 def test_namedtuple(tup):
     A = da.random.random((20, 20), chunks=(10, 10))
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_creation.py 2022.02.0+dfsg-1/dask/array/tests/test_creation.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_creation.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_creation.py	2022-02-11 16:21:12.000000000 +0000
@@ -401,6 +401,7 @@ def test_eye():
     assert_eq(da.eye(9, chunks=3, dtype=int), np.eye(9, dtype=int))
     assert_eq(da.eye(10, chunks=3, dtype=int), np.eye(10, dtype=int))
     assert_eq(da.eye(10, chunks=-1, dtype=int), np.eye(10, dtype=int))
+    assert_eq(da.eye(9, chunks=3, dtype=None), np.eye(9, dtype=None))
 
     with dask.config.set({"array.chunk-size": "50 MiB"}):
         x = da.eye(10000, "auto")
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_cupy_core.py 2022.02.0+dfsg-1/dask/array/tests/test_cupy_core.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_cupy_core.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_cupy_core.py	2022-02-11 16:21:12.000000000 +0000
@@ -19,7 +19,7 @@ functions = [
     lambda x: da.expm1(x),
     lambda x: 2 * x,
     lambda x: x / 2,
-    lambda x: x ** 2,
+    lambda x: x**2,
     lambda x: x + x,
     lambda x: x * x,
     lambda x: x[0],
@@ -40,9 +40,7 @@ functions = [
             "(with https://github.com/cupy/cupy/pull/2418)",
         ),
     ),
-    pytest.param(
-        lambda x: x.moment(order=0),
-    ),
+    pytest.param(lambda x: x.moment(order=0)),
     lambda x: x.moment(order=2),
     pytest.param(
         lambda x: x.std(),
@@ -77,9 +75,7 @@ functions = [
     lambda x: x.sum(axis=(1, 2)),
     lambda x: x.astype(np.complex128),
     lambda x: x.map_blocks(lambda x: x * 2),
-    pytest.param(
-        lambda x: x.round(1),
-    ),
+    pytest.param(lambda x: x.round(1)),
     lambda x: x.reshape((x.shape[0] * x.shape[1], x.shape[2])),
     # Rechunking here is required, see https://github.com/dask/dask/issues/2561
     lambda x: (x.rechunk(x.shape)).reshape((x.shape[1], x.shape[0], x.shape[2])),
@@ -88,9 +84,7 @@ functions = [
     lambda x: x > 0.5,
     lambda x: x.rechunk((4, 4, 4)),
     lambda x: x.rechunk((2, 2, 1)),
-    pytest.param(
-        lambda x: da.einsum("ijk,ijk", x, x),
-    ),
+    pytest.param(lambda x: da.einsum("ijk,ijk", x, x)),
     lambda x: np.isneginf(x),
     lambda x: np.isposinf(x),
     lambda x: np.isreal(x),
@@ -308,17 +302,45 @@ def test_setitem_extended_API_0d():
     assert_eq(x, dx.compute())
 
 
-def test_setitem_extended_API_1d():
+@pytest.mark.parametrize(
+    "index, value",
+    [
+        [Ellipsis, -1],
+        [slice(2, 8, 2), -2],
+        [slice(8, None, 2), -3],
+        pytest.param(
+            slice(8, None, 2),
+            [-30],
+            marks=pytest.mark.skip(reason="Unsupported assigning `list` to CuPy array"),
+        ),
+        [slice(1, None, -2), -4],
+        pytest.param(
+            slice(1, None, -2),
+            [-40],
+            marks=pytest.mark.skip(reason="Unsupported assigning `list` to CuPy array"),
+        ),
+        [slice(3, None, 2), -5],
+        [slice(-3, None, -2), -6],
+        [slice(1, None, -2), -4],
+        [slice(3, None, 2), -5],
+        pytest.param(
+            slice(3, None, 2),
+            [10, 11, 12, 13],
+            marks=pytest.mark.skip(reason="Unsupported assigning `list` to CuPy array"),
+        ),
+        pytest.param(
+            slice(-4, None, -2),
+            [14, 15, 16, 17],
+            marks=pytest.mark.skip(reason="Unsupported assigning `list` to CuPy array"),
+        ),
+    ],
+)
+def test_setitem_extended_API_1d(index, value):
     # 1-d array
     x = cupy.arange(10)
-    dx = da.from_array(x.copy(), chunks=(4, 6))
-
-    x[2:8:2] = -1
-    dx[2:8:2] = -1
-    assert_eq(x, dx.compute())
-
-    x[...] = -11
-    dx[...] = -11
+    dx = da.from_array(x, chunks=(4, 6))
+    dx[index] = value
+    x[index] = value
     assert_eq(x, dx.compute())
 
 
@@ -335,14 +357,14 @@ def test_setitem_extended_API_1d():
             (slice(None), 2),
             range(6),
             marks=pytest.mark.skip(
-                reason="Assigning `range` to CuPy array is not supported",
+                reason="Assigning `range` to CuPy array is not supported"
             ),
         ),
         pytest.param(
             3,
             range(10),
             marks=pytest.mark.skip(
-                reason="Assigning `range` to CuPy array is not supported",
+                reason="Assigning `range` to CuPy array is not supported"
             ),
         ),
         [(slice(None), [3, 5, 6]), [-30, -31, -32]],
@@ -350,17 +372,13 @@ def test_setitem_extended_API_1d():
         pytest.param(
             (slice(None, 2), slice(None, 3)),
             [-50, -51, -52],
-            marks=pytest.mark.skip(
-                reason="Unsupported assigning `list` to CuPy array",
-            ),
+            marks=pytest.mark.skip(reason="Unsupported assigning `list` to CuPy array"),
         ),
         [(slice(None), [6, 1, 3]), [-60, -61, -62]],
         pytest.param(
             (slice(1, 3), slice(1, 4)),
             [[-70, -71, -72]],
-            marks=pytest.mark.skip(
-                reason="Unsupported assigning `list` to CuPy array",
-            ),
+            marks=pytest.mark.skip(reason="Unsupported assigning `list` to CuPy array"),
         ),
         pytest.param(
             (slice(None), [9, 8, 8]),
@@ -380,7 +398,23 @@ def test_setitem_extended_API_1d():
             ),
             -1,
             marks=pytest.mark.skip(
-                reason="Unsupported assigning Dask Array to CuPy array",
+                reason="Unsupported assigning Dask Array to CuPy array"
+            ),
+        ),
+        [slice(5, None, 2), -99],
+        pytest.param(
+            slice(5, None, 2),
+            range(1, 11),
+            marks=pytest.mark.skip(
+                reason="Assigning `range` to CuPy array is not supported"
+            ),
+        ),
+        [slice(1, None, -2), -98],
+        pytest.param(
+            slice(1, None, -2),
+            range(11, 21),
+            marks=pytest.mark.skip(
+                reason="Assigning `range` to CuPy array is not supported"
             ),
         ),
     ],
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_cupy_slicing.py 2022.02.0+dfsg-1/dask/array/tests/test_cupy_slicing.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_cupy_slicing.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_cupy_slicing.py	2022-02-11 16:21:12.000000000 +0000
@@ -34,7 +34,7 @@ def test_index_with_int_dask_array(x_chu
 @pytest.mark.skipif(not _numpy_120, reason="NEP-35 is not available")
 @pytest.mark.parametrize("idx_chunks", [None, 3, 2, 1])
 @pytest.mark.parametrize("x_chunks", [(3, 5), (2, 3), (1, 2), (1, 1)])
-def test_index_with_int_dask_array(x_chunks, idx_chunks):
+def test_index_with_int_dask_array_nep35(x_chunks, idx_chunks):
     # test data is crafted to stress use cases:
     # - pick from different chunks of x out of order
     # - a chunk of x contains no matches
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_linalg.py 2022.02.0+dfsg-1/dask/array/tests/test_linalg.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_linalg.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_linalg.py	2022-02-11 16:21:12.000000000 +0000
@@ -453,14 +453,14 @@ def test_dask_svd_self_consistent(m, n):
         assert d_e.dtype == e.dtype
 
 
-@pytest.mark.parametrize("iterator", [("power", 1), ("QR", 1)])
+@pytest.mark.parametrize("iterator", ["power", "QR"])
 def test_svd_compressed_compute(iterator):
     x = da.ones((100, 100), chunks=(10, 10))
     u, s, v = da.linalg.svd_compressed(
-        x, k=2, iterator=iterator[0], n_power_iter=iterator[1], compute=True, seed=123
+        x, k=2, iterator=iterator, n_power_iter=1, compute=True, seed=123
     )
     uu, ss, vv = da.linalg.svd_compressed(
-        x, k=2, iterator=iterator[0], n_power_iter=iterator[1], seed=123
+        x, k=2, iterator=iterator, n_power_iter=1, seed=123
     )
 
     assert len(v.dask) < len(vv.dask)
@@ -536,20 +536,6 @@ def test_svd_compressed_shapes(m, n, k,
     assert v.shape == (r, n)
 
 
-@pytest.mark.parametrize("iterator", [("power", 1), ("QR", 1)])
-def test_svd_compressed_compute(iterator):
-    x = da.ones((100, 100), chunks=(10, 10))
-    u, s, v = da.linalg.svd_compressed(
-        x, 2, iterator=iterator[0], n_power_iter=iterator[1], compute=True, seed=123
-    )
-    uu, ss, vv = da.linalg.svd_compressed(
-        x, 2, iterator=iterator[0], n_power_iter=iterator[1], seed=123
-    )
-
-    assert len(v.dask) < len(vv.dask)
-    assert_eq(v, vv)
-
-
 def _check_lu_result(p, l, u, A):
     assert np.allclose(p.dot(l).dot(u), A)
 
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_masked.py 2022.02.0+dfsg-1/dask/array/tests/test_masked.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_masked.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_masked.py	2022-02-11 16:21:12.000000000 +0000
@@ -51,7 +51,7 @@ functions = [
     lambda x: da.expm1(x),
     lambda x: 2 * x,
     lambda x: x / 2,
-    lambda x: x ** 2,
+    lambda x: x**2,
     lambda x: x + x,
     lambda x: x * x,
     lambda x: x[0],
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_optimization.py 2022.02.0+dfsg-1/dask/array/tests/test_optimization.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_optimization.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_optimization.py	2022-02-11 16:21:12.000000000 +0000
@@ -271,25 +271,6 @@ def test_dont_fuse_numpy_arrays():
         assert sum(isinstance(v, np.ndarray) for v in dsk.values()) == 1
 
 
-def test_minimize_data_transfer():
-    zarr = pytest.importorskip("zarr")
-    x = zarr.ones((100,))
-    y = da.from_array(x, chunks=25)
-    z = y + 1
-    dsk = z.__dask_optimize__(z.dask, z.__dask_keys__())
-
-    keys = list(dsk)
-    results = dask.get(dsk, keys)
-    big_key = [k for k, r in zip(keys, results) if r is x][0]
-    dependencies, dependents = dask.core.get_deps(dsk)
-    deps = dependents[big_key]
-
-    assert len(deps) == 4
-    for dep in deps:
-        assert dsk[dep][0] in (getitem, getter)
-        assert dsk[dep][1] == big_key
-
-
 def test_fuse_slices_with_alias():
     dsk = {
         "x": np.arange(16).reshape((4, 4)),
@@ -435,7 +416,7 @@ def test_double_dependencies():
 def test_fuse_roots():
     x = da.ones(10, chunks=(2,))
     y = da.zeros(10, chunks=(2,))
-    z = (x + 1) + (2 * y ** 2)
+    z = (x + 1) + (2 * y**2)
     (zz,) = dask.optimize(z)
     # assert len(zz.dask) == 5
     assert sum(map(dask.istask, zz.dask.values())) == 5  # there are some aliases
@@ -447,7 +428,7 @@ def test_fuse_roots_annotations():
     y = da.zeros(10, chunks=(2,))
 
     with dask.annotate(foo="bar"):
-        y = y ** 2
+        y = y**2
 
     z = (x + 1) + (2 * y)
     hlg = dask.blockwise.optimize_blockwise(z.dask)
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_random.py 2022.02.0+dfsg-1/dask/array/tests/test_random.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_random.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_random.py	2022-02-11 16:21:12.000000000 +0000
@@ -196,17 +196,12 @@ def test_array_broadcasting():
     assert da.random.normal(
         np.ones((1, 4)), da.ones((2, 3, 4), chunks=(2, 3, 4)), chunks=(2, 3, 4)
     ).compute().shape == (2, 3, 4)
-    assert (
-        da.random.normal(
-            scale=np.ones((1, 4)),
-            loc=da.ones((2, 3, 4), chunks=(2, 3, 4)),
-            size=(2, 2, 3, 4),
-            chunks=(2, 2, 3, 4),
-        )
-        .compute()
-        .shape
-        == (2, 2, 3, 4)
-    )
+    assert da.random.normal(
+        scale=np.ones((1, 4)),
+        loc=da.ones((2, 3, 4), chunks=(2, 3, 4)),
+        size=(2, 2, 3, 4),
+        chunks=(2, 2, 3, 4),
+    ).compute().shape == (2, 2, 3, 4)
 
     with pytest.raises(ValueError):
         da.random.normal(arr, np.ones((3, 1)), size=(2, 3, 4), chunks=3)
@@ -363,14 +358,6 @@ def test_randint_dtype():
     assert x.compute().dtype == "uint8"
 
 
-def test_doc_wraps_deprecated():
-    with pytest.warns(FutureWarning):
-
-        @da.random.doc_wraps(np.random.normal)
-        def f():
-            pass
-
-
 def test_raises_bad_kwarg():
     with pytest.raises(Exception) as info:
         da.random.standard_normal(size=(10,), dtype="float64")
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_rechunk.py 2022.02.0+dfsg-1/dask/array/tests/test_rechunk.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_rechunk.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_rechunk.py	2022-02-11 16:21:12.000000000 +0000
@@ -128,7 +128,7 @@ def test_rechunk_expand():
 
 def test_rechunk_expand2():
     (a, b) = (3, 2)
-    orig = np.random.uniform(0, 1, a ** b).reshape((a,) * b)
+    orig = np.random.uniform(0, 1, a**b).reshape((a,) * b)
     for off, off2 in product(range(1, a - 1), range(1, a - 1)):
         old = ((a - off, off),) * b
         x = da.from_array(orig, chunks=old)
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_reductions.py 2022.02.0+dfsg-1/dask/array/tests/test_reductions.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_reductions.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_reductions.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,6 +1,6 @@
 import os
 import warnings
-from itertools import zip_longest
+from itertools import permutations, zip_longest
 
 import pytest
 
@@ -809,3 +809,27 @@ def test_nan_func_does_not_warn(func):
     with pytest.warns(None) as rec:
         getattr(da, func)(d).compute()
     assert not rec  # did not warn
+
+
+@pytest.mark.parametrize("chunks", list(permutations(((2, 1) * 8, (3,) * 8, (6,) * 4))))
+@pytest.mark.parametrize("split_every", [2, 4])
+@pytest.mark.parametrize(
+    "axes", list(permutations((0, 1, 2), 2)) + list(permutations((0, 1, 2)))
+)
+def test_chunk_structure_independence(axes, split_every, chunks):
+    # Reducing an array should not depend on its chunk-structure!!!
+    # See Issue #8541: https://github.com/dask/dask/issues/8541
+    shape = tuple(np.sum(s) for s in chunks)
+    np_array = np.arange(np.prod(shape)).reshape(*shape)
+    x = da.from_array(np_array, chunks=chunks)
+    reduced_x = da.reduction(
+        x,
+        lambda x, axis, keepdims: x,
+        lambda x, axis, keepdims: x,
+        keepdims=True,
+        axis=axes,
+        split_every=split_every,
+        dtype=x.dtype,
+        meta=x._meta,
+    )
+    _assert_eq(reduced_x, np_array, check_chunks=False, check_shape=False)
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_routines.py 2022.02.0+dfsg-1/dask/array/tests/test_routines.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_routines.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_routines.py	2022-02-11 16:21:12.000000000 +0000
@@ -4,6 +4,7 @@ import sys
 from numbers import Number
 
 import pytest
+from numpy import AxisError
 
 from dask.delayed import delayed
 
@@ -379,7 +380,7 @@ def test_tensordot_more_than_26_dims():
     ndim = 27
     x = np.broadcast_to(1, [2] * ndim)
     dx = da.from_array(x, chunks=-1)
-    assert_eq(da.tensordot(dx, dx, ndim), np.array(2 ** ndim))
+    assert_eq(da.tensordot(dx, dx, ndim), np.array(2**ndim))
 
 
 def test_dot_method():
@@ -1413,6 +1414,25 @@ def test_ravel_with_array_like():
     assert isinstance(da.ravel([(0,), (0,)]), da.core.Array)
 
 
+@pytest.mark.parametrize("axis", [None, 0, 1, -1, (0, 1), (0, 2), (1, 2), 2])
+def test_expand_dims(axis):
+    a = np.arange(10)
+    d = da.from_array(a, chunks=(3,))
+
+    if axis is None:
+        with pytest.raises(TypeError):
+            da.expand_dims(d, axis=axis)
+    elif axis == 2:
+        with pytest.raises(AxisError):
+            da.expand_dims(d, axis=axis)
+    else:
+        a_e = np.expand_dims(a, axis=axis)
+        d_e = da.expand_dims(d, axis=axis)
+
+        assert_eq(d_e, a_e)
+        assert same_keys(d_e, da.expand_dims(d, axis=axis))
+
+
 @pytest.mark.parametrize("is_func", [True, False])
 @pytest.mark.parametrize("axis", [None, 0, -1, (0, -1)])
 def test_squeeze(is_func, axis):
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_slicing.py 2022.02.0+dfsg-1/dask/array/tests/test_slicing.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_slicing.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_slicing.py	2022-02-11 16:21:12.000000000 +0000
@@ -11,7 +11,6 @@ from dask import config
 from dask.array.slicing import (
     _sanitize_index_element,
     _slice_1d,
-    cached_cumsum,
     make_block_sorted_slices,
     new_blockdim,
     normalize_index,
@@ -977,29 +976,6 @@ def test_pathological_unsorted_slicing()
     assert "out-of-order" in str(info.list[0])
 
 
-def test_cached_cumsum():
-    a = (1, 2, 3, 4)
-    x = cached_cumsum(a)
-    y = cached_cumsum(a, initial_zero=True)
-    assert x == (1, 3, 6, 10)
-    assert y == (0, 1, 3, 6, 10)
-
-
-def test_cached_cumsum_nan():
-    a = (1, np.nan, 3)
-    x = cached_cumsum(a)
-    y = cached_cumsum(a, initial_zero=True)
-    np.testing.assert_equal(x, (1, np.nan, np.nan))
-    np.testing.assert_equal(y, (0, 1, np.nan, np.nan))
-
-
-def test_cached_cumsum_non_tuple():
-    a = [1, 2, 3]
-    assert cached_cumsum(a) == (1, 3, 6)
-    a[1] = 4
-    assert cached_cumsum(a) == (1, 5, 8)
-
-
 @pytest.mark.parametrize("params", [(2, 2, 1), (5, 3, 2)])
 def test_setitem_with_different_chunks_preserves_shape(params):
     """Reproducer for https://github.com/dask/dask/issues/3730.
@@ -1065,3 +1041,9 @@ def test_slice_array_3d_with_bool_numpy_
     actual = array[mask].compute()
     expected = np.arange(13, 24)
     assert_eq(actual, expected)
+
+
+def test_slice_array_null_dimension():
+    array = da.from_array(np.zeros((3, 0)))
+    expected = np.zeros((3, 0))[[0]]
+    assert_eq(array[[0]], expected)
diff -pruN 2022.01.0+dfsg-1/dask/array/tests/test_sparse.py 2022.02.0+dfsg-1/dask/array/tests/test_sparse.py
--- 2022.01.0+dfsg-1/dask/array/tests/test_sparse.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/tests/test_sparse.py	2022-02-11 16:21:12.000000000 +0000
@@ -22,7 +22,7 @@ functions = [
     lambda x: da.expm1(x),
     lambda x: 2 * x,
     lambda x: x / 2,
-    lambda x: x ** 2,
+    lambda x: x**2,
     lambda x: x + x,
     lambda x: x * x,
     lambda x: x[0],
diff -pruN 2022.01.0+dfsg-1/dask/array/utils.py 2022.02.0+dfsg-1/dask/array/utils.py
--- 2022.01.0+dfsg-1/dask/array/utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -226,7 +226,9 @@ def _check_chunks(x):
     return x
 
 
-def _get_dt_meta_computed(x, check_shape=True, check_graph=True, check_chunks=True):
+def _get_dt_meta_computed(
+    x, check_shape=True, check_graph=True, check_chunks=True, scheduler=None
+):
     x_original = x
     x_meta = None
     x_computed = None
@@ -240,7 +242,7 @@ def _get_dt_meta_computed(x, check_shape
         if check_chunks:
             # Replace x with persisted version to avoid computing it twice.
             x = _check_chunks(x)
-        x = x.compute(scheduler="sync")
+        x = x.compute(scheduler=scheduler)
         x_computed = x
         if hasattr(x, "todense"):
             x = x.todense()
@@ -266,6 +268,7 @@ def assert_eq(
     check_meta=True,
     check_chunks=True,
     check_type=True,
+    scheduler="sync",
     **kwargs,
 ):
     a_original = a
@@ -277,10 +280,18 @@ def assert_eq(
         b = np.array(b)
 
     a, adt, a_meta, a_computed = _get_dt_meta_computed(
-        a, check_shape=check_shape, check_graph=check_graph, check_chunks=check_chunks
+        a,
+        check_shape=check_shape,
+        check_graph=check_graph,
+        check_chunks=check_chunks,
+        scheduler=scheduler,
     )
     b, bdt, b_meta, b_computed = _get_dt_meta_computed(
-        b, check_shape=check_shape, check_graph=check_graph, check_chunks=check_chunks
+        b,
+        check_shape=check_shape,
+        check_graph=check_graph,
+        check_chunks=check_chunks,
+        scheduler=scheduler,
     )
 
     if str(adt) != str(bdt):
diff -pruN 2022.01.0+dfsg-1/dask/array/wrap.py 2022.02.0+dfsg-1/dask/array/wrap.py
--- 2022.01.0+dfsg-1/dask/array/wrap.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/array/wrap.py	2022-02-11 16:21:12.000000000 +0000
@@ -5,7 +5,8 @@ import numpy as np
 from tlz import curry
 
 from ..base import tokenize
-from ..layers import BlockwiseCreateArray
+from ..blockwise import blockwise as core_blockwise
+from ..layers import ArrayChunkShapeDep
 from ..utils import funcname
 from .core import Array, normalize_chunks
 from .utils import meta_from_array
@@ -64,12 +65,16 @@ def wrap_func_shape_as_first_arg(func, *
     kwargs = parsed["kwargs"]
     func = partial(func, dtype=dtype, **kwargs)
 
-    graph = BlockwiseCreateArray(
-        name,
+    out_ind = dep_ind = tuple(range(len(shape)))
+    graph = core_blockwise(
         func,
-        shape,
-        chunks,
+        name,
+        out_ind,
+        ArrayChunkShapeDep(chunks),
+        dep_ind,
+        numblocks={},
     )
+
     return Array(graph, name, chunks, dtype=dtype, meta=kwargs.get("meta", None))
 
 
diff -pruN 2022.01.0+dfsg-1/dask/bag/chunk.py 2022.02.0+dfsg-1/dask/bag/chunk.py
--- 2022.01.0+dfsg-1/dask/bag/chunk.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bag/chunk.py	2022-02-11 16:21:12.000000000 +0000
@@ -20,7 +20,7 @@ def groupby_tasks_group_hash(x, hash, gr
 def var_chunk(seq):
     squares, total, n = 0.0, 0.0, 0
     for x in seq:
-        squares += x ** 2
+        squares += x**2
         total += x
         n += 1
     return squares, total, n
diff -pruN 2022.01.0+dfsg-1/dask/bag/core.py 2022.02.0+dfsg-1/dask/bag/core.py
--- 2022.01.0+dfsg-1/dask/bag/core.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bag/core.py	2022-02-11 16:21:12.000000000 +0000
@@ -1474,7 +1474,7 @@ class Bag(DaskMethodsMixin):
         grouper,
         method=None,
         npartitions=None,
-        blocksize=2 ** 20,
+        blocksize=2**20,
         max_branch=None,
         shuffle=None,
     ):
@@ -1531,7 +1531,7 @@ class Bag(DaskMethodsMixin):
             msg = "Shuffle must be 'disk' or 'tasks'"
             raise NotImplementedError(msg)
 
-    def to_dataframe(self, meta=None, columns=None):
+    def to_dataframe(self, meta=None, columns=None, optimize_graph=True):
         """Create Dask Dataframe from a Dask Bag.
 
         Bag should contain tuples, dict records, or scalars.
@@ -1559,6 +1559,10 @@ class Bag(DaskMethodsMixin):
             result (any names not found in the data will become all-NA
             columns).  Note that if ``meta`` is provided, column names will be
             taken from there and this parameter is invalid.
+        optimize_graph : bool, optional
+            If True [default], the graph is optimized before converting into
+            :class:`dask.dataframe.DataFrame`.
+
 
         Examples
         --------
@@ -1596,14 +1600,15 @@ class Bag(DaskMethodsMixin):
         # the empty frame
         cols = list(meta.columns)
         dtypes = meta.dtypes.to_dict()
-        name = "to_dataframe-" + tokenize(self, cols, dtypes)
-        dsk = self.__dask_optimize__(self.dask, self.__dask_keys__())
 
-        for i in range(self.npartitions):
-            dsk[(name, i)] = (to_dataframe, (self.name, i), cols, dtypes)
+        dfs = self.map_partitions(to_dataframe, cols, dtypes)
+        if optimize_graph:
+            dsk = self.__dask_optimize__(dfs.dask, dfs.__dask_keys__())
+        else:
+            dsk = dfs.dask
 
         divisions = [None] * (self.npartitions + 1)
-        return dd.DataFrame(dsk, name, meta, divisions)
+        return dd.DataFrame(dsk, dfs.name, meta, divisions)
 
     def to_delayed(self, optimize_graph=True):
         """Convert into a list of ``dask.delayed`` objects, one per partition.
@@ -1711,7 +1716,7 @@ def accumulate_part(binop, seq, initial,
     return res[1:], res[-1]
 
 
-def partition(grouper, sequence, npartitions, p, nelements=2 ** 20):
+def partition(grouper, sequence, npartitions, p, nelements=2**20):
     """Partition a bag along a grouper, store partitions on disk."""
     for block in partition_all(nelements, sequence):
         d = groupby(grouper, block)
@@ -2304,7 +2309,7 @@ def _reduce(binop, sequence, initial=no_
 
 def make_group(k, stage):
     def h(x):
-        return x[0] // k ** stage % k
+        return x[0] // k**stage % k
 
     return h
 
@@ -2323,7 +2328,7 @@ def groupby_tasks(b, grouper, hash=hash,
     splits = []
     joins = []
 
-    inputs = [tuple(digit(i, j, k) for j in range(stages)) for i in range(k ** stages)]
+    inputs = [tuple(digit(i, j, k) for j in range(stages)) for i in range(k**stages)]
 
     b2 = b.map(partial(chunk.groupby_tasks_group_hash, hash=hash, grouper=grouper))
 
@@ -2399,7 +2404,7 @@ def groupby_tasks(b, grouper, hash=hash,
     return type(b)(graph, name, len(inputs))
 
 
-def groupby_disk(b, grouper, npartitions=None, blocksize=2 ** 20):
+def groupby_disk(b, grouper, npartitions=None, blocksize=2**20):
     if npartitions is None:
         npartitions = b.npartitions
     token = tokenize(b, grouper, npartitions, blocksize)
diff -pruN 2022.01.0+dfsg-1/dask/bag/tests/test_bag.py 2022.02.0+dfsg-1/dask/bag/tests/test_bag.py
--- 2022.01.0+dfsg-1/dask/bag/tests/test_bag.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bag/tests/test_bag.py	2022-02-11 16:21:12.000000000 +0000
@@ -31,7 +31,7 @@ from dask.bag.core import (
 from dask.bag.utils import assert_eq
 from dask.delayed import Delayed
 from dask.utils import filetexts, tmpdir, tmpfile
-from dask.utils_test import add, inc
+from dask.utils_test import add, hlg_layer_topological, inc
 
 dsk = {("x", 0): (range, 5), ("x", 1): (range, 5), ("x", 2): (range, 5)}
 
@@ -736,7 +736,7 @@ def test_from_empty_sequence():
 
 def test_product():
     b2 = b.product(b)
-    assert b2.npartitions == b.npartitions ** 2
+    assert b2.npartitions == b.npartitions**2
     assert set(b2) == {(i, j) for i in L for j in L}
 
     x = db.from_sequence([1, 2, 3, 4])
@@ -1161,15 +1161,12 @@ def test_from_delayed_iterator():
 
     delayed_records = delayed(lazy_records, pure=False)
     bag = db.from_delayed([delayed_records(5) for _ in range(5)])
-    assert (
-        db.compute(
-            bag.count(),
-            bag.pluck("operations").count(),
-            bag.pluck("operations").flatten().count(),
-            scheduler="sync",
-        )
-        == (25, 25, 50)
-    )
+    assert db.compute(
+        bag.count(),
+        bag.pluck("operations").count(),
+        bag.pluck("operations").flatten().count(),
+        scheduler="sync",
+    ) == (25, 25, 50)
 
 
 def test_range():
@@ -1293,7 +1290,7 @@ def test_groupby_tasks():
 
     b = db.from_sequence(range(1000), npartitions=100)
     out = b.groupby(lambda x: x % 123, shuffle="tasks")
-    assert len(out.dask) < 100 ** 2
+    assert len(out.dask) < 100**2
     partitions = dask.get(out.dask, out.__dask_keys__())
 
     for a in partitions:
@@ -1587,6 +1584,13 @@ def test_dask_layers_to_delayed(optimize
     # `da.Array.to_delayed` causes the layer name to not match the key.
     # Ensure the layer name is propagated between `Delayed` and `Item`.
     da = pytest.importorskip("dask.array")
+    i = db.Item.from_delayed(da.ones(1).to_delayed()[0])
+    name = i.key[0]
+    assert i.key[1:] == (0,)
+    assert i.dask.layers.keys() == {"delayed-" + name}
+    assert i.dask.dependencies == {"delayed-" + name: set()}
+    assert i.__dask_layers__() == ("delayed-" + name,)
+
     arr = da.ones(1) + 1
     delayed = arr.to_delayed(optimize_graph=optimize)[0]
     i = db.Item.from_delayed(delayed)
@@ -1605,3 +1609,39 @@ def test_dask_layers_to_delayed(optimize
 
     with pytest.raises(ValueError, match="not in"):
         db.Item(arr.dask, (arr.name,), layer="foo")
+
+
+def test_to_dataframe_optimize_graph():
+    pytest.importorskip("dask.dataframe")
+    from dask.dataframe.utils import assert_eq as assert_eq_df
+
+    x = db.from_sequence(
+        [{"name": "test1", "v1": 1}, {"name": "test2", "v1": 2}], npartitions=2
+    )
+
+    # linear `map` tasks will be fused by graph optimization
+    with dask.annotate(foo=True):
+        y = x.map(lambda a: dict(**a, v2=a["v1"] + 1))
+        y = y.map(lambda a: dict(**a, v3=a["v2"] + 1))
+        y = y.map(lambda a: dict(**a, v4=a["v3"] + 1))
+
+    # verifying the maps are not fused yet
+    assert len(y.dask) == y.npartitions * 4
+
+    # with optimizations
+    d = y.to_dataframe()
+
+    # All the `map` tasks have been fused
+    assert len(d.dask) < len(y.dask)
+
+    # no optimizations
+    d2 = y.to_dataframe(optimize_graph=False)
+
+    # Graph hasn't been fused. It contains all the original tasks,
+    # plus one extra layer converting to DataFrame
+    assert len(d2.dask) == len(y.dask) + d.npartitions
+
+    # Annotations are still there
+    assert hlg_layer_topological(d2.dask, 1).annotations == {"foo": True}
+
+    assert_eq_df(d, d2)
diff -pruN 2022.01.0+dfsg-1/dask/bag/tests/test_text.py 2022.02.0+dfsg-1/dask/bag/tests/test_text.py
--- 2022.01.0+dfsg-1/dask/bag/tests/test_text.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bag/tests/test_text.py	2022-02-11 16:21:12.000000000 +0000
@@ -31,7 +31,7 @@ files = {
 
 expected = "".join([files[v] for v in sorted(files)])
 
-fmt_bs = [(fmt, None) for fmt in compr] + [(None, "10 B")]
+fmt_bs = [(fmt, None) for fmt in compr] + [(None, "10 B")]  # type: ignore
 
 encodings = ["ascii", "utf-8"]  # + ['utf-16', 'utf-16-le', 'utf-16-be']
 fmt_bs_enc_path = [
diff -pruN 2022.01.0+dfsg-1/dask/bag/utils.py 2022.02.0+dfsg-1/dask/bag/utils.py
--- 2022.01.0+dfsg-1/dask/bag/utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bag/utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,7 +1,7 @@
-def assert_eq(a, b):
+def assert_eq(a, b, scheduler="sync"):
     if hasattr(a, "compute"):
-        a = a.compute(scheduler="sync")
+        a = a.compute(scheduler=scheduler)
     if hasattr(b, "compute"):
-        b = b.compute(scheduler="sync")
+        b = b.compute(scheduler=scheduler)
 
     assert a == b
diff -pruN 2022.01.0+dfsg-1/dask/base.py 2022.02.0+dfsg-1/dask/base.py
--- 2022.01.0+dfsg-1/dask/base.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/base.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,3 +1,5 @@
+from __future__ import annotations
+
 import datetime
 import inspect
 import os
@@ -5,6 +7,7 @@ import pickle
 import threading
 import uuid
 from collections import OrderedDict
+from collections.abc import Callable, Iterator, Mapping
 from concurrent.futures import Executor
 from contextlib import contextmanager
 from dataclasses import fields, is_dataclass
@@ -12,7 +15,6 @@ from functools import partial
 from hashlib import md5
 from numbers import Integral, Number
 from operator import getitem
-from typing import Iterator, Mapping, Set
 
 from packaging.version import parse as parse_version
 from tlz import curry, groupby, identity, merge
@@ -938,11 +940,11 @@ def normalize_object(o):
     )
 
 
-function_cache = {}
+function_cache: dict[Callable, Callable] = {}
 function_cache_lock = threading.Lock()
 
 
-def normalize_function(func):
+def normalize_function(func: Callable) -> Callable:
     try:
         return function_cache[func]
     except KeyError:
@@ -958,7 +960,7 @@ def normalize_function(func):
         return _normalize_function(func)
 
 
-def _normalize_function(func):
+def _normalize_function(func: Callable) -> Callable:
     if isinstance(func, Compose):
         first = getattr(func, "first", None)
         funcs = reversed((first,) + func.funcs) if first else func.funcs
@@ -1317,7 +1319,7 @@ def wait(x, timeout=None, return_when="A
         return x
 
 
-def get_collection_names(collection) -> Set[str]:
+def get_collection_names(collection) -> set[str]:
     """Infer the collection names from the dask keys, under the assumption that all keys
     are either tuples with matching first element, and that element is a string, or
     there is exactly one key and it is a string.
diff -pruN 2022.01.0+dfsg-1/dask/blockwise.py 2022.02.0+dfsg-1/dask/blockwise.py
--- 2022.01.0+dfsg-1/dask/blockwise.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/blockwise.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,18 +1,10 @@
+from __future__ import annotations
+
 import itertools
 import os
+from collections.abc import Hashable, Iterable, Mapping, Sequence
 from itertools import product
-from typing import (
-    Any,
-    Hashable,
-    Iterable,
-    List,
-    Mapping,
-    Optional,
-    Sequence,
-    Set,
-    Tuple,
-    Union,
-)
+from typing import Any
 
 import tlz as toolz
 
@@ -54,16 +46,16 @@ class BlockwiseDep:
     dask.blockwise.BlockwiseDepDict
     """
 
-    numblocks: Tuple[int, ...]
+    numblocks: tuple[int, ...]
     produces_tasks: bool
 
-    def __getitem__(self, idx: Tuple[int, ...]) -> Any:
+    def __getitem__(self, idx: tuple[int, ...]) -> Any:
         """Return Blockwise-function arguments for a specific index"""
         raise NotImplementedError(
             "Must define `__getitem__` for `BlockwiseDep` subclass."
         )
 
-    def get(self, idx: Tuple[int, ...], default) -> Any:
+    def get(self, idx: tuple[int, ...], default) -> Any:
         """BlockwiseDep ``__getitem__`` Wrapper"""
         try:
             return self.__getitem__(idx)
@@ -71,7 +63,7 @@ class BlockwiseDep:
             return default
 
     def __dask_distributed_pack__(
-        self, required_indices: Optional[List[Tuple[int, ...]]] = None
+        self, required_indices: list[tuple[int, ...]] | None = None
     ):
         """Client-side serialization for ``BlockwiseDep`` objects.
 
@@ -157,7 +149,7 @@ class BlockwiseDepDict(BlockwiseDep):
     def __init__(
         self,
         mapping: dict,
-        numblocks: Optional[Tuple[int, ...]] = None,
+        numblocks: tuple[int, ...] | None = None,
         produces_tasks: bool = False,
     ):
         self.mapping = mapping
@@ -166,11 +158,11 @@ class BlockwiseDepDict(BlockwiseDep):
         # By default, assume 1D shape
         self.numblocks = numblocks or (len(mapping),)
 
-    def __getitem__(self, idx: Tuple[int, ...]) -> Any:
+    def __getitem__(self, idx: tuple[int, ...]) -> Any:
         return self.mapping[idx]
 
     def __dask_distributed_pack__(
-        self, required_indices: Optional[List[Tuple[int, ...]]] = None
+        self, required_indices: list[tuple[int, ...]] | None = None
     ):
         from distributed.protocol import to_serialize
 
@@ -188,6 +180,32 @@ class BlockwiseDepDict(BlockwiseDep):
         return cls(**state)
 
 
+class BlockIndex(BlockwiseDep):
+    """Index BlockwiseDep argument
+
+    The purpose of this class is to provide each
+    block of a ``Blockwise``-based operation with
+    the current block index.
+    """
+
+    produces_tasks: bool = False
+
+    def __init__(self, numblocks: tuple[int, ...]):
+        # NOTE: Unused - Just needs to be set to
+        # follow the `BlockwiseDep` interface
+        self.numblocks = numblocks
+
+    def __getitem__(self, idx: tuple[int, ...]) -> tuple[int, ...]:
+        return idx
+
+    def __dask_distributed_pack__(self, **kwargs):
+        return {"numblocks": self.numblocks}
+
+    @classmethod
+    def __dask_distributed_unpack__(cls, state):
+        return cls(**state)
+
+
 def subs(task, substitution):
     """Create a new task with the values substituted
 
@@ -235,6 +253,11 @@ def blockwise(
     This is like the ``make_blockwise_graph`` function, but rather than construct a
     dict, it returns a symbolic Blockwise object.
 
+    ``*arrind_pairs`` is similar to those in `make_blockwise_graph`, but in addition to
+    allowing for collections it can accept BlockwiseDep instances, which allows for lazy
+    evaluation of arguments to ``func`` which might be different for different
+    chunks/paritions.
+
     See Also
     --------
     make_blockwise_graph
@@ -327,7 +350,7 @@ class Blockwise(Layer):
     dsk: dict
         A small graph to apply per-output-block.  May include keys from the
         input indices.
-    indices: Tuple[Tuple[str, Optional[Tuple[str, ...]]], ...]
+    indices: tuple[tuple[str, tuple[str, ...] | None], ...]
         An ordered mapping from input key name, like ``'x'``
         to input indices, like ``('i', 'j')``
         Or includes literals, which have ``None`` for an index value.
@@ -341,14 +364,14 @@ class Blockwise(Layer):
     new_axes: Mapping
         New index dimensions that may have been created and their size,
         e.g. ``{'j': 2, 'k': 3}``
-    output_blocks: Set[Tuple[int, ...]]
+    output_blocks: set[tuple[int, ...]]
         Specify a specific set of required output blocks. Since the graph
         will only contain the necessary tasks to generate these outputs,
         this kwarg can be used to "cull" the abstract layer (without needing
         to materialize the low-level graph).
     annotations: dict (optional)
         Layer annotations
-    io_deps: Dict[str, BlockwiseDep] (optional)
+    io_deps: dict[str, BlockwiseDep] (optional)
         Dictionary containing the mapping between "place-holder" collection
         keys and ``BlockwiseDep``-based objects.
         **WARNING**: This argument should only be used internally (for culling,
@@ -362,26 +385,26 @@ class Blockwise(Layer):
     """
 
     output: str
-    output_indices: Tuple[str, ...]
+    output_indices: tuple[str, ...]
     dsk: Mapping[str, tuple]
-    indices: Tuple[Tuple[str, Optional[Tuple[str, ...]]], ...]
+    indices: tuple[tuple[str, tuple[str, ...] | None], ...]
     numblocks: Mapping[str, Sequence[int]]
-    concatenate: Optional[bool]
+    concatenate: bool | None
     new_axes: Mapping[str, int]
-    output_blocks: Optional[Set[Tuple[int, ...]]]
+    output_blocks: set[tuple[int, ...]] | None
 
     def __init__(
         self,
         output: str,
         output_indices: Iterable[str],
         dsk: Mapping[str, tuple],
-        indices: Iterable[Tuple[Union[str, BlockwiseDep], Optional[Iterable[str]]]],
+        indices: Iterable[tuple[str | BlockwiseDep, Iterable[str] | None]],
         numblocks: Mapping[str, Sequence[int]],
-        concatenate: bool = None,
-        new_axes: Mapping[str, int] = None,
-        output_blocks: Set[Tuple[int, ...]] = None,
-        annotations: Mapping[str, Any] = None,
-        io_deps: Optional[Mapping[str, BlockwiseDep]] = None,
+        concatenate: bool | None = None,
+        new_axes: Mapping[str, int] | None = None,
+        output_blocks: set[tuple[int, ...]] | None = None,
+        annotations: Mapping[str, Any] | None = None,
+        io_deps: Mapping[str, BlockwiseDep] | None = None,
     ):
         super().__init__(annotations=annotations)
         self.output = output
@@ -703,7 +726,7 @@ class Blockwise(Layer):
 
     def cull(
         self, keys: set, all_hlg_keys: Iterable
-    ) -> Tuple[Layer, Mapping[Hashable, set]]:
+    ) -> tuple[Layer, Mapping[Hashable, set]]:
         # Culling is simple for Blockwise layers.  We can just
         # collect a set of required output blocks (tuples), and
         # only construct graph for these blocks in `make_blockwise_graph`
@@ -725,7 +748,7 @@ class Blockwise(Layer):
         keys: set,
         seed: Hashable,
         bind_to: Hashable = None,
-    ) -> Tuple[Layer, bool]:
+    ) -> tuple[Layer, bool]:
         names = {get_name_from_key(k) for k in keys}
         # We assume that 'keys' will contain either all or none of the output keys of
         # each of the layers, because clone/bind are always invoked at collection level.
@@ -1314,7 +1337,7 @@ def rewrite_blockwise(inputs):
 
     Parameters
     ----------
-    inputs : List[Blockwise]
+    inputs : list[Blockwise]
 
     Returns
     -------
diff -pruN 2022.01.0+dfsg-1/dask/bytes/__init__.py 2022.02.0+dfsg-1/dask/bytes/__init__.py
--- 2022.01.0+dfsg-1/dask/bytes/__init__.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bytes/__init__.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,17 +1 @@
-import warnings
-
-from fsspec.core import open as fs_open_file
-from fsspec.core import open_files as fs_open_files
-
-from ..utils import _deprecated
 from .core import read_bytes
-
-
-@_deprecated(use_instead="fsspec.core.open")
-def open_file(*arg, **kwargs):
-    return fs_open_file(*arg, **kwargs)
-
-
-@_deprecated(use_instead="fsspec.core.open_files")
-def open_files(*arg, **kwargs):
-    return fs_open_files(*arg, **kwargs)
diff -pruN 2022.01.0+dfsg-1/dask/bytes/tests/test_http.py 2022.02.0+dfsg-1/dask/bytes/tests/test_http.py
--- 2022.01.0+dfsg-1/dask/bytes/tests/test_http.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bytes/tests/test_http.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,3 +1,5 @@
+from __future__ import annotations
+
 import os
 import subprocess
 import sys
@@ -13,7 +15,11 @@ from dask.utils import tmpdir
 
 files = ["a", "b"]
 requests = pytest.importorskip("requests")
-errs = requests.exceptions.RequestException, FileNotFoundError
+
+errs: tuple[type[Exception], ...] = (
+    requests.exceptions.RequestException,
+    FileNotFoundError,
+)
 if parse_version(fsspec.__version__) > parse_version("0.7.4"):
     aiohttp = pytest.importorskip("aiohttp")
     errs = errs + (aiohttp.client_exceptions.ClientResponseError,)
diff -pruN 2022.01.0+dfsg-1/dask/bytes/tests/test_local.py 2022.02.0+dfsg-1/dask/bytes/tests/test_local.py
--- 2022.01.0+dfsg-1/dask/bytes/tests/test_local.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bytes/tests/test_local.py	2022-02-11 16:21:12.000000000 +0000
@@ -188,7 +188,7 @@ def test_read_bytes_delimited():
             assert ours == test
 
 
-fmt_bs = [(fmt, None) for fmt in compr] + [(fmt, 10) for fmt in compr]
+fmt_bs = [(fmt, None) for fmt in compr] + [(fmt, 10) for fmt in compr]  # type: ignore
 
 
 @pytest.mark.parametrize("fmt,blocksize", fmt_bs)
diff -pruN 2022.01.0+dfsg-1/dask/bytes/tests/test_s3.py 2022.02.0+dfsg-1/dask/bytes/tests/test_s3.py
--- 2022.01.0+dfsg-1/dask/bytes/tests/test_s3.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/bytes/tests/test_s3.py	2022-02-11 16:21:12.000000000 +0000
@@ -17,10 +17,15 @@ pytest.importorskip("flask")  # server m
 requests = pytest.importorskip("requests")
 
 from fsspec.compression import compr
-from fsspec.core import open_files
+from fsspec.core import get_fs_token_paths, open_files
 from s3fs import S3FileSystem as DaskS3FileSystem
 from tlz import concat, valmap
 
+try:
+    import fsspec.parquet as fsspec_parquet
+except ImportError:
+    fsspec_parquet = None
+
 from dask import compute
 from dask.bytes.core import read_bytes
 from dask.bytes.utils import compress
@@ -371,7 +376,8 @@ def test_read_bytes_delimited(s3, blocks
 
 
 @pytest.mark.parametrize(
-    "fmt,blocksize", [(fmt, None) for fmt in compr] + [(fmt, 10) for fmt in compr]
+    "fmt,blocksize",
+    [(fmt, None) for fmt in compr] + [(fmt, 10) for fmt in compr],  # type: ignore
 )
 def test_compression(s3, fmt, blocksize, s3so):
     if fmt not in compress:
@@ -480,6 +486,67 @@ def test_parquet(s3, engine, s3so, metad
 
     dd.utils.assert_eq(data, df2)
 
+    # Check that `open_file_options` arguments are
+    # really passed through to fsspec
+    if fsspec_parquet:
+
+        # Passing `open_file_options` kwargs will fail
+        # if you set an unsupported engine
+        with pytest.raises(ValueError):
+            dd.read_parquet(
+                url,
+                engine=engine,
+                storage_options=s3so,
+                open_file_options={
+                    "precache_options": {"method": "parquet", "engine": "foo"},
+                },
+            ).compute()
+
+        # ...but should work fine if you modify the
+        # maximum block-transfer size (max_block)
+        dd.read_parquet(
+            url,
+            engine=engine,
+            storage_options=s3so,
+            open_file_options={
+                "precache_options": {"method": "parquet", "max_block": 8_000},
+            },
+        ).compute()
+
+    # Check "open_file_func"
+    fs = get_fs_token_paths(url, storage_options=s3so)[0]
+
+    def _open(*args, check=True, **kwargs):
+        assert check
+        return fs.open(*args, **kwargs)
+
+    # Should fail if `check=False`
+    with pytest.raises(AssertionError):
+        dd.read_parquet(
+            url,
+            engine=engine,
+            storage_options=s3so,
+            open_file_options={"open_file_func": _open, "check": False},
+        ).compute()
+
+    # Should succeed otherwise
+    df3 = dd.read_parquet(
+        url,
+        engine=engine,
+        storage_options=s3so,
+        open_file_options={"open_file_func": _open},
+    )
+    dd.utils.assert_eq(data, df3)
+
+    # Check that `cache_type="all"` result is same
+    df4 = dd.read_parquet(
+        url,
+        engine=engine,
+        storage_options=s3so,
+        open_file_options={"cache_type": "all"},
+    )
+    dd.utils.assert_eq(data, df4)
+
 
 @pytest.mark.parametrize("engine", ["pyarrow", "fastparquet"])
 def test_parquet_append(s3, engine, s3so):
@@ -553,8 +620,8 @@ def test_parquet_wstoragepars(s3, s3so):
     assert s3.current().default_fill_cache is True
 
     dd.read_parquet(
-        url, storage_options=dict(**s3so, **{"default_block_size": 2 ** 20})
+        url, storage_options=dict(**s3so, **{"default_block_size": 2**20})
     )
-    assert s3.current().default_block_size == 2 ** 20
+    assert s3.current().default_block_size == 2**20
     with s3.current().open(url + "/_metadata") as f:
-        assert f.blocksize == 2 ** 20
+        assert f.blocksize == 2**20
diff -pruN 2022.01.0+dfsg-1/dask/callbacks.py 2022.02.0+dfsg-1/dask/callbacks.py
--- 2022.01.0+dfsg-1/dask/callbacks.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/callbacks.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,4 +1,8 @@
+from __future__ import annotations
+
+from collections.abc import Callable
 from contextlib import contextmanager
+from typing import ClassVar
 
 __all__ = ["Callback", "add_callbacks"]
 
@@ -43,7 +47,7 @@ class Callback:
     ...     x.compute()
     """
 
-    active = set()
+    active: ClassVar[set[tuple[Callable | None, ...]]] = set()
 
     def __init__(
         self, start=None, start_state=None, pretask=None, posttask=None, finish=None
@@ -60,7 +64,7 @@ class Callback:
             self._finish = finish
 
     @property
-    def _callback(self):
+    def _callback(self) -> tuple[Callable | None, ...]:
         fields = ["_start", "_start_state", "_pretask", "_posttask", "_finish"]
         return tuple(getattr(self, i, None) for i in fields)
 
@@ -72,10 +76,10 @@ class Callback:
     def __exit__(self, *args):
         self._cm.__exit__(*args)
 
-    def register(self):
+    def register(self) -> None:
         Callback.active.add(self._callback)
 
-    def unregister(self):
+    def unregister(self) -> None:
         Callback.active.remove(self._callback)
 
 
diff -pruN 2022.01.0+dfsg-1/dask/compatibility.py 2022.02.0+dfsg-1/dask/compatibility.py
--- 2022.01.0+dfsg-1/dask/compatibility.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/compatibility.py	2022-02-11 16:21:12.000000000 +0000
@@ -6,7 +6,7 @@ try:
     from math import prod
 except ImportError:
     # Python < 3.8
-    def prod(iterable, *, start=1):
+    def prod(iterable, *, start=1):  # type: ignore
         acc = start
         for el in iterable:
             acc *= el
@@ -14,17 +14,3 @@ except ImportError:
 
 
 _PY_VERSION = parse_version(".".join(map(str, sys.version_info[:3])))
-
-
-def __getattr__(name):
-    if name == "PY_VERSION":
-        import warnings
-
-        warnings.warn(
-            "dask.compatibility.PY_VERSION is deprecated and will be removed "
-            "in a future release.",
-            category=FutureWarning,
-        )
-        return _PY_VERSION
-    else:
-        raise AttributeError(f"module {__name__} has no attribute {name}")
diff -pruN 2022.01.0+dfsg-1/dask/config.py 2022.02.0+dfsg-1/dask/config.py
--- 2022.01.0+dfsg-1/dask/config.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/config.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,42 +1,65 @@
+from __future__ import annotations
+
 import ast
 import base64
-import builtins
+import builtins  # Explicitly use builtins.set as 'set' will be shadowed by a function
 import json
 import os
+import site
 import sys
 import threading
 import warnings
-from collections.abc import Mapping
+from collections.abc import Mapping, Sequence
+from typing import TYPE_CHECKING, Any
 
 import yaml
 
+if TYPE_CHECKING:
+    from typing_extensions import Literal
+
 no_default = "__no_default__"
 
 
-paths = [
-    os.getenv("DASK_ROOT_CONFIG", "/etc/dask"),
-    os.path.join(sys.prefix, "etc", "dask"),
-    os.path.join(os.path.expanduser("~"), ".config", "dask"),
-    os.path.join(os.path.expanduser("~"), ".dask"),
-]
+def _get_paths():
+    """Get locations to search for YAML configuration files.
+
+    This logic exists as a separate function for testing purposes.
+    """
+
+    paths = [
+        os.getenv("DASK_ROOT_CONFIG", "/etc/dask"),
+        os.path.join(sys.prefix, "etc", "dask"),
+        *[os.path.join(prefix, "etc", "dask") for prefix in site.PREFIXES],
+        os.path.join(os.path.expanduser("~"), ".config", "dask"),
+    ]
+    if "DASK_CONFIG" in os.environ:
+        paths.append(os.environ["DASK_CONFIG"])
+
+    # Remove duplicate paths while preserving ordering
+    paths = list(reversed(list(dict.fromkeys(reversed(paths)))))
+
+    return paths
+
+
+paths = _get_paths()
 
 if "DASK_CONFIG" in os.environ:
     PATH = os.environ["DASK_CONFIG"]
-    paths.append(PATH)
 else:
     PATH = os.path.join(os.path.expanduser("~"), ".config", "dask")
 
 
-global_config = config = {}
+config: dict = {}
+global_config = config  # alias
 
 
 config_lock = threading.Lock()
 
 
-defaults = []
+defaults: list[Mapping] = []
 
 
-def canonical_name(k, config):
+def canonical_name(k: str, config: dict) -> str:
     """Return the canonical name for a key.
 
     Handles user choice of '-' or '_' conventions by standardizing on whichever
@@ -59,7 +82,7 @@ def canonical_name(k, config):
     return k
 
 
-def update(old, new, priority="new"):
+def update(old: dict, new: Mapping, priority: Literal["old", "new"] = "new") -> dict:
     """Update a nested dictionary with values from another
 
     This is like dict.update except that it smoothly merges nested values
@@ -102,7 +125,7 @@ def update(old, new, priority="new"):
     return old
 
 
-def merge(*dicts):
+def merge(*dicts: Mapping) -> dict:
     """Update a sequence of nested dictionaries
 
     This prefers the values in the latter dictionaries to those in the former
@@ -118,13 +141,13 @@ def merge(*dicts):
     --------
     dask.config.update
     """
-    result = {}
+    result: dict = {}
     for d in dicts:
         update(result, d)
     return result
 
 
-def collect_yaml(paths=paths):
+def collect_yaml(paths: Sequence[str] = paths) -> list[dict]:
     """Collect configuration from yaml files
 
     This searches through a list of paths, expands to find all yaml or json
@@ -165,7 +188,7 @@ def collect_yaml(paths=paths):
     return configs
 
 
-def collect_env(env=None):
+def collect_env(env: Mapping[str, str] | None = None) -> dict:
     """Collect config from environment variables
 
     This grabs environment variables of the form "DASK_FOO__BAR_BAZ=123" and
@@ -196,13 +219,12 @@ def collect_env(env=None):
             except (SyntaxError, ValueError):
                 d[varname] = value
 
-    result = {}
+    result: dict = {}
     set(d, config=result)
-
     return result
 
 
-def ensure_file(source, destination=None, comment=True):
+def ensure_file(source: str, destination: str = None, comment: bool = True) -> None:
     """
     Copy file to default location if it does not already exist
 
@@ -299,7 +321,17 @@ class set:
     dask.config.get
     """
 
-    def __init__(self, arg=None, config=config, lock=config_lock, **kwargs):
+    config: dict
+    # [(op, path, value), ...]
+    _record: list[tuple[Literal["insert", "replace"], tuple[str, ...], Any]]
+
+    def __init__(
+        self,
+        arg: Mapping = None,
+        config: dict = config,
+        lock: threading.Lock = config_lock,
+        **kwargs,
+    ):
         with lock:
             self.config = config
             self._record = []
@@ -333,7 +365,14 @@ class set:
                 else:
                     d.pop(path[-1], None)
 
-    def _assign(self, keys, value, d, path=(), record=True):
+    def _assign(
+        self,
+        keys: Sequence[str],
+        value: Any,
+        d: dict,
+        path: tuple[str, ...] = (),
+        record: bool = True,
+    ) -> None:
         """Assign value into a nested configuration dictionary
 
         Parameters
@@ -344,7 +383,7 @@ class set:
         d : dict
             The part of the nested dictionary into which we want to assign the
             value
-        path : Tuple[str], optional
+        path : tuple[str], optional
             The path history up to this point.
         record : bool, optional
             Whether this operation needs to be recorded to allow for rollback.
@@ -370,16 +409,16 @@ class set:
             self._assign(keys[1:], value, d[key], path, record=record)
 
 
-def collect(paths=paths, env=None):
+def collect(paths: list[str] = paths, env: Mapping[str, str] = None) -> dict:
     """
     Collect configuration from paths and environment variables
 
     Parameters
     ----------
-    paths : List[str]
+    paths : list[str]
         A list of paths to search for yaml config files
 
-    env : dict
+    env : Mapping[str, str]
         The system environment variables
 
     Returns
@@ -399,7 +438,9 @@ def collect(paths=paths, env=None):
     return merge(*configs)
 
 
-def refresh(config=config, defaults=defaults, **kwargs):
+def refresh(
+    config: dict = config, defaults: list[Mapping] = defaults, **kwargs
+) -> None:
     """
     Update configuration by re-reading yaml files and env variables
 
@@ -431,7 +472,12 @@ def refresh(config=config, defaults=defa
     update(config, collect(**kwargs))
 
 
-def get(key, default=no_default, config=config, override_with=None):
+def get(
+    key: str,
+    default: Any = no_default,
+    config: dict = config,
+    override_with: Any | None = None,
+) -> Any:
     """
     Get elements from global config
 
@@ -478,7 +524,7 @@ def get(key, default=no_default, config=
     return result
 
 
-def rename(aliases, config=config):
+def rename(aliases: Mapping, config: dict = config) -> None:
     """Rename old keys to new keys
 
     This helps migrate older configuration versions over time
@@ -497,7 +543,9 @@ def rename(aliases, config=config):
     set(new, config=config)
 
 
-def update_defaults(new, config=config, defaults=defaults):
+def update_defaults(
+    new: Mapping, config: dict = config, defaults: list[Mapping] = defaults
+) -> None:
     """Add a new set of defaults to the configuration
 
     It does two things:
@@ -510,7 +558,7 @@ def update_defaults(new, config=config,
     update(config, new, priority="old")
 
 
-def expand_environment_variables(config):
+def expand_environment_variables(config: Any) -> Any:
     """Expand environment variables in a nested config dictionary
 
     This function will recursively search through any nested dictionaries
@@ -535,7 +583,7 @@ def expand_environment_variables(config)
     elif isinstance(config, str):
         return os.path.expandvars(config)
     elif isinstance(config, (list, tuple, builtins.set)):
-        return type(config)([expand_environment_variables(v) for v in config])
+        return type(config)(expand_environment_variables(v) for v in config)
     else:
         return config
 
@@ -559,7 +607,7 @@ deprecations = {
 }
 
 
-def check_deprecations(key: str, deprecations: dict = deprecations):
+def check_deprecations(key: str, deprecations: dict = deprecations) -> str:
     """Check if the provided value has been renamed or removed
 
     Parameters
@@ -603,7 +651,7 @@ def check_deprecations(key: str, depreca
         return key
 
 
-def serialize(data):
+def serialize(data: Any) -> str:
     """Serialize config data into a string.
 
     Typically used to pass config via the ``DASK_INTERNAL_INHERIT_CONFIG`` environment variable.
@@ -622,7 +670,7 @@ def serialize(data):
     return base64.urlsafe_b64encode(json.dumps(data).encode()).decode()
 
 
-def deserialize(data):
+def deserialize(data: str) -> Any:
     """De-serialize config data into the original object.
 
     Typically when receiving config via the ``DASK_INTERNAL_INHERIT_CONFIG`` environment variable.
@@ -641,7 +689,7 @@ def deserialize(data):
     return json.loads(base64.urlsafe_b64decode(data.encode()).decode())
 
 
-def _initialize():
+def _initialize() -> None:
     fn = os.path.join(os.path.dirname(__file__), "dask.yaml")
 
     with open(fn) as f:
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/accessor.py 2022.02.0+dfsg-1/dask/dataframe/accessor.py
--- 2022.01.0+dfsg-1/dask/dataframe/accessor.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/accessor.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,4 +1,7 @@
+from __future__ import annotations
+
 from functools import partial
+from typing import ClassVar
 
 import numpy as np
 import pandas as pd
@@ -23,7 +26,7 @@ class Accessor:
     Subclasses should define ``_accessor_name``
     """
 
-    _not_implemented = set()
+    _not_implemented: ClassVar[set[str]] = set()
 
     def __init__(self, series):
         from .core import Series
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/backends.py 2022.02.0+dfsg-1/dask/dataframe/backends.py
--- 2022.01.0+dfsg-1/dask/dataframe/backends.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/backends.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,4 +1,7 @@
+from __future__ import annotations
+
 import warnings
+from typing import Iterable
 
 import numpy as np
 import pandas as pd
@@ -37,7 +40,6 @@ from .utils import (
     _empty_series,
     _nonempty_scalar,
     _scalar_from_dtype,
-    is_categorical_dtype,
     is_float_na_dtype,
     is_integer_na_dtype,
 )
@@ -61,16 +63,16 @@ def _(x):
 
 
 @make_meta_dispatch.register((pd.Series, pd.DataFrame))
-def make_meta_pandas(x, index=None):
+def _(x, index=None):
     return x.iloc[:0]
 
 
 @make_meta_dispatch.register(pd.Index)
-def make_meta_index(x, index=None):
+def _(x, index=None):
     return x[0:0]
 
 
-meta_object_types = (pd.Series, pd.DataFrame, pd.Index, pd.MultiIndex)
+meta_object_types: tuple[type, ...] = (pd.Series, pd.DataFrame, pd.Index, pd.MultiIndex)
 try:
     import scipy.sparse as sp
 
@@ -126,7 +128,7 @@ def make_meta_object(x, index=None):
         )
     if isinstance(x, tuple) and len(x) == 2:
         return _empty_series(x[0], x[1], index=index)
-    elif isinstance(x, (list, tuple)):
+    elif isinstance(x, Iterable) and not isinstance(x, str):
         if not all(isinstance(i, tuple) and len(i) == 2 for i in x):
             raise ValueError(f"Expected iterable of tuples of (name, dtype), got {x}")
         return pd.DataFrame(
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/_compat.py 2022.02.0+dfsg-1/dask/dataframe/_compat.py
--- 2022.01.0+dfsg-1/dask/dataframe/_compat.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/_compat.py	2022-02-11 16:21:12.000000000 +0000
@@ -13,6 +13,10 @@ PANDAS_GT_130 = PANDAS_VERSION >= parse_
 PANDAS_GT_131 = PANDAS_VERSION >= parse_version("1.3.1")
 PANDAS_GT_133 = PANDAS_VERSION >= parse_version("1.3.3")
 PANDAS_GT_140 = PANDAS_VERSION >= parse_version("1.4.0")
+# FIXME: Using `.release` below as versions like `1.5.0.dev0+268.gbe8d1ec880`
+# are less than `1.5.0` with `packaging.version`. Update to use `parse_version("1.5.0")`
+# below once `pandas=1.5.0` is released
+PANDAS_GT_150 = PANDAS_VERSION.release >= (1, 5, 0)
 
 import pandas.testing as tm
 
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/core.py 2022.02.0+dfsg-1/dask/dataframe/core.py
--- 2022.01.0+dfsg-1/dask/dataframe/core.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/core.py	2022-02-11 16:21:12.000000000 +0000
@@ -25,6 +25,7 @@ from ..blockwise import Blockwise, Block
 from ..context import globalmethod
 from ..delayed import Delayed, delayed, unpack_collections
 from ..highlevelgraph import HighLevelGraph
+from ..layers import DataFrameTreeReduction
 from ..utils import (
     IndexCallable,
     M,
@@ -47,6 +48,7 @@ from ..utils import (
 )
 from ..widgets import get_template
 from . import methods
+from ._compat import PANDAS_GT_140, PANDAS_GT_150
 from .accessor import DatetimeAccessor, StringAccessor
 from .categorical import CategoricalAccessor, categorize
 from .dispatch import (
@@ -402,7 +404,7 @@ class _Frame(DaskMethodsMixin, OperatorM
         """Return number of partitions"""
         return len(self.divisions) - 1
 
-    @property
+    @property  # type: ignore
     @derived_from(pd.DataFrame)
     def attrs(self):
         return self._meta.attrs
@@ -1499,10 +1501,12 @@ Dask Name: {name}, {task} tasks"""
 
     @derived_from(pd.DataFrame)
     def replace(self, to_replace=None, value=None, regex=False):
+        # In PANDAS_GT_140 pandas starts using no_default instead of None
+        value_kwarg = {"value": value} if value is not None else {}
         return self.map_partitions(
             M.replace,
             to_replace=to_replace,
-            value=value,
+            **value_kwarg,
             regex=regex,
             enforce_metadata=False,
         )
@@ -1568,6 +1572,7 @@ Dask Name: {name}, {task} tasks"""
         method=None,
         compute=True,
         parallel=False,
+        engine_kwargs=None,
     ):
         """See dd.to_sql docstring for more information"""
         from .io import to_sql
@@ -1585,6 +1590,7 @@ Dask Name: {name}, {task} tasks"""
             method=method,
             compute=compute,
             parallel=parallel,
+            engine_kwargs=engine_kwargs,
         )
 
     def to_json(self, filename, *args, **kwargs):
@@ -1969,6 +1975,7 @@ Dask Name: {name}, {task} tasks"""
             chunk_kwargs={"dropna": dropna},
             aggregate_kwargs={"dropna": dropna},
         )
+        mode_series.name = self.name
         return mode_series
 
     @_numeric_only
@@ -2910,6 +2917,12 @@ Dask Name: {name}, {task} tasks"""
 
     @derived_from(pd.Series)
     def append(self, other, interleave_partitions=False):
+        if PANDAS_GT_140:
+            warnings.warn(
+                "The frame.append method is deprecated and will be removed from"
+                "dask in a future version. Use dask.dataframe.concat instead.",
+                FutureWarning,
+            )
         # because DataFrame.append will override the method,
         # wrap by pd.Series.append docstring
         from .multi import concat
@@ -3656,7 +3669,8 @@ Dask Name: {name}, {task} tasks""".forma
 
     @derived_from(pd.Series)
     def to_frame(self, name=None):
-        return self.map_partitions(M.to_frame, name, meta=self._meta.to_frame(name))
+        args = [] if name is None else [name]
+        return self.map_partitions(M.to_frame, *args, meta=self._meta.to_frame(*args))
 
     @derived_from(pd.Series)
     def to_string(self, max_rows=5):
@@ -3815,6 +3829,12 @@ Dask Name: {name}, {task} tasks""".forma
     @property
     @derived_from(pd.Series)
     def is_monotonic(self):
+        if PANDAS_GT_150:
+            warnings.warn(
+                "is_monotonic is deprecated and will be removed in a future version. "
+                "Use is_monotonic_increasing instead.",
+                FutureWarning,
+            )
         return self.is_monotonic_increasing
 
     @property
@@ -3839,6 +3859,11 @@ Dask Name: {name}, {task} tasks""".forma
             token="monotonic_decreasing",
         )
 
+    @derived_from(pd.Series)
+    def view(self, dtype):
+        meta = self._meta.view(dtype)
+        return self.map_partitions(M.view, dtype, meta=meta)
+
 
 class Index(Series):
     _partition_type = pd.Index
@@ -3990,12 +4015,12 @@ class Index(Series):
     def to_frame(self, index=True, name=None):
         if not index:
             raise NotImplementedError()
+        args = [index] if name is None else [index, name]
 
         return self.map_partitions(
             M.to_frame,
-            index,
-            name,
-            meta=self._meta.to_frame(index, name),
+            *args,
+            meta=self._meta.to_frame(*args),
             transform_divisions=False,
         )
 
@@ -4022,6 +4047,12 @@ class Index(Series):
     @property
     @derived_from(pd.Index)
     def is_monotonic(self):
+        if PANDAS_GT_150:
+            warnings.warn(
+                "is_monotonic is deprecated and will be removed in a future version. "
+                "Use is_monotonic_increasing instead.",
+                FutureWarning,
+            )
         return super().is_monotonic_increasing
 
     @property
@@ -5174,7 +5205,6 @@ class DataFrame(_Frame):
             mode_series = Series.mode(
                 col_series, dropna=dropna, split_every=split_every
             )
-            mode_series.name = col_series.name
             mode_series_list.append(mode_series)
 
         name = "concat-" + tokenize(*mode_series_list)
@@ -5856,63 +5886,45 @@ def apply_concat_apply(
         split_out_setup_kwargs,
     )
 
-    # Chunk
-    a = f"{token or funcname(chunk)}-chunk-{token_key}"
-    if len(args) == 1 and isinstance(args[0], _Frame) and not chunk_kwargs:
-        dsk = {
-            (a, 0, i, 0): (chunk, key) for i, key in enumerate(args[0].__dask_keys__())
-        }
-    else:
-        dsk = {
-            (a, 0, i, 0): (
-                apply,
-                chunk,
-                [(x._name, i) if isinstance(x, _Frame) else x for x in args],
-                chunk_kwargs,
-            )
-            for i in range(npartitions)
-        }
+    # Blockwise Chunk Layer
+    chunk_name = f"{token or funcname(chunk)}-chunk-{token_key}"
+    chunked = map_partitions(
+        chunk,
+        *args,
+        token=chunk_name,
+        # NOTE: We are NOT setting the correct
+        # `meta` here on purpose. We are using
+        # `map_partitions` as a convenient way
+        # to build a `Blockwise` layer, and need
+        # to avoid the metadata emulation step.
+        meta=dfs[0],
+        enforce_metadata=False,
+        transform_divisions=False,
+        align_dataframes=False,
+        **chunk_kwargs,
+    )
 
-    # Split
+    # Blockwise Split Layer
     if split_out and split_out > 1:
-        split_prefix = "split-%s" % token_key
-        shard_prefix = "shard-%s" % token_key
-        for i in range(npartitions):
-            dsk[(split_prefix, i)] = (
-                hash_shard,
-                (a, 0, i, 0),
-                split_out,
-                split_out_setup,
-                split_out_setup_kwargs,
-                ignore_index,
-            )
-            for j in range(split_out):
-                dsk[(shard_prefix, 0, i, j)] = (getitem, (split_prefix, i), j)
-        a = shard_prefix
-    else:
-        split_out = 1
-
-    # Combine
-    b = f"{token or funcname(combine)}-combine-{token_key}"
-    k = npartitions
-    depth = 0
-    while k > split_every:
-        for part_i, inds in enumerate(partition_all(split_every, range(k))):
-            for j in range(split_out):
-                conc = (_concat, [(a, depth, i, j) for i in inds], ignore_index)
-                if combine_kwargs:
-                    dsk[(b, depth + 1, part_i, j)] = (
-                        apply,
-                        combine,
-                        [conc],
-                        combine_kwargs,
-                    )
-                else:
-                    dsk[(b, depth + 1, part_i, j)] = (combine, conc)
-        k = part_i + 1
-        a = b
-        depth += 1
+        chunked = chunked.map_partitions(
+            hash_shard,
+            split_out,
+            split_out_setup,
+            split_out_setup_kwargs,
+            ignore_index,
+            token="split-%s" % token_key,
+            # NOTE: We are NOT setting the correct
+            # `meta` here on purpose. We are using
+            # `map_partitions` as a convenient way
+            # to build a `Blockwise` layer, and need
+            # to avoid the metadata emulation step.
+            meta=dfs[0],
+            enforce_metadata=False,
+            transform_divisions=False,
+            align_dataframes=False,
+        )
 
+    # Handle sort behavior
     if sort is not None:
         if sort and split_out > 1:
             raise NotImplementedError(
@@ -5922,14 +5934,21 @@ def apply_concat_apply(
         aggregate_kwargs = aggregate_kwargs or {}
         aggregate_kwargs["sort"] = sort
 
-    # Aggregate
-    for j in range(split_out):
-        b = f"{token or funcname(aggregate)}-agg-{token_key}"
-        conc = (_concat, [(a, depth, i, j) for i in range(k)], ignore_index)
-        if aggregate_kwargs:
-            dsk[(b, j)] = (apply, aggregate, [conc], aggregate_kwargs)
-        else:
-            dsk[(b, j)] = (aggregate, conc)
+    # Tree-Reduction Layer
+    final_name = f"{token or funcname(aggregate)}-agg-{token_key}"
+    layer = DataFrameTreeReduction(
+        final_name,
+        chunked._name,
+        npartitions,
+        partial(_concat, ignore_index=ignore_index),
+        partial(combine, **combine_kwargs) if combine_kwargs else combine,
+        finalize_func=partial(aggregate, **aggregate_kwargs)
+        if aggregate_kwargs
+        else aggregate,
+        split_every=split_every,
+        split_out=split_out if (split_out and split_out > 1) else None,
+        tree_node_name=f"{token or funcname(combine)}-combine-{token_key}",
+    )
 
     if meta is no_default:
         meta_chunk = _emulate(chunk, *args, udf=True, **chunk_kwargs)
@@ -5942,11 +5961,9 @@ def apply_concat_apply(
         parent_meta=dfs[0]._meta,
     )
 
-    graph = HighLevelGraph.from_collections(b, dsk, dependencies=dfs)
-
-    divisions = [None] * (split_out + 1)
-
-    return new_dd_object(graph, b, meta, divisions, parent_meta=dfs[0]._meta)
+    graph = HighLevelGraph.from_collections(final_name, layer, dependencies=(chunked,))
+    divisions = [None] * ((split_out or 1) + 1)
+    return new_dd_object(graph, final_name, meta, divisions, parent_meta=dfs[0]._meta)
 
 
 aca = apply_concat_apply
@@ -6051,8 +6068,10 @@ def map_partitions(
         # Use non-normalized kwargs here, as we want the real values (not
         # delayed values)
         meta = _emulate(func, *args, udf=True, **kwargs)
+        meta_is_emulated = True
     else:
         meta = make_meta(meta, index=meta_index, parent_meta=parent_meta)
+        meta_is_emulated = False
 
     if all(isinstance(arg, Scalar) for arg in args):
         layer = {
@@ -6061,6 +6080,13 @@ def map_partitions(
         graph = HighLevelGraph.from_collections(name, layer, dependencies=args)
         return Scalar(graph, name, meta)
     elif not (has_parallel_type(meta) or is_arraylike(meta) and meta.shape):
+        if not meta_is_emulated:
+            warnings.warn(
+                "Meta is not valid, `map_partitions` expects output to be a pandas object. "
+                "Try passing a pandas object as meta or a dict or tuple representing the "
+                "(name, dtype) of the columns. In the future the meta you passed will not work.",
+                FutureWarning,
+            )
         # If `meta` is not a pandas object, the concatenated results will be a
         # different type
         meta = make_meta(_concat([meta]), index=meta_index)
@@ -7203,7 +7229,7 @@ def new_dd_object(dsk, name, meta, divis
         return get_parallel_type(meta)(dsk, name, meta, divisions)
 
 
-def partitionwise_graph(func, name, *args, **kwargs):
+def partitionwise_graph(func, layer_name, *args, **kwargs):
     """
     Apply a function partition-wise across arguments to create layer of a graph
 
@@ -7218,8 +7244,9 @@ def partitionwise_graph(func, name, *arg
     Parameters
     ----------
     func: callable
-    name: str
-        descriptive name for the operation
+    layer_name: str
+        Descriptive name for the operation. Used as the output name
+        in the resulting ``Blockwise`` graph layer.
     *args:
     **kwargs:
 
@@ -7269,7 +7296,7 @@ def partitionwise_graph(func, name, *arg
         else:
             pairs.extend([arg, None])
     return blockwise(
-        func, name, "i", *pairs, numblocks=numblocks, concatenate=True, **kwargs
+        func, layer_name, "i", *pairs, numblocks=numblocks, concatenate=True, **kwargs
     )
 
 
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/_dtypes.py 2022.02.0+dfsg-1/dask/dataframe/_dtypes.py
--- 2022.01.0+dfsg-1/dask/dataframe/_dtypes.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/_dtypes.py	2022-02-11 16:21:12.000000000 +0000
@@ -4,7 +4,7 @@ from .extensions import make_array_nonem
 
 
 @make_array_nonempty.register(pd.DatetimeTZDtype)
-def _dtype(dtype):
+def _(dtype):
     return pd.array([pd.Timestamp(1), pd.NaT], dtype=dtype)
 
 
@@ -24,7 +24,7 @@ def _(x):
 
 
 @make_array_nonempty.register(pd.BooleanDtype)
-def _dtype(dtype):
+def _(dtype):
     return pd.array([True, pd.NA], dtype=dtype)
 
 
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/groupby.py 2022.02.0+dfsg-1/dask/dataframe/groupby.py
--- 2022.01.0+dfsg-1/dask/dataframe/groupby.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/groupby.py	2022-02-11 16:21:12.000000000 +0000
@@ -342,7 +342,7 @@ def _var_agg(g, levels, ddof, sort=False
     n = g[g.columns[-nc // 3 :]].rename(columns=lambda c: c[0])
 
     # TODO: replace with _finalize_var?
-    result = x2 - x ** 2 / n
+    result = x2 - x**2 / n
     div = n - ddof
     div[div < 0] = 0
     result /= div
@@ -988,7 +988,7 @@ def _finalize_var(df, count_column, sum_
     x = df[sum_column]
     x2 = df[sum2_column]
 
-    result = x2 - x ** 2 / n
+    result = x2 - x**2 / n
     div = n - ddof
     div[div < 0] = 0
     result /= div
@@ -1105,7 +1105,7 @@ class _GroupBy:
             by_meta, group_keys=group_keys, **self.observed, **self.dropna
         )
 
-    @property
+    @property  # type: ignore
     @_deprecated()
     def index(self):
         return self.by
@@ -1231,7 +1231,7 @@ class _GroupBy:
         cumpart_ext = cumpart_raw_frame.assign(
             **{
                 i: self.obj[i]
-                if np.isscalar(i) and i in self.obj.columns
+                if np.isscalar(i) and i in getattr(self.obj, "columns", [])
                 else self.obj.index
                 for i in by
             }
@@ -1282,9 +1282,12 @@ class _GroupBy:
                 aggregate,
                 initial,
             )
-        graph = HighLevelGraph.from_collections(
-            name, dask, dependencies=[cumpart_raw, cumpart_ext, cumlast]
-        )
+
+        dependencies = [cumpart_raw]
+        if self.obj.npartitions > 1:
+            dependencies += [cumpart_ext, cumlast]
+
+        graph = HighLevelGraph.from_collections(name, dask, dependencies=dependencies)
         return new_dd_object(graph, name, chunk(self._meta), self.obj.divisions)
 
     def _shuffle(self, meta):
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/hyperloglog.py 2022.02.0+dfsg-1/dask/dataframe/hyperloglog.py
--- 2022.01.0+dfsg-1/dask/dataframe/hyperloglog.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/hyperloglog.py	2022-02-11 16:21:12.000000000 +0000
@@ -75,6 +75,6 @@ def estimate_count(Ms, b):
         V = (M == 0).sum()
         if V:
             return m * np.log(m / V)
-    if E > 2 ** 32 / 30.0:
-        return -(2 ** 32) * np.log1p(-E / 2 ** 32)
+    if E > 2**32 / 30.0:
+        return -(2**32) * np.log1p(-E / 2**32)
     return E
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/__init__.py 2022.02.0+dfsg-1/dask/dataframe/__init__.py
--- 2022.01.0+dfsg-1/dask/dataframe/__init__.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/__init__.py	2022-02-11 16:21:12.000000000 +0000
@@ -23,6 +23,8 @@ try:
         read_fwf,
         read_hdf,
         read_json,
+        read_sql,
+        read_sql_query,
         read_sql_table,
         read_table,
         to_bag,
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/csv.py 2022.02.0+dfsg-1/dask/dataframe/io/csv.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/csv.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/csv.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,3 +1,4 @@
+import os
 from collections.abc import Mapping
 from io import BytesIO
 from warnings import catch_warnings, simplefilter, warn
@@ -436,7 +437,7 @@ def auto_blocksize(total_memory, cpu_cou
 
 
 def _infer_block_size():
-    default = 2 ** 25
+    default = 2**25
     if psutil is not None:
         with catch_warnings():
             simplefilter("ignore", RuntimeWarning)
@@ -766,7 +767,7 @@ read_fwf = make_reader(pd.read_fwf, "rea
 def _write_csv(df, fil, *, depend_on=None, **kwargs):
     with fil as f:
         df.to_csv(f, **kwargs)
-    return None
+    return os.path.normpath(fil.path)
 
 
 def to_csv(
@@ -953,8 +954,7 @@ def to_csv(
 
         import dask
 
-        dask.compute(*values, **compute_kwargs)
-        return [f.path for f in files]
+        return list(dask.compute(*values, **compute_kwargs))
     else:
         return values
 
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/__init__.py 2022.02.0+dfsg-1/dask/dataframe/io/__init__.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/__init__.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/__init__.py	2022-02-11 16:21:12.000000000 +0000
@@ -12,7 +12,7 @@ from .io import (
     to_records,
 )
 from .json import read_json, to_json
-from .sql import read_sql_table, to_sql
+from .sql import read_sql, read_sql_query, read_sql_table, to_sql
 
 try:
     from .parquet import read_parquet, to_parquet
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/io.py 2022.02.0+dfsg-1/dask/dataframe/io/io.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/io.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/io.py	2022-02-11 16:21:12.000000000 +0000
@@ -570,10 +570,11 @@ def from_delayed(
 
     Parameters
     ----------
-    dfs : list of Delayed
+    dfs : list of Delayed or Future
         An iterable of ``dask.delayed.Delayed`` objects, such as come from
-        ``dask.delayed`` These comprise the individual partitions of the
-        resulting dataframe.
+        ``dask.delayed`` or an iterable of ``distributed.Future`` objects,
+        such as come from ``client.submit`` interface. These comprise the individual
+        partitions of the resulting dataframe.
     $META
     divisions : tuple, str, optional
         Partition boundaries along the index.
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/json.py 2022.02.0+dfsg-1/dask/dataframe/io/json.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/json.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/json.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,4 +1,6 @@
 import io
+import os
+from itertools import zip_longest
 
 import pandas as pd
 from fsspec.core import open_files
@@ -90,8 +92,7 @@ def to_json(
     if compute:
         if compute_kwargs is None:
             compute_kwargs = dict()
-        dask_compute(parts, **compute_kwargs)
-        return [f.path for f in outfiles]
+        return list(dask_compute(*parts, **compute_kwargs))
     else:
         return parts
 
@@ -99,6 +100,7 @@ def to_json(
 def write_json_partition(df, openfile, kwargs):
     with openfile as f:
         df.to_json(f, **kwargs)
+    return os.path.normpath(openfile.path)
 
 
 @insert_meta_param_description
@@ -108,12 +110,14 @@ def read_json(
     lines=None,
     storage_options=None,
     blocksize=None,
-    sample=2 ** 20,
+    sample=2**20,
     encoding="utf-8",
     errors="strict",
     compression="infer",
     meta=None,
     engine=pd.read_json,
+    include_path_column=False,
+    path_converter=None,
     **kwargs,
 ):
     """Create a dataframe from a set of JSON files
@@ -148,7 +152,7 @@ def read_json(
         newline character.
     sample: int
         Number of bytes to pre-load, to provide an empty dataframe structure
-        to any blocks without data. Only relevant is using blocksize.
+        to any blocks without data. Only relevant when using blocksize.
     encoding, errors:
         Text conversion, ``see bytes.decode()``
     compression : string or None
@@ -156,6 +160,14 @@ def read_json(
     engine : function object, default ``pd.read_json``
         The underlying function that dask will use to read JSON files. By
         default, this will be the pandas JSON reader (``pd.read_json``).
+    include_path_column : bool or str, optional
+        Include a column with the file path where each row in the dataframe
+        originated. If ``True``, a new column is added to the dataframe called
+        ``path``. If ``str``, sets new column name. Default is ``False``.
+    path_converter : function or None, optional
+        A function that takes one argument and returns a string. Used to convert
+        paths in the ``path`` column, for instance, to strip a common prefix from
+        all the paths.
     $META
 
     Returns
@@ -191,24 +203,62 @@ def read_json(
             "input (orient='records', lines=True)."
         )
     storage_options = storage_options or {}
+    if include_path_column is True:
+        include_path_column = "path"
+
+    if path_converter is None:
+        path_converter = lambda x: x
+
     if blocksize:
-        first, chunks = read_bytes(
+        b_out = read_bytes(
             url_path,
             b"\n",
             blocksize=blocksize,
             sample=sample,
             compression=compression,
+            include_path=include_path_column,
             **storage_options,
         )
-        chunks = list(flatten(chunks))
+        if include_path_column:
+            first, chunks, paths = b_out
+            first_path = path_converter(paths[0])
+            path_dtype = pd.CategoricalDtype(path_converter(p) for p in paths)
+            flat_paths = flatten(
+                [path_converter(p)] * len(chunk) for p, chunk in zip(paths, chunks)
+            )
+        else:
+            first, chunks = b_out
+            first_path = None
+            flat_paths = (None,)
+            path_dtype = None
+
+        flat_chunks = flatten(chunks)
         if meta is None:
-            meta = read_json_chunk(first, encoding, errors, engine, kwargs)
+            meta = read_json_chunk(
+                first,
+                encoding,
+                errors,
+                engine,
+                include_path_column,
+                first_path,
+                path_dtype,
+                kwargs,
+            )
         meta = make_meta(meta)
         parts = [
-            delayed(read_json_chunk)(chunk, encoding, errors, engine, kwargs, meta=meta)
-            for chunk in chunks
+            delayed(read_json_chunk)(
+                chunk,
+                encoding,
+                errors,
+                engine,
+                include_path_column,
+                path,
+                path_dtype,
+                kwargs,
+                meta=meta,
+            )
+            for chunk, path in zip_longest(flat_chunks, flat_paths)
         ]
-        return from_delayed(parts, meta=meta)
     else:
         files = open_files(
             url_path,
@@ -218,22 +268,52 @@ def read_json(
             compression=compression,
             **storage_options,
         )
+        path_dtype = pd.CategoricalDtype(path_converter(f.path) for f in files)
         parts = [
-            delayed(read_json_file)(f, orient, lines, engine, kwargs) for f in files
+            delayed(read_json_file)(
+                f,
+                orient,
+                lines,
+                engine,
+                include_path_column,
+                path_converter(f.path),
+                path_dtype,
+                kwargs,
+            )
+            for f in files
         ]
-        return from_delayed(parts, meta=meta)
 
+    return from_delayed(parts, meta=meta)
 
-def read_json_chunk(chunk, encoding, errors, engine, kwargs, meta=None):
+
+def read_json_chunk(
+    chunk, encoding, errors, engine, column_name, path, path_dtype, kwargs, meta=None
+):
     s = io.StringIO(chunk.decode(encoding, errors))
     s.seek(0)
     df = engine(s, orient="records", lines=True, **kwargs)
     if meta is not None and df.empty:
         return meta
-    else:
-        return df
 
+    if column_name:
+        df = add_path_column(df, column_name, path, path_dtype)
 
-def read_json_file(f, orient, lines, engine, kwargs):
-    with f as f:
-        return engine(f, orient=orient, lines=lines, **kwargs)
+    return df
+
+
+def read_json_file(f, orient, lines, engine, column_name, path, path_dtype, kwargs):
+    with f as open_file:
+        df = engine(open_file, orient=orient, lines=lines, **kwargs)
+    if column_name:
+        df = add_path_column(df, column_name, path, path_dtype)
+    return df
+
+
+def add_path_column(df, column_name, path, dtype):
+    if column_name in df.columns:
+        raise ValueError(
+            f"Files already contain the column name: '{column_name}', so the path "
+            "column cannot use this name. Please set `include_path_column` to a "
+            "unique name."
+        )
+    return df.assign(**{column_name: pd.Series([path] * len(df), dtype=dtype)})
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/parquet/arrow.py 2022.02.0+dfsg-1/dask/dataframe/io/parquet/arrow.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/parquet/arrow.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/parquet/arrow.py	2022-02-11 16:21:12.000000000 +0000
@@ -2,7 +2,6 @@ import json
 import warnings
 from collections import defaultdict
 from datetime import datetime
-from functools import partial
 
 import numpy as np
 import pandas as pd
@@ -17,7 +16,12 @@ from ....core import flatten
 from ....delayed import Delayed
 from ....utils import getargspec, natural_sort_key
 from ...utils import clear_known_categories
-from ..utils import _get_pyarrow_dtypes, _meta_from_dtypes
+from ..utils import (
+    _get_pyarrow_dtypes,
+    _is_local_fs,
+    _meta_from_dtypes,
+    _open_input_files,
+)
 from .core import create_metadata_file
 from .utils import (
     Engine,
@@ -25,9 +29,11 @@ from .utils import (
     _get_aggregation_depth,
     _normalize_index_columns,
     _parse_pandas_metadata,
+    _process_open_file_options,
     _row_groups_to_parts,
     _set_metadata_task_size,
     _sort_and_analyze_paths,
+    _split_user_options,
 )
 
 # Check PyArrow version for feature support
@@ -208,17 +214,45 @@ def _read_table_from_path(
     are specified (otherwise fragments are converted directly
     into tables).
     """
+
+    # Define file-opening options
+    read_kwargs = kwargs.get("read", {}).copy()
+    precache_options, open_file_options = _process_open_file_options(
+        read_kwargs.pop("open_file_options", {}),
+        **(
+            {
+                "allow_precache": False,
+                "default_cache": "none",
+            }
+            if _is_local_fs(fs)
+            else {
+                "columns": columns,
+                "row_groups": row_groups if row_groups == [None] else [row_groups],
+                "default_engine": "pyarrow",
+                "default_cache": "none",
+            }
+        ),
+    )
+
     if partition_keys:
         tables = []
-        for rg in row_groups:
-            piece = pq.ParquetDatasetPiece(
-                path,
-                row_group=rg,
-                partition_keys=partition_keys,
-                open_file_func=partial(fs.open, mode="rb"),
-            )
-            arrow_table = piece_to_arrow_func(piece, columns, partitions, **kwargs)
-            tables.append(arrow_table)
+        with _open_input_files(
+            [path],
+            fs=fs,
+            precache_options=precache_options,
+            **open_file_options,
+        )[0] as fil:
+            for rg in row_groups:
+                piece = pq.ParquetDatasetPiece(
+                    path,
+                    row_group=rg,
+                    partition_keys=partition_keys,
+                    open_file_func=lambda _path, **_kwargs: fil,
+                )
+                arrow_table = piece_to_arrow_func(
+                    piece, columns, partitions, **read_kwargs
+                )
+                tables.append(arrow_table)
 
         if len(row_groups) > 1:
             # NOTE: Not covered by pytest
@@ -226,12 +260,18 @@ def _read_table_from_path(
         else:
             return tables[0]
     else:
-        with fs.open(path, mode="rb") as fil:
+        with _open_input_files(
+            [path],
+            fs=fs,
+            precache_options=precache_options,
+            **open_file_options,
+        )[0] as fil:
             if row_groups == [None]:
                 return pq.ParquetFile(fil).read(
                     columns=columns,
                     use_threads=False,
                     use_pandas_metadata=True,
+                    **read_kwargs,
                 )
             else:
                 return pq.ParquetFile(fil).read_row_groups(
@@ -239,6 +279,7 @@ def _read_table_from_path(
                     columns=columns,
                     use_threads=False,
                     use_pandas_metadata=True,
+                    **read_kwargs,
                 )
 
 
@@ -339,7 +380,7 @@ class ArrowDatasetEngine(Engine):
             aggregate_files,
             ignore_metadata_file,
             metadata_task_size,
-            **kwargs.get("dataset", {}),
+            kwargs,
         )
 
         # Stage 2: Generate output `meta`
@@ -721,8 +762,8 @@ class ArrowDatasetEngine(Engine):
         else:
             return []
 
-    @staticmethod
-    def write_metadata(parts, fmd, fs, path, append=False, **kwargs):
+    @classmethod
+    def write_metadata(cls, parts, meta, fs, path, append=False, **kwargs):
         schema = parts[0][0].get("schema", None)
         parts = [p for p in parts if p[0]["meta"] is not None]
         if parts:
@@ -736,8 +777,8 @@ class ArrowDatasetEngine(Engine):
 
             # Aggregate metadata and write to _metadata file
             metadata_path = fs.sep.join([path, "_metadata"])
-            if append and fmd is not None:
-                _meta = fmd
+            if append and meta is not None:
+                _meta = meta
                 i_start = 0
             else:
                 _meta = parts[0][0]["meta"]
@@ -765,8 +806,7 @@ class ArrowDatasetEngine(Engine):
         aggregate_files,
         ignore_metadata_file,
         metadata_task_size,
-        require_extension=(".parq", ".parquet"),
-        **dataset_kwargs,
+        kwargs,
     ):
         """pyarrow.dataset version of _collect_dataset_info
         Use pyarrow.dataset API to construct a dictionary of all
@@ -778,7 +818,13 @@ class ArrowDatasetEngine(Engine):
         # Use pyarrow.dataset API
         ds = None
         valid_paths = None  # Only used if `paths` is a list containing _metadata
-        _dataset_kwargs = dataset_kwargs.copy()
+
+        # Extract "supported" key-word arguments from `kwargs`
+        (
+            _dataset_kwargs,
+            read_kwargs,
+            user_kwargs,
+        ) = _split_user_options(**kwargs)
 
         # Discover Partitioning - Note that we need to avoid creating
         # this factory until it is actually used.  The `partitioning`
@@ -791,9 +837,10 @@ class ArrowDatasetEngine(Engine):
             {"obj": pa_ds.HivePartitioning},
         )
 
-        # Check that we are not silently ignoring any dataset_kwargs
-        if _dataset_kwargs:
-            raise ValueError(f"Unsupported dataset_kwargs: {_dataset_kwargs.keys()}")
+        # Set require_extension option
+        require_extension = _dataset_kwargs.pop(
+            "require_extension", (".parq", ".parquet")
+        )
 
         # Case-dependent pyarrow.dataset creation
         has_metadata_file = False
@@ -814,6 +861,7 @@ class ArrowDatasetEngine(Engine):
                         *partitioning.get("args", []),
                         **partitioning.get("kwargs", {}),
                     ),
+                    **_dataset_kwargs,
                 )
                 has_metadata_file = True
                 if gather_statistics is None:
@@ -845,6 +893,7 @@ class ArrowDatasetEngine(Engine):
                             *partitioning.get("args", []),
                             **partitioning.get("kwargs", {}),
                         ),
+                        **_dataset_kwargs,
                     )
                     has_metadata_file = True
                     if gather_statistics is None:
@@ -865,6 +914,7 @@ class ArrowDatasetEngine(Engine):
                     *partitioning.get("args", []),
                     **partitioning.get("kwargs", {}),
                 ),
+                **_dataset_kwargs,
             )
 
         # At this point, we know if `split_row_groups` should be
@@ -977,6 +1027,11 @@ class ArrowDatasetEngine(Engine):
             "partition_names": partition_names,
             "partitioning": partitioning,
             "metadata_task_size": metadata_task_size,
+            "kwargs": {
+                "dataset": _dataset_kwargs,
+                "read": read_kwargs,
+                **user_kwargs,
+            },
         }
 
     @classmethod
@@ -1127,6 +1182,7 @@ class ArrowDatasetEngine(Engine):
         categories = dataset_info["categories"]
         has_metadata_file = dataset_info["has_metadata_file"]
         valid_paths = dataset_info["valid_paths"]
+        kwargs = dataset_info["kwargs"]
 
         # Ensure metadata_task_size is set
         # (Using config file or defaults)
@@ -1177,6 +1233,7 @@ class ArrowDatasetEngine(Engine):
             "categories": categories,
             "filters": filters,
             "schema": schema,
+            **kwargs,
         }
 
         # Check if this is a very simple case where we can just return
@@ -1512,6 +1569,7 @@ class ArrowDatasetEngine(Engine):
                         *partitioning.get("args", []),
                         **partitioning.get("kwargs", {}),
                     ),
+                    **kwargs.get("dataset", {}),
                 )
                 frags = list(ds.get_fragments())
                 assert len(frags) == 1
@@ -1699,7 +1757,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
         aggregate_files,
         ignore_metadata_file,
         metadata_task_size,
-        **dataset_kwargs,
+        kwargs,
     ):
         """pyarrow-legacy version of _collect_dataset_info
         Use the ParquetDataset API to construct a dictionary of all
@@ -1714,6 +1772,13 @@ class ArrowLegacyEngine(ArrowDatasetEngi
         if metadata_task_size:
             raise ValueError("metadata_task_size not supported in ArrowLegacyEngine")
 
+        # Extract "supported" key-word arguments from `kwargs`
+        (
+            dataset_kwargs,
+            read_kwargs,
+            user_kwargs,
+        ) = _split_user_options(**kwargs)
+
         (
             schema,
             metadata,
@@ -1753,6 +1818,11 @@ class ArrowLegacyEngine(ArrowDatasetEngi
             "partition_keys": partition_info["partition_keys"],
             "partition_names": partition_info["partition_names"],
             "partitions": partition_info["partitions"],
+            "kwargs": {
+                "dataset": dataset_kwargs,
+                "read": read_kwargs,
+                **user_kwargs,
+            },
         }
 
     @classmethod
@@ -1780,6 +1850,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
             dataset_info["gather_statistics"],
             dataset_info["chunksize"],
             dataset_info["aggregation_depth"],
+            dataset_info["kwargs"],
         )
 
     @classmethod
@@ -1955,6 +2026,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
         gather_statistics,
         chunksize,
         aggregation_depth,
+        kwargs,
     ):
         """Construct ``parts`` for ddf construction
 
@@ -1980,7 +2052,11 @@ class ArrowLegacyEngine(ArrowDatasetEngi
             for full_path in metadata:
                 part = {"piece": (full_path, None, partition_keys.get(full_path, None))}
                 parts.append(part)
-            common_kwargs = {"partitions": partition_obj, "categories": categories}
+            common_kwargs = {
+                "partitions": partition_obj,
+                "categories": categories,
+                **kwargs,
+            }
             return parts, stats, common_kwargs
 
         # Use final metadata info to update our options for
@@ -2015,6 +2091,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
             fs,
             chunksize,
             aggregation_depth,
+            kwargs,
         )
 
     @classmethod
@@ -2221,6 +2298,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
         fs,
         chunksize,
         aggregation_depth,
+        kwargs,
     ):
         """Process row-groups and statistics.
 
@@ -2266,6 +2344,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
             "partitions": partition_info["partitions"],
             "categories": categories,
             "filters": filters,
+            **kwargs,
         }
 
         return parts, stats, common_kwargs
@@ -2285,7 +2364,7 @@ class ArrowLegacyEngine(ArrowDatasetEngi
     ):
         """Read in a pyarrow table.
 
-        This method is overrides the `ArrowLegacyEngine` implementation.
+        This method is overrides the `ArrowDatasetEngine` implementation.
         """
 
         return _read_table_from_path(
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/parquet/core.py 2022.02.0+dfsg-1/dask/dataframe/io/parquet/core.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/parquet/core.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/parquet/core.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,20 +1,23 @@
+from __future__ import annotations
+
 import math
 import warnings
 
 import tlz as toolz
 from fsspec.core import get_fs_token_paths
-from fsspec.implementations.local import LocalFileSystem
 from fsspec.utils import stringify_path
 from packaging.version import parse as parse_version
 
 from ....base import compute_as_if_collection, tokenize
+from ....blockwise import BlockIndex
 from ....delayed import Delayed
 from ....highlevelgraph import HighLevelGraph
 from ....layers import DataFrameIOLayer
 from ....utils import apply, import_required, natural_sort_key, parse_bytes
 from ...core import DataFrame, Scalar, new_dd_object
 from ...methods import concat
-from .utils import _sort_and_analyze_paths
+from ..utils import _is_local_fs
+from .utils import Engine, _sort_and_analyze_paths
 
 try:
     import snappy
@@ -64,8 +67,8 @@ class ParquetFunctionWrapper:
         self.common_kwargs = toolz.merge(common_kwargs, kwargs or {})
 
     def project_columns(self, columns):
-        """Return a new ParquetFunctionWrapper object with
-        a sub-column projection.
+        """Return a new ParquetFunctionWrapper object
+        with a sub-column projection.
         """
         if columns == self.columns:
             return self
@@ -95,6 +98,73 @@ class ParquetFunctionWrapper:
         )
 
 
+class ToParquetFunctionWrapper:
+    """
+    Parquet Function-Wrapper Class
+
+    Writes a DataFrame partition into a distinct parquet
+    file. When called, the function also requires the
+    current block index (via ``blockwise.BlockIndex``).
+    """
+
+    def __init__(
+        self,
+        engine,
+        path,
+        fs,
+        partition_on,
+        write_metadata_file,
+        i_offset,
+        name_function,
+        kwargs_pass,
+    ):
+        self.engine = engine
+        self.path = path
+        self.fs = fs
+        self.partition_on = partition_on
+        self.write_metadata_file = write_metadata_file
+        self.i_offset = i_offset
+        self.name_function = name_function
+        self.kwargs_pass = kwargs_pass
+
+        # NOTE: __name__ must be with "to-parquet"
+        # for the name of the resulting `Blockwise`
+        # layer to begin with "to-parquet"
+        self.__name__ = "to-parquet"
+
+    def __dask_tokenize__(self):
+        return (
+            self.engine,
+            self.path,
+            self.fs,
+            self.partition_on,
+            self.write_metadata_file,
+            self.i_offset,
+            self.name_function,
+            self.kwargs_pass,
+        )
+
+    def __call__(self, df, block_index: tuple[int]):
+        # Get partition index from block index tuple
+        part_i = block_index[0]
+        filename = (
+            f"part.{part_i + self.i_offset}.parquet"
+            if self.name_function is None
+            else self.name_function(part_i + self.i_offset)
+        )
+
+        # Write out data
+        return self.engine.write_partition(
+            df,
+            self.path,
+            self.fs,
+            filename,
+            self.partition_on,
+            self.write_metadata_file,
+            **(dict(self.kwargs_pass, head=True) if part_i == 0 else self.kwargs_pass),
+        )
+
+
 def read_parquet(
     path,
     columns=None,
@@ -162,6 +232,14 @@ def read_parquet(
         data written by dask/fastparquet, not otherwise.
     storage_options : dict, default None
         Key/value pairs to be passed on to the file-system backend, if any.
+    open_file_options : dict, default None
+        Key/value arguments to be passed along to ``AbstractFileSystem.open``
+        when each parquet data file is open for reading. Experimental
+        (optimized) "precaching" for remote file systems (e.g. S3, GCS) can
+        be enabled by adding ``{"method": "parquet"}`` under the
+        ``"precache_options"`` key. Also, a custom file-open function can be
+        used (instead of ``AbstractFileSystem.open``), by specifying the
+        desired function under the ``"open_file_func"`` key.
     engine : str, default 'auto'
         Parquet reader library to use. Options include: 'auto', 'fastparquet',
         'pyarrow', 'pyarrow-dataset', and 'pyarrow-legacy'. Defaults to 'auto',
@@ -239,9 +317,13 @@ def read_parquet(
         the second level corresponds to the kwargs that will be passed on to
         the underlying ``pyarrow`` or ``fastparquet`` function.
         Supported top-level keys: 'dataset' (for opening a ``pyarrow`` dataset),
-        'file' (for opening a ``fastparquet`` ``ParquetFile``), 'read' (for the
-        backend read function), 'arrow_to_pandas' (for controlling the arguments
-        passed to convert from a ``pyarrow.Table.to_pandas()``)
+        'file' or 'dataset' (for opening a ``fastparquet.ParquetFile``), 'read'
+        (for the backend read function), 'arrow_to_pandas' (for controlling the
+        arguments passed to convert from a ``pyarrow.Table.to_pandas()``).
+        Any element of kwargs that is not defined under these top-level keys
+        will be passed through to the `engine.read_partitions` classmethod as a
+        stand-alone argument (and will be ignored by the engine implementations
+        defined in ``dask.dataframe``).
 
     Examples
     --------
@@ -254,56 +336,49 @@ def read_parquet(
     """
 
     if "read_from_paths" in kwargs:
+        kwargs.pop("read_from_paths")
         warnings.warn(
             "`read_from_paths` is no longer supported and will be ignored.",
             FutureWarning,
         )
 
+    # Store initial function arguments
+    input_kwargs = {
+        "columns": columns,
+        "filters": filters,
+        "categories": categories,
+        "index": index,
+        "storage_options": storage_options,
+        "engine": engine,
+        "gather_statistics": gather_statistics,
+        "ignore_metadata_file": ignore_metadata_file,
+        "metadata_task_size": metadata_task_size,
+        "split_row_groups": split_row_groups,
+        "chunksize=": chunksize,
+        "aggregate_files": aggregate_files,
+        **kwargs,
+    }
+
     if isinstance(columns, str):
-        df = read_parquet(
-            path,
-            columns=[columns],
-            filters=filters,
-            categories=categories,
-            index=index,
-            storage_options=storage_options,
-            engine=engine,
-            gather_statistics=gather_statistics,
-            ignore_metadata_file=ignore_metadata_file,
-            split_row_groups=split_row_groups,
-            chunksize=chunksize,
-            aggregate_files=aggregate_files,
-            metadata_task_size=metadata_task_size,
-        )
+        input_kwargs["columns"] = [columns]
+        df = read_parquet(path, **input_kwargs)
         return df[columns]
 
     if columns is not None:
         columns = list(columns)
 
-    label = "read-parquet-"
-    output_name = label + tokenize(
-        path,
-        columns,
-        filters,
-        categories,
-        index,
-        storage_options,
-        engine,
-        gather_statistics,
-        ignore_metadata_file,
-        metadata_task_size,
-        split_row_groups,
-        chunksize,
-        aggregate_files,
-    )
-
     if isinstance(engine, str):
         engine = get_engine(engine)
 
     if hasattr(path, "name"):
         path = stringify_path(path)
-    fs, _, paths = get_fs_token_paths(path, mode="rb", storage_options=storage_options)
 
+    # Update input_kwargs and tokenize inputs
+    label = "read-parquet-"
+    input_kwargs.update({"columns": columns, "engine": engine})
+    output_name = label + tokenize(path, **input_kwargs)
+
+    fs, _, paths = get_fs_token_paths(path, mode="rb", storage_options=storage_options)
     paths = sorted(paths, key=natural_sort_key)  # numeric rather than glob ordering
 
     auto_index_allowed = False
@@ -394,10 +469,15 @@ def read_parquet(
                 meta,
                 columns,
                 index,
-                kwargs,
+                {},  # All kwargs should now be in `common_kwargs`
                 common_kwargs,
             ),
             label=label,
+            creation_info={
+                "func": read_parquet,
+                "args": (path,),
+                "kwargs": input_kwargs,
+            },
         )
         graph = HighLevelGraph({output_name: layer}, {output_name: set()})
 
@@ -600,7 +680,7 @@ def to_parquet(
     path = fs._strip_protocol(path)
 
     if overwrite:
-        if isinstance(fs, LocalFileSystem):
+        if _is_local_fs(fs):
             working_dir = fs.expand_path(".")[0]
             if path.rstrip("/") == working_dir.rstrip("/"):
                 raise ValueError(
@@ -701,81 +781,70 @@ def to_parquet(
         **kwargs_pass,
     )
 
-    # check name_function is valid
-    if name_function is not None and not callable(name_function):
-        raise ValueError("``name_function`` must be a callable with one argument.")
-
-    # Use i_offset and df.npartitions to define file-name list
-    filenames = [
-        f"part.{i + i_offset}.parquet"
-        if name_function is None
-        else name_function(i + i_offset)
-        for i in range(df.npartitions)
-    ]
+    # Check that custom name_function is valid,
+    # and that it will produce unique names
+    if name_function is not None:
+        if not callable(name_function):
+            raise ValueError("``name_function`` must be a callable with one argument.")
+        filenames = [name_function(i + i_offset) for i in range(df.npartitions)]
+        if len(set(filenames)) < len(filenames):
+            raise ValueError("``name_function`` must produce unique filenames.")
 
-    if name_function is not None and len(set(filenames)) < len(filenames):
-        raise ValueError("``name_function`` must produce unique filenames.")
-
-    # Construct IO graph
-    dsk = {}
-    name = "to-parquet-" + tokenize(
-        df,
-        fs,
-        path,
-        append,
-        ignore_divisions,
-        partition_on,
-        division_info,
-        index_cols,
-        schema,
-    )
-    part_tasks = []
+    # Create Blockwise layer for parquet-data write
     kwargs_pass["fmd"] = meta
     kwargs_pass["compression"] = compression
     kwargs_pass["index_cols"] = index_cols
     kwargs_pass["schema"] = schema
-    for d, filename in enumerate(filenames):
-        dsk[(name, d)] = (
-            apply,
-            engine.write_partition,
-            [
-                (df._name, d),
-                path,
-                fs,
-                filename,
-                partition_on,
-                write_metadata_file,
-            ],
-            toolz.merge(kwargs_pass, {"head": True}) if d == 0 else kwargs_pass,
-        )
-        part_tasks.append((name, d))
-
-    final_name = "metadata-" + name
-    # Collect metadata and write _metadata
+    data_write = df.map_partitions(
+        ToParquetFunctionWrapper(
+            engine,
+            path,
+            fs,
+            partition_on,
+            write_metadata_file,
+            i_offset,
+            name_function,
+            kwargs_pass,
+        ),
+        BlockIndex((df.npartitions,)),
+        # Pass in the original metadata to avoid
+        # metadata emulation in `map_partitions`.
+        # This is necessary, because we are not
+        # expecting a dataframe-like output.
+        meta=df._meta,
+        enforce_metadata=False,
+        transform_divisions=False,
+        align_dataframes=False,
+    )
 
+    # Collect metadata and write _metadata.
+    # TODO: Use tree-reduction layer (when available)
+    meta_name = "metadata-" + data_write._name
     if write_metadata_file:
-        dsk[(final_name, 0)] = (
-            apply,
-            engine.write_metadata,
-            [
-                part_tasks,
-                meta,
-                fs,
-                path,
-            ],
-            {"append": append, "compression": compression},
-        )
+        dsk = {
+            (meta_name, 0): (
+                apply,
+                engine.write_metadata,
+                [
+                    data_write.__dask_keys__(),
+                    meta,
+                    fs,
+                    path,
+                ],
+                {"append": append, "compression": compression},
+            )
+        }
     else:
-        dsk[(final_name, 0)] = (lambda x: None, part_tasks)
-
-    graph = HighLevelGraph.from_collections(final_name, dsk, dependencies=[df])
+        dsk = {(meta_name, 0): (lambda x: None, data_write.__dask_keys__())}
 
+    # Convert data_write + dsk to computable collection
+    graph = HighLevelGraph.from_collections(meta_name, dsk, dependencies=(data_write,))
     if compute:
         return compute_as_if_collection(
-            Scalar, graph, [(final_name, 0)], **compute_kwargs
+            Scalar, graph, [(meta_name, 0)], **compute_kwargs
         )
     else:
-        return Scalar(graph, final_name, "")
+        return Scalar(graph, meta_name, "")
 
 
 def create_metadata_file(
@@ -920,7 +989,7 @@ def create_metadata_file(
     return out
 
 
-_ENGINES = {}
+_ENGINES: dict[str, Engine] = {}
 
 
 def get_engine(engine):
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/parquet/fastparquet.py 2022.02.0+dfsg-1/dask/dataframe/io/parquet/fastparquet.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/parquet/fastparquet.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/parquet/fastparquet.py	2022-02-11 16:21:12.000000000 +0000
@@ -3,6 +3,7 @@ import pickle
 import threading
 import warnings
 from collections import OrderedDict, defaultdict
+from contextlib import ExitStack
 
 import numpy as np
 import pandas as pd
@@ -20,9 +21,8 @@ except ImportError:
 from ....base import tokenize
 from ....delayed import Delayed
 from ....utils import natural_sort_key
-from ...methods import concat
 from ...utils import UNKNOWN_CATEGORIES
-from ..utils import _meta_from_dtypes
+from ..utils import _is_local_fs, _meta_from_dtypes, _open_input_files
 
 #########################
 # Fastparquet interface #
@@ -33,9 +33,11 @@ from .utils import (
     _get_aggregation_depth,
     _normalize_index_columns,
     _parse_pandas_metadata,
+    _process_open_file_options,
     _row_groups_to_parts,
     _set_metadata_task_size,
     _sort_and_analyze_paths,
+    _split_user_options,
 )
 
 # Thread lock required to reset row-groups
@@ -370,8 +372,7 @@ class FastParquetEngine(Engine):
         aggregate_files,
         ignore_metadata_file,
         metadata_task_size,
-        require_extension=(".parq", ".parquet"),
-        **kwargs,
+        kwargs,
     ):
 
         # Define the parquet-file (pf) object to use for metadata,
@@ -384,8 +385,15 @@ class FastParquetEngine(Engine):
         # dataset.  If _metadata is available, set `gather_statistics=True`
         # (if `gather_statistics=None`).
 
+        # Extract "supported" key-word arguments from `kwargs`.
+        # Split items into `dataset_kwargs` and `read_kwargs`
+        dataset_kwargs, read_kwargs, user_kwargs = _split_user_options(**kwargs)
+
         parts = []
         _metadata_exists = False
+        require_extension = dataset_kwargs.pop(
+            "require_extension", (".parq", ".parquet")
+        )
         if len(paths) == 1 and fs.isdir(paths[0]):
 
             # This is a directory.
@@ -414,7 +422,7 @@ class FastParquetEngine(Engine):
                 pf = ParquetFile(
                     fs.sep.join([base, "_metadata"]),
                     open_with=fs.open,
-                    **kwargs,
+                    **dataset_kwargs,
                 )
                 if gather_statistics is None:
                     gather_statistics = True
@@ -431,7 +439,9 @@ class FastParquetEngine(Engine):
                             "No files satisfy the `require_extension` criteria "
                             f"(files must end with {require_extension})."
                         )
-                pf = ParquetFile(paths[:1], open_with=fs.open, root=base, **kwargs)
+                pf = ParquetFile(
+                    paths[:1], open_with=fs.open, root=base, **dataset_kwargs
+                )
                 scheme = get_file_scheme(fns)
                 pf.file_scheme = scheme
                 pf.cats = paths_to_cats(fns, scheme)
@@ -454,13 +464,15 @@ class FastParquetEngine(Engine):
                 pf = ParquetFile(
                     fs.sep.join([base, "_metadata"]),
                     open_with=fs.open,
-                    **kwargs,
+                    **dataset_kwargs,
                 )
             else:
                 # Rely on metadata for 0th file.
                 # Will need to pass a list of paths to read_partition
                 scheme = get_file_scheme(fns)
-                pf = ParquetFile(paths[:1], open_with=fs.open, root=base, **kwargs)
+                pf = ParquetFile(
+                    paths[:1], open_with=fs.open, root=base, **dataset_kwargs
+                )
                 pf.file_scheme = scheme
                 pf.cats = paths_to_cats(fns, scheme)
                 if not gather_statistics:
@@ -501,7 +513,11 @@ class FastParquetEngine(Engine):
             "aggregate_files": aggregate_files,
             "aggregation_depth": aggregation_depth,
             "metadata_task_size": metadata_task_size,
-            "kwargs": kwargs,
+            "kwargs": {
+                "dataset": dataset_kwargs,
+                "read": read_kwargs,
+                **user_kwargs,
+            },
         }
 
     @classmethod
@@ -611,6 +627,7 @@ class FastParquetEngine(Engine):
         categories_dict = dataset_info["categories_dict"]
         has_metadata_file = dataset_info["has_metadata_file"]
         metadata_task_size = dataset_info["metadata_task_size"]
+        kwargs = dataset_info["kwargs"]
 
         # Ensure metadata_task_size is set
         # (Using config file or defaults)
@@ -668,6 +685,7 @@ class FastParquetEngine(Engine):
             "root_cats": pf.cats,
             "root_file_scheme": pf.file_scheme,
             "base_path": base_path,
+            **kwargs,
         }
 
         # Check if this is a very simple case where we can just
@@ -853,11 +871,7 @@ class FastParquetEngine(Engine):
             aggregate_files,
             ignore_metadata_file,
             metadata_task_size,
-            **{
-                # Support "file" key for backward compat
-                **kwargs.get("file", {}),
-                **kwargs.get("dataset", {}),
-            },
+            kwargs,
         )
 
         # Stage 2: Generate output `meta`
@@ -943,7 +957,7 @@ class FastParquetEngine(Engine):
                     [p[0] for p in pieces],
                     open_with=fs.open,
                     root=base_path or False,
-                    **kwargs.get("file", {}),
+                    **kwargs.get("dataset", {}),
                 )
                 for piece in pieces:
                     _pf = (
@@ -953,7 +967,7 @@ class FastParquetEngine(Engine):
                             piece[0],
                             open_with=fs.open,
                             root=base_path or False,
-                            **kwargs.get("file", {}),
+                            **kwargs.get("dataset", {}),
                         )
                     )
                     n_local_row_groups = len(_pf.row_groups)
@@ -1009,35 +1023,113 @@ class FastParquetEngine(Engine):
                 lambda *args: parquet_file.dtypes
             )  # ugly patch, could be fixed
 
-            if set(columns).issubset(
-                parquet_file.columns + list(parquet_file.cats.keys())
-            ):
-                # Convert ParquetFile to pandas
-                return parquet_file.to_pandas(
-                    columns=columns,
-                    categories=categories,
-                    index=index,
-                )
-            else:
-                # Read necessary row-groups and concatenate
-                dfs = []
-                for row_group in row_groups:
-                    dfs.append(
-                        parquet_file.read_row_group_file(
-                            row_group,
-                            columns,
-                            categories,
-                            index=index,
-                            **kwargs.get("read", {}),
-                        )
-                    )
-                return concat(dfs, axis=0) if len(dfs) > 1 else dfs[0]
+            # Convert ParquetFile to pandas
+            return cls.pf_to_pandas(
+                parquet_file,
+                fs=fs,
+                columns=columns,
+                categories=categories,
+                index=index,
+                **kwargs.get("read", {}),
+            )
 
         else:
             # `sample` is NOT a tuple
             raise ValueError(f"Expected tuple, got {type(sample)}")
 
     @classmethod
+    def pf_to_pandas(
+        cls,
+        pf,
+        fs=None,
+        columns=None,
+        categories=None,
+        index=None,
+        open_file_options=None,
+        **kwargs,
+    ):
+        # This method was mostly copied from the fastparquet
+        # `ParquetFile.to_pandas` definition. We maintain our
+        # own implmentation in Dask to enable better remote
+        # file-handling control
+
+        # Handle selected columns
+        if columns is not None:
+            columns = columns[:]
+        else:
+            columns = pf.columns + list(pf.cats)
+        if index:
+            columns += [i for i in index if i not in columns]
+
+        # Extract row-groups and pre-allocate df
+        rgs = pf.row_groups
+        size = sum(rg.num_rows for rg in rgs)
+        df, views = pf.pre_allocate(size, columns, categories, index)
+        start = 0
+
+        # Get a map of file names -> row-groups
+        fn_rg_map = defaultdict(list)
+        for rg in rgs:
+            fn = pf.row_group_filename(rg)
+            fn_rg_map[fn].append(rg)
+
+        # Define file-opening options
+        precache_options, open_file_options = _process_open_file_options(
+            open_file_options,
+            **(
+                {
+                    "allow_precache": False,
+                    "default_cache": "readahead",
+                }
+                if _is_local_fs(fs)
+                else {
+                    "metadata": pf,
+                    "columns": list(set(columns).intersection(pf.columns)),
+                    "row_groups": [rgs for rgs in fn_rg_map.values()],
+                    "default_engine": "fastparquet",
+                    "default_cache": "readahead",
+                }
+            ),
+        )
+
+        with ExitStack() as stack:
+
+            for fn, infile in zip(
+                fn_rg_map.keys(),
+                _open_input_files(
+                    list(fn_rg_map.keys()),
+                    fs=fs,
+                    context_stack=stack,
+                    precache_options=precache_options,
+                    **open_file_options,
+                ),
+            ):
+                for rg in fn_rg_map[fn]:
+                    thislen = rg.num_rows
+                    parts = {
+                        name: (
+                            v
+                            if name.endswith("-catdef")
+                            else v[start : start + thislen]
+                        )
+                        for (name, v) in views.items()
+                    }
+
+                    # Add row-group data to df
+                    pf.read_row_group_file(
+                        rg,
+                        columns,
+                        categories,
+                        index,
+                        assign=parts,
+                        partition_meta=pf.partition_meta,
+                        infile=infile,
+                        **kwargs,
+                    )
+                    start += thislen
+        return df
+
+    @classmethod
     def initialize_write(
         cls,
         df,
@@ -1196,9 +1288,9 @@ class FastParquetEngine(Engine):
             return []
 
     @classmethod
-    def write_metadata(cls, parts, fmd, fs, path, append=False, **kwargs):
-        _meta = copy.copy(fmd)
-        rgs = fmd.row_groups
+    def write_metadata(cls, parts, meta, fs, path, append=False, **kwargs):
+        _meta = copy.copy(meta)
+        rgs = meta.row_groups
         if parts:
             for rg in parts:
                 if rg is not None:
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/parquet/utils.py 2022.02.0+dfsg-1/dask/dataframe/io/parquet/utils.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/parquet/utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/parquet/utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,11 +1,11 @@
 import re
 
 import pandas as pd
-from fsspec.implementations.local import LocalFileSystem
 
 from .... import config
 from ....core import flatten
 from ....utils import natural_sort_key
+from ..utils import _is_local_fs
 
 
 class Engine:
@@ -683,8 +683,63 @@ def _set_metadata_task_size(metadata_tas
         # If a default value is not specified in the config file,
         # otherwise we use "0"
         config_str = "dataframe.parquet.metadata-task-size-" + (
-            "local" if isinstance(fs, LocalFileSystem) else "remote"
+            "local" if _is_local_fs(fs) else "remote"
         )
         return config.get(config_str, 0)
 
     return metadata_task_size
+
+
+def _process_open_file_options(
+    open_file_options,
+    metadata=None,
+    columns=None,
+    row_groups=None,
+    default_engine=None,
+    default_cache="readahead",
+    allow_precache=True,
+):
+    # Process `open_file_options`.
+    # Set default values and extract `precache_options`
+    open_file_options = (open_file_options or {}).copy()
+    precache_options = open_file_options.pop("precache_options", {}).copy()
+    if not allow_precache:
+        # Precaching not allowed
+        # (probably because the file system is local)
+        precache_options = {}
+    if "open_file_func" not in open_file_options:
+        if precache_options.get("method", None) == "parquet":
+            open_file_options["cache_type"] = open_file_options.get(
+                "cache_type", "parts"
+            )
+            precache_options.update(
+                {
+                    "metadata": metadata,
+                    "columns": columns,
+                    "row_groups": row_groups,
+                    "engine": precache_options.get("engine", default_engine),
+                }
+            )
+        else:
+            open_file_options["cache_type"] = open_file_options.get(
+                "cache_type", default_cache
+            )
+            open_file_options["mode"] = open_file_options.get("mode", "rb")
+    return precache_options, open_file_options
+
+
+def _split_user_options(**kwargs):
+    # Check user-defined options.
+    # Split into "file" and "dataset"-specific kwargs
+    user_kwargs = kwargs.copy()
+    dataset_options = {
+        **user_kwargs.pop("file", {}).copy(),
+        **user_kwargs.pop("dataset", {}).copy(),
+    }
+    read_options = user_kwargs.pop("read", {}).copy()
+    read_options["open_file_options"] = user_kwargs.pop("open_file_options", {}).copy()
+    return (
+        dataset_options,
+        read_options,
+        user_kwargs,
+    )
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/sql.py 2022.02.0+dfsg-1/dask/dataframe/io/sql.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/sql.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/sql.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,3 +1,5 @@
+import warnings
+
 import numpy as np
 import pandas as pd
 
@@ -9,7 +11,453 @@ from .. import methods
 from .io import from_delayed, from_pandas
 
 
+def read_sql_query(
+    sql,
+    con,
+    index_col,
+    divisions=None,
+    npartitions=None,
+    limits=None,
+    bytes_per_chunk="256 MiB",
+    head_rows=5,
+    meta=None,
+    engine_kwargs=None,
+    **kwargs,
+):
+    """
+    Read SQL query into a DataFrame.
+
+    If neither ``divisions`` or ``npartitions`` is given, the memory footprint of the
+    first few rows will be determined, and partitions of size ~256MB will
+    be used.
+
+    Parameters
+    ----------
+    sql : SQLAlchemy Selectable
+        SQL query to be executed. TextClause is not supported
+    con : str
+        Full sqlalchemy URI for the database connection
+    index_col : str
+        Column which becomes the index, and defines the partitioning. Should
+        be a indexed column in the SQL server, and any orderable type. If the
+        type is number or time, then partition boundaries can be inferred from
+        ``npartitions`` or ``bytes_per_chunk``; otherwise must supply explicit
+        ``divisions``.
+    divisions: sequence
+        Values of the index column to split the table by. If given, this will
+        override ``npartitions`` and ``bytes_per_chunk``. The divisions are the value
+        boundaries of the index column used to define the partitions. For
+        example, ``divisions=list('acegikmoqsuwz')`` could be used to partition
+        a string column lexographically into 12 partitions, with the implicit
+        assumption that each partition contains similar numbers of records.
+    npartitions : int
+        Number of partitions, if ``divisions`` is not given. Will split the values
+        of the index column linearly between ``limits``, if given, or the column
+        max/min. The index column must be numeric or time for this to work
+    limits: 2-tuple or None
+        Manually give upper and lower range of values for use with ``npartitions``;
+        if None, first fetches max/min from the DB. Upper limit, if
+        given, is inclusive.
+    bytes_per_chunk : str or int
+        If both ``divisions`` and ``npartitions`` is None, this is the target size of
+        each partition, in bytes
+    head_rows : int
+        How many rows to load for inferring the data-types, and memory per row
+    meta : empty DataFrame or None
+        If provided, do not attempt to infer dtypes, but use these, coercing
+        all chunks on load
+    engine_kwargs : dict or None
+        Specific db engine parameters for sqlalchemy
+    kwargs : dict
+        Additional parameters to pass to `pd.read_sql()`
+
+    Returns
+    -------
+    dask.dataframe
+
+    See Also
+    --------
+    read_sql_table : Read SQL database table into a DataFrame.
+    """
+    import sqlalchemy as sa
+
+    if not isinstance(con, str):
+        raise TypeError(
+            "'con' must be of type str, not "
+            + str(type(con))
+            + "Note: Dask does not support SQLAlchemy connectables here"
+        )
+    if index_col is None:
+        raise ValueError("Must specify index column to partition on")
+    if not isinstance(index_col, (str, sa.Column, sa.sql.elements.ColumnClause)):
+        raise ValueError(
+            "'index_col' must be of type str or sa.Column, not " + str(type(index_col))
+        )
+    if not head_rows > 0:
+        if meta is None:
+            raise ValueError("Must provide 'meta' if 'head_rows' is 0")
+        if divisions is None and npartitions is None:
+            raise ValueError(
+                "Must provide 'divisions' or 'npartitions' if 'head_rows' is 0"
+            )
+    if divisions and npartitions:
+        raise TypeError("Must supply either 'divisions' or 'npartitions', not both")
+
+    engine_kwargs = {} if engine_kwargs is None else engine_kwargs
+    engine = sa.create_engine(con, **engine_kwargs)
+
+    index = (
+        sa.Column(index_col)
+        if isinstance(index_col, str)
+        else sa.Column(index_col.name, index_col.type)
+    )
+
+    kwargs["index_col"] = index.name
+
+    if head_rows > 0:
+        # derive metadata from first few rows
+        q = sql.limit(head_rows)
+        head = pd.read_sql(q, engine, **kwargs)
+
+        if len(head) == 0:
+            # no results at all
+            return from_pandas(head, npartitions=1)
+
+        bytes_per_row = (head.memory_usage(deep=True, index=True)).sum() / head_rows
+        if meta is None:
+            meta = head.iloc[:0]
+
+    if divisions is None:
+        if limits is None:
+            # calculate max and min for given index
+            q = sa.sql.select(
+                [sa.sql.func.max(index), sa.sql.func.min(index)]
+            ).select_from(sql.subquery())
+            minmax = pd.read_sql(q, engine)
+            maxi, mini = minmax.iloc[0]
+            dtype = minmax.dtypes["max_1"]
+        else:
+            mini, maxi = limits
+            dtype = pd.Series(limits).dtype
+
+        if npartitions is None:
+            q = sa.sql.select([sa.sql.func.count(index)]).select_from(sql.subquery())
+            count = pd.read_sql(q, engine)["count_1"][0]
+            npartitions = (
+                int(
+                    round(
+                        count * bytes_per_row / dask.utils.parse_bytes(bytes_per_chunk)
+                    )
+                )
+                or 1
+            )
+        if dtype.kind == "M":
+            divisions = methods.tolist(
+                pd.date_range(
+                    start=mini,
+                    end=maxi,
+                    freq="%iS" % ((maxi - mini).total_seconds() / npartitions),
+                )
+            )
+            divisions[0] = mini
+            divisions[-1] = maxi
+        elif dtype.kind in ["i", "u", "f"]:
+            divisions = np.linspace(mini, maxi, npartitions + 1).tolist()
+        else:
+            raise TypeError(
+                'Provided index column is of type "{}".  If divisions is not provided the '
+                "index column type must be numeric or datetime.".format(dtype)
+            )
+
+    parts = []
+    lowers, uppers = divisions[:-1], divisions[1:]
+    for i, (lower, upper) in enumerate(zip(lowers, uppers)):
+        cond = index <= upper if i == len(lowers) - 1 else index < upper
+        q = sql.where(sa.sql.and_(index >= lower, cond))
+        parts.append(
+            delayed(_read_sql_chunk)(
+                q, con, meta, engine_kwargs=engine_kwargs, **kwargs
+            )
+        )
+
+    engine.dispose()
+
+    return from_delayed(parts, meta, divisions=divisions)
+
+
 def read_sql_table(
+    table_name,
+    con,
+    index_col,
+    divisions=None,
+    npartitions=None,
+    limits=None,
+    columns=None,
+    bytes_per_chunk="256 MiB",
+    head_rows=5,
+    schema=None,
+    meta=None,
+    engine_kwargs=None,
+    **kwargs,
+):
+    """
+    Read SQL database table into a DataFrame.
+
+    If neither ``divisions`` or ``npartitions`` is given, the memory footprint of the
+    first few rows will be determined, and partitions of size ~256MB will
+    be used.
+
+    Parameters
+    ----------
+    table_name : str
+        Name of SQL table in database.
+    con : str
+        Full sqlalchemy URI for the database connection
+    index_col : str
+        Column which becomes the index, and defines the partitioning. Should
+        be a indexed column in the SQL server, and any orderable type. If the
+        type is number or time, then partition boundaries can be inferred from
+        ``npartitions`` or ``bytes_per_chunk``; otherwise must supply explicit
+        ``divisions``.
+    columns : sequence of str or SqlAlchemy column or None
+        Which columns to select; if None, gets all. Note can be a mix of str and SqlAlchemy columns
+    schema : str or None
+        Pass this to sqlalchemy to select which DB schema to use within the
+        URI connection
+    divisions: sequence
+        Values of the index column to split the table by. If given, this will
+        override ``npartitions`` and ``bytes_per_chunk``. The divisions are the value
+        boundaries of the index column used to define the partitions. For
+        example, ``divisions=list('acegikmoqsuwz')`` could be used to partition
+        a string column lexographically into 12 partitions, with the implicit
+        assumption that each partition contains similar numbers of records.
+    npartitions : int
+        Number of partitions, if ``divisions`` is not given. Will split the values
+        of the index column linearly between ``limits``, if given, or the column
+        max/min. The index column must be numeric or time for this to work
+    limits: 2-tuple or None
+        Manually give upper and lower range of values for use with ``npartitions``;
+        if None, first fetches max/min from the DB. Upper limit, if
+        given, is inclusive.
+    bytes_per_chunk : str or int
+        If both ``divisions`` and ``npartitions`` is None, this is the target size of
+        each partition, in bytes
+    head_rows : int
+        How many rows to load for inferring the data-types, and memory per row
+    meta : empty DataFrame or None
+        If provided, do not attempt to infer dtypes, but use these, coercing
+        all chunks on load
+    engine_kwargs : dict or None
+        Specific db engine parameters for sqlalchemy
+    kwargs : dict
+        Additional parameters to pass to `pd.read_sql()`
+
+    Returns
+    -------
+    dask.dataframe
+
+    See Also
+    --------
+    read_sql_query : Read SQL query into a DataFrame.
+
+    Examples
+    --------
+    >>> df = dd.read_sql_table('accounts', 'sqlite:///path/to/bank.db',
+    ...                  npartitions=10, index_col='id')  # doctest: +SKIP
+    """
+    import sqlalchemy as sa
+    from sqlalchemy import sql
+
+    if "table" in kwargs:
+        warnings.warn(
+            "The `table` keyword has been replaced by `table_name`. Please use `table_name` instead.",
+            DeprecationWarning,
+        )
+        table_name = kwargs.pop("table")
+    if "uri" in kwargs:
+        warnings.warn(
+            "The `uri` keyword has been replaced by `con`. Please use `con` instead.",
+            DeprecationWarning,
+        )
+        con = kwargs.pop("uri")
+    deprecated_args = False
+    if not isinstance(table_name, str):
+        warnings.warn(
+            "`read_sql_table` will no longer support {}; please use a `table_name` of type str instead "
+            "or use `read_sql_query`, if you are using a SQLAlchemy query".format(
+                type(table_name)
+            ),
+            DeprecationWarning,
+        )
+        deprecated_args = True
+    if columns is not None:
+        for col in columns:
+            if not isinstance(col, (sa.Column, str)):
+                warnings.warn(
+                    "`columns` will no longer support SQLAlchemy selectables; please use `read_sql_query` "
+                    "instead",
+                    DeprecationWarning,
+                )
+                deprecated_args = True
+
+    if not _gt14():
+        warnings.warn(
+            "Dask will soon require SQLAlchemy 1.4 or newer. "
+            "Please update your SQLAlchemy version. "
+            "Friendly note: Upgrading to SQLAlchemy 1.4 may brake code. Do it with caution. ",
+            category=DeprecationWarning,
+        )
+    if deprecated_args or not _gt14():
+        return _old_read_sql_table(
+            table=table_name,
+            uri=con,
+            index_col=index_col,
+            divisions=divisions,
+            npartitions=npartitions,
+            limits=limits,
+            columns=columns,
+            bytes_per_chunk=bytes_per_chunk,
+            head_rows=head_rows,
+            schema=schema,
+            meta=meta,
+            engine_kwargs=engine_kwargs,
+            **kwargs,
+        )
+
+    if not isinstance(con, str):
+        raise TypeError(
+            "`con` must be of type str, not "
+            + str(type(con))
+            + "Note: Dask does not support SQLAlchemy connectables here"
+        )
+
+    engine_kwargs = {} if engine_kwargs is None else engine_kwargs
+    engine = sa.create_engine(con, **engine_kwargs)
+    m = sa.MetaData()
+    if isinstance(table_name, str):
+        table_name = sa.Table(
+            table_name, m, autoload=True, autoload_with=engine, schema=schema
+        )
+    else:
+        raise TypeError(
+            "`table_name` must be of type str, not " + str(type(table_name))
+        )
+    engine.dispose()
+
+    columns = (
+        [
+            (
+                sa.Column(c, table_name.columns[c].type)
+                if isinstance(c, str)
+                else sa.Column(c.name, c.type)
+            )
+            for c in columns
+        ]
+        if columns
+        else [sa.Column(c.name, c.type) for c in table_name.columns]
+    )
+    index = (
+        sa.Column(index_col, table_name.columns[index_col].type)
+        if isinstance(index_col, str)
+        else sa.Column(index_col.name, index_col.type)
+    )
+
+    if index.name not in [c.name for c in columns]:
+        columns.append(index)
+
+    query = sql.select(columns).select_from(table_name)
+
+    return read_sql_query(
+        sql=query,
+        con=con,
+        index_col=index,
+        divisions=divisions,
+        npartitions=npartitions,
+        limits=limits,
+        bytes_per_chunk=bytes_per_chunk,
+        head_rows=head_rows,
+        meta=meta,
+        engine_kwargs=engine_kwargs,
+        **kwargs,
+    )
+
+
+def read_sql(sql, con, index_col, **kwargs):
+    """
+    Read SQL query or database table into a DataFrame.
+
+    This function is a convenience wrapper around ``read_sql_table`` and
+    ``read_sql_query``. It will delegate to the specific function depending
+    on the provided input. A SQL query will be routed to ``read_sql_query``,
+    while a database table name will be routed to ``read_sql_table``.
+    Note that the delegated function might have more specific notes about
+    their functionality not listed here.
+
+    Parameters
+    ----------
+    sql : str or SQLAlchemy Selectable
+        Name of SQL table in database or SQL query to be executed. TextClause is not supported
+    con : str
+        Full sqlalchemy URI for the database connection
+    index_col : str
+        Column which becomes the index, and defines the partitioning. Should
+        be a indexed column in the SQL server, and any orderable type. If the
+        type is number or time, then partition boundaries can be inferred from
+        ``npartitions`` or ``bytes_per_chunk``; otherwise must supply explicit
+        ``divisions``.
+
+    Returns
+    -------
+    dask.dataframe
+
+    See Also
+    --------
+    read_sql_table : Read SQL database table into a DataFrame.
+    read_sql_query : Read SQL query into a DataFrame.
+    """
+    if isinstance(sql, str):
+        return read_sql_table(sql, con, index_col, **kwargs)
+    else:
+        return read_sql_query(sql, con, index_col, **kwargs)
+
+
+def _read_sql_chunk(q, uri, meta, engine_kwargs=None, **kwargs):
+    import sqlalchemy as sa
+
+    engine_kwargs = engine_kwargs or {}
+    engine = sa.create_engine(uri, **engine_kwargs)
+    df = pd.read_sql(q, engine, **kwargs)
+    engine.dispose()
+    if len(df) == 0:
+        return meta
+    elif len(meta.dtypes.to_dict()) == 0:
+        # only index column in loaded
+        # required only for pandas < 1.0.0
+        return df
+    else:
+        return df.astype(meta.dtypes.to_dict(), copy=False)
+
+
+def _gt14() -> bool:
+    """
+    Check if sqlalchemy.__version__ is at least 1.4.0, when several
+    deprecations were made.
+    """
+    import sqlalchemy
+
+    if (
+        sqlalchemy.__version__.startswith("0.")
+        or sqlalchemy.__version__.startswith("1.0")
+        or sqlalchemy.__version__.startswith("1.1")
+        or sqlalchemy.__version__.startswith("1.2")
+        or sqlalchemy.__version__.startswith("1.3")
+    ):
+        return False
+    else:
+        return True
+
+
+def _old_read_sql_table(
     table,
     uri,
     index_col,
@@ -26,11 +474,9 @@ def read_sql_table(
 ):
     """
     Create dataframe from an SQL table.
-
     If neither divisions or npartitions is given, the memory footprint of the
     first few rows will be determined, and partitions of size ~256MB will
     be used.
-
     Parameters
     ----------
     table : string or sqlalchemy expression
@@ -48,9 +494,7 @@ def read_sql_table(
         ``index_col=sql.func.abs(sql.column("value")).label("abs(value)")``, or
         ``index_col=cast(sql.column("id"),types.BigInteger).label("id")`` to convert
         the textfield ``id`` to ``BigInteger``.
-
         Note ``sql``, ``cast``, ``types`` methods comes from ``sqlalchemy`` module.
-
         Labeling columns created by functions or arithmetic operations is
         required.
     divisions: sequence
@@ -89,11 +533,9 @@ def read_sql_table(
         Specific db engine parameters for sqlalchemy
     kwargs : dict
         Additional parameters to pass to `pd.read_sql()`
-
     Returns
     -------
     dask.dataframe
-
     Examples
     --------
     >>> df = dd.read_sql_table('accounts', 'sqlite:///path/to/bank.db',
@@ -103,6 +545,14 @@ def read_sql_table(
     from sqlalchemy import sql
     from sqlalchemy.sql import elements
 
+    warnings.warn(
+        "You are using a compatibility version of `read_sql_table` that will be "
+        "removed in a future version of dask. This function existst to support "
+        "old versions of SQLAlchemy (< 1.4). This compatibility function is less "
+        "stable than the new version. We recommend you update your code.",
+        DeprecationWarning,
+    )
+
     if index_col is None:
         raise ValueError("Must specify index column to partition on")
 
@@ -215,23 +665,6 @@ def read_sql_table(
     return from_delayed(parts, meta, divisions=divisions)
 
 
-def _read_sql_chunk(q, uri, meta, engine_kwargs=None, **kwargs):
-    import sqlalchemy as sa
-
-    engine_kwargs = engine_kwargs or {}
-    engine = sa.create_engine(uri, **engine_kwargs)
-    df = pd.read_sql(q, engine, **kwargs)
-    engine.dispose()
-    if len(df) == 0:
-        return meta
-    elif len(meta.dtypes.to_dict()) == 0:
-        # only index column in loaded
-        # required only for pandas < 1.0.0
-        return df
-    else:
-        return df.astype(meta.dtypes.to_dict(), copy=False)
-
-
 def to_sql(
     df,
     name: str,
@@ -245,6 +678,7 @@ def to_sql(
     method=None,
     compute=True,
     parallel=False,
+    engine_kwargs=None,
 ):
     """Store Dask Dataframe to a SQL table
 
@@ -301,6 +735,8 @@ def to_sql(
         When true, have each block append itself to the DB table concurrently. This can result in DB rows being in a
         different order than the source DataFrame's corresponding rows. When false, load each block into the SQL DB in
         sequence.
+    engine_kwargs : dict or None
+        Specific db engine parameters for sqlalchemy
 
     Raises
     ------
@@ -353,13 +789,18 @@ def to_sql(
     >>> result
     [(0, 0, '00'), (1, 1, '11'), (2, 2, '22'), (3, 3, '33')]
     """
+    import sqlalchemy as sa
+
     if not isinstance(uri, str):
         raise ValueError(f"Expected URI to be a string, got {type(uri)}.")
 
+    engine_kwargs = {} if engine_kwargs is None else engine_kwargs
+    engine = sa.create_engine(uri, **engine_kwargs)
+
     # This is the only argument we add on top of what Pandas supports
     kwargs = dict(
         name=name,
-        con=uri,
+        con=engine,
         schema=schema,
         if_exists=if_exists,
         index=index,
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_csv.py 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_csv.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_csv.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_csv.py	2022-02-11 16:21:12.000000000 +0000
@@ -12,7 +12,6 @@ from fsspec.compression import compr
 from tlz import partition_all, valmap
 
 import dask
-import dask.dataframe as dd
 from dask.base import compute_as_if_collection
 from dask.bytes.core import read_bytes
 from dask.bytes.utils import compress
@@ -1305,13 +1304,20 @@ def test_to_csv():
 
         with tmpdir() as dn:
             r = a.to_csv(dn, index=False, compute=False)
-            dask.compute(*r, scheduler="sync")
+            paths = dask.compute(*r, scheduler="sync")
+            # this is a tuple rather than a list since it's the output of dask.compute
+            assert paths == tuple(
+                os.path.join(dn, f"{n}.part") for n in range(npartitions)
+            )
             result = dd.read_csv(os.path.join(dn, "*")).compute().reset_index(drop=True)
             assert_eq(result, df)
 
         with tmpdir() as dn:
             fn = os.path.join(dn, "data_*.csv")
-            a.to_csv(fn, index=False)
+            paths = a.to_csv(fn, index=False)
+            assert paths == [
+                os.path.join(dn, f"data_{n}.csv") for n in range(npartitions)
+            ]
             result = dd.read_csv(fn).compute().reset_index(drop=True)
             assert_eq(result, df)
 
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_hdf.py 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_hdf.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_hdf.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_hdf.py	2022-02-11 16:21:12.000000000 +0000
@@ -241,7 +241,7 @@ def test_to_hdf_modes_multiple_nodes():
         a.to_hdf(fn, "/data2")
         a.to_hdf(fn, "/data*", mode="a")
         out = dd.read_hdf(fn, "/data*")
-        assert_eq(df.append(df), out)
+        assert_eq(dd.concat([df, df]), out)
 
     # overwriting a file with a single partition
     a = dd.from_pandas(df, 1)
@@ -257,7 +257,7 @@ def test_to_hdf_modes_multiple_nodes():
         a.to_hdf(fn, "/data2")
         a.to_hdf(fn, "/data*", mode="a")
         out = dd.read_hdf(fn, "/data*")
-        assert_eq(df.append(df), out)
+        assert_eq(dd.concat([df, df]), out)
 
     # overwriting a file with two partitions
     a = dd.from_pandas(df, 2)
@@ -274,7 +274,7 @@ def test_to_hdf_modes_multiple_nodes():
         a.to_hdf(fn, "/data2")
         a.to_hdf(fn, "/data*", mode="a", append=False)
         out = dd.read_hdf(fn, "/data*")
-        assert_eq(df.append(df), out)
+        assert_eq(dd.concat([df, df]), out)
 
 
 def test_to_hdf_modes_multiple_files():
@@ -290,7 +290,7 @@ def test_to_hdf_modes_multiple_files():
         a.to_hdf(os.path.join(dn, "data2"), "/data")
         a.to_hdf(fn, "/data", mode="a")
         out = dd.read_hdf(fn, "/data*")
-        assert_eq(df.append(df), out)
+        assert_eq(dd.concat([df, df]), out)
 
     # appending two partitions to existing data
     a = dd.from_pandas(df, 2)
@@ -299,7 +299,7 @@ def test_to_hdf_modes_multiple_files():
         a.to_hdf(os.path.join(dn, "data2"), "/data")
         a.to_hdf(fn, "/data", mode="a")
         out = dd.read_hdf(fn, "/data")
-        assert_eq(df.append(df), out)
+        assert_eq(dd.concat([df, df]), out)
 
     # overwriting a file with two partitions
     a = dd.from_pandas(df, 2)
@@ -317,7 +317,7 @@ def test_to_hdf_modes_multiple_files():
         a.to_hdf(os.path.join(dn, "data1"), "/data")
         a.to_hdf(fn, "/data", mode="a", append=False)
         out = dd.read_hdf(fn, "/data")
-        assert_eq(df.append(df), out)
+        assert_eq(dd.concat([df, df]), out)
 
 
 def test_to_hdf_link_optimizations():
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_json.py 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_json.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_json.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_json.py	2022-02-11 16:21:12.000000000 +0000
@@ -4,6 +4,7 @@ import os
 import pandas as pd
 import pytest
 
+import dask
 import dask.dataframe as dd
 from dask.dataframe.utils import assert_eq
 from dask.utils import tmpdir, tmpfile
@@ -13,17 +14,96 @@ ddf = dd.from_pandas(df, npartitions=2)
 
 
 @pytest.mark.parametrize("orient", ["split", "records", "index", "columns", "values"])
+def test_read_json_with_path_column(orient):
+    with tmpfile("json") as f:
+        df.to_json(f, orient=orient, lines=False)
+        actual = dd.read_json(f, orient=orient, lines=False, include_path_column=True)
+        actual_pd = pd.read_json(f, orient=orient, lines=False)
+        # The default column name when include_path_colum is True is "path"
+        # The paths on Windows are converted to forward slash somewhere in the file
+        # reading chain in Dask, so we have to do the same here.
+        actual_pd["path"] = pd.Series(
+            (f.replace(os.sep, "/"),) * len(actual_pd), dtype="category"
+        )
+        assert actual.path.dtype == "category"
+        assert_eq(actual, actual_pd)
+
+
+def test_read_json_path_column_with_duplicate_name_is_error():
+    with tmpfile("json") as f:
+        df.to_json(f, orient="records", lines=False)
+        with pytest.raises(ValueError, match="Files already contain"):
+            dd.read_json(f, orient="records", lines=False, include_path_column="x")
+
+
+def test_read_json_with_path_converter():
+    path_column_name = "filenames"
+
+    def path_converter(x):
+        return "asdf.json"
+
+    with tmpfile("json") as f:
+        df.to_json(f, orient="records", lines=False)
+        actual = dd.read_json(
+            f,
+            orient="records",
+            lines=False,
+            include_path_column=path_column_name,
+            path_converter=path_converter,
+        )
+        actual_pd = pd.read_json(f, orient="records", lines=False)
+        actual_pd[path_column_name] = pd.Series(
+            (path_converter(f),) * len(actual_pd), dtype="category"
+        )
+        assert_eq(actual, actual_pd)
+
+
+def test_read_orient_not_records_and_lines():
+    with pytest.raises(ValueError, match="Line-delimited JSON"):
+        dd.read_json("nofile.json", orient="split", lines=True)
+
+
+def test_write_orient_not_records_and_lines():
+    with tmpfile("json") as f:
+        with pytest.raises(ValueError, match="Line-delimited JSON"):
+            dd.to_json(ddf, f, orient="split", lines=True)
+
+
+@pytest.mark.parametrize("blocksize", [5, 15, 33, 200, 90000])
+def test_read_json_multiple_files_with_path_column(blocksize, tmpdir):
+    fil1 = str(tmpdir.join("fil1.json")).replace(os.sep, "/")
+    fil2 = str(tmpdir.join("fil2.json")).replace(os.sep, "/")
+    df = pd.DataFrame({"x": range(5), "y": ["a", "b", "c", "d", "e"]})
+    df2 = df.assign(x=df.x + 0.5)
+    orient = "records"
+    lines = True
+    df.to_json(fil1, orient=orient, lines=lines)
+    df2.to_json(fil2, orient=orient, lines=lines)
+    path_dtype = pd.CategoricalDtype((fil1, fil2))
+    df["path"] = pd.Series((fil1,) * len(df), dtype=path_dtype)
+    df2["path"] = pd.Series((fil2,) * len(df2), dtype=path_dtype)
+    sol = pd.concat([df, df2])
+    res = dd.read_json(
+        str(tmpdir.join("fil*.json")),
+        orient=orient,
+        lines=lines,
+        include_path_column=True,
+        blocksize=blocksize,
+    )
+    assert_eq(res, sol, check_index=False)
+
+
+@pytest.mark.parametrize("orient", ["split", "records", "index", "columns", "values"])
 def test_read_json_basic(orient):
     with tmpfile("json") as f:
         df.to_json(f, orient=orient, lines=False)
         actual = dd.read_json(f, orient=orient, lines=False)
         actual_pd = pd.read_json(f, orient=orient, lines=False)
 
-        out = actual.compute()
-        assert_eq(out, actual_pd)
+        assert_eq(actual, actual_pd)
         if orient == "values":
-            out.columns = list(df.columns)
-        assert_eq(out, df)
+            actual.columns = list(df.columns)
+        assert_eq(actual, df)
 
 
 @pytest.mark.parametrize("fkeyword", ["pandas", "json"])
@@ -77,10 +157,9 @@ def test_write_json_basic(orient):
         fn = os.path.join(path, "1.json")
         df.to_json(fn, orient=orient, lines=False)
         actual = dd.read_json(fn, orient=orient, lines=False)
-        out = actual.compute()
         if orient == "values":
-            out.columns = list(df.columns)
-        assert_eq(out, df)
+            actual.columns = list(df.columns)
+        assert_eq(actual, df)
 
 
 def test_to_json_with_get():
@@ -126,7 +205,7 @@ def test_json_compressed(compression):
     with tmpdir() as path:
         dd.to_json(ddf, path, compression=compression)
         actual = dd.read_json(os.path.join(path, "*"), compression=compression)
-        assert_eq(df, actual.compute(), check_index=False)
+        assert_eq(df, actual, check_index=False)
 
 
 def test_read_json_inferred_compression():
@@ -134,4 +213,18 @@ def test_read_json_inferred_compression(
         fn = os.path.join(path, "*.json.gz")
         dd.to_json(ddf, fn, compression="gzip")
         actual = dd.read_json(fn)
-        assert_eq(df, actual.compute(), check_index=False)
+        assert_eq(df, actual, check_index=False)
+
+
+def test_to_json_results():
+    with tmpfile("json") as f:
+        paths = ddf.to_json(f)
+        assert paths == [os.path.join(f, f"{n}.part") for n in range(ddf.npartitions)]
+
+    with tmpfile("json") as f:
+        list_of_delayed = ddf.to_json(f, compute=False)
+        paths = dask.compute(*list_of_delayed)
+        # this is a tuple rather than a list since it's the output of dask.compute
+        assert paths == tuple(
+            os.path.join(f, f"{n}.part") for n in range(ddf.npartitions)
+        )
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_parquet.py 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_parquet.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_parquet.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_parquet.py	2022-02-11 16:21:12.000000000 +0000
@@ -20,6 +20,7 @@ from dask.dataframe.optimize import opti
 from dask.dataframe.utils import assert_eq
 from dask.layers import DataFrameIOLayer
 from dask.utils import natural_sort_key
+from dask.utils_test import hlg_layer
 
 try:
     import fastparquet
@@ -1639,7 +1640,7 @@ def test_to_parquet_lazy(tmpdir, schedul
 @FASTPARQUET_MARK
 def test_timestamp96(tmpdir):
     fn = str(tmpdir)
-    df = pd.DataFrame({"a": ["now"]}, dtype="M8[ns]")
+    df = pd.DataFrame({"a": [pd.to_datetime("now", utc=True)]})
     ddf = dd.from_pandas(df, 1)
     ddf.to_parquet(fn, write_index=False, times="int96")
     pf = fastparquet.ParquetFile(fn)
@@ -2460,11 +2461,13 @@ def test_getitem_optimization(tmpdir, en
     out = ddf.to_frame().to_parquet(tmp_path_wt, engine=engine, compute=False)
     dsk = optimize_dataframe_getitem(out.dask, keys=[out.key])
 
-    read = [key for key in dsk.layers if key.startswith("read-parquet")][0]
-    subgraph = dsk.layers[read]
-    assert isinstance(subgraph, DataFrameIOLayer)
-    assert subgraph.columns == ["B"]
-    assert next(iter(subgraph.dsk.values()))[0].columns == ["B"]
+    subgraph_rd = hlg_layer(dsk, "read-parquet")
+    assert isinstance(subgraph_rd, DataFrameIOLayer)
+    assert subgraph_rd.columns == ["B"]
+    assert next(iter(subgraph_rd.dsk.values()))[0].columns == ["B"]
+
+    subgraph_wt = hlg_layer(dsk, "to-parquet")
+    assert isinstance(subgraph_wt, Blockwise)
 
     assert_eq(ddf.compute(optimize_graph=False), ddf.compute())
 
@@ -2501,6 +2504,32 @@ def test_getitem_optimization_multi(tmpd
     assert_eq(a3, b3)
 
 
+def test_layer_creation_info(tmpdir, engine):
+    df = pd.DataFrame({"a": range(10), "b": ["cat", "dog"] * 5})
+    dd.from_pandas(df, npartitions=1).to_parquet(
+        tmpdir, engine=engine, partition_on=["b"]
+    )
+
+    # Apply filters directly in dd.read_parquet
+    filters = [("b", "==", "cat")]
+    ddf1 = dd.read_parquet(tmpdir, engine=engine, filters=filters)
+    assert "dog" not in ddf1["b"].compute()
+
+    # Results will not match if we use dd.read_parquet
+    # without filters
+    ddf2 = dd.read_parquet(tmpdir, engine=engine)
+    with pytest.raises(AssertionError):
+        assert_eq(ddf1, ddf2)
+
+    # However, we can use `creation_info` to regenerate
+    # the same collection with `filters` defined
+    info = ddf2.dask.layers[ddf2._name].creation_info
+    kwargs = info.get("kwargs", {})
+    kwargs["filters"] = filters
+    ddf3 = info["func"](*info.get("args", []), **kwargs)
+    assert_eq(ddf1, ddf3)
+
+
 @ANY_ENGINE_MARK
 def test_blockwise_parquet_annotations(tmpdir):
     df = pd.DataFrame({"a": np.arange(40, dtype=np.int32)})
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_sql.py 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_sql.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/tests/test_sql.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/tests/test_sql.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,11 +1,12 @@
 import io
+import sys
 from contextlib import contextmanager
 
 import pytest
 
 # import dask
-from dask.dataframe.io.sql import read_sql_table
-from dask.dataframe.utils import assert_eq
+from dask.dataframe.io.sql import read_sql, read_sql_query, read_sql_table
+from dask.dataframe.utils import PANDAS_GT_120, assert_eq
 from dask.utils import tmpfile
 
 pd = pytest.importorskip("pandas")
@@ -14,6 +15,9 @@ pytest.importorskip("sqlalchemy")
 pytest.importorskip("sqlite3")
 np = pytest.importorskip("numpy")
 
+if not PANDAS_GT_120:
+    pytestmark = pytest.mark.filterwarnings("ignore")
+
 
 data = """
 name,number,age,negish
@@ -54,7 +58,8 @@ def test_empty(db):
 
         dask_df = read_sql_table(table.name, uri, index_col="id", npartitions=1)
         assert dask_df.index.name == "id"
-        assert dask_df.col2.dtype == np.dtype("int64")
+        # The dtype of the empty result might no longer be as expected
+        # assert dask_df.col2.dtype == np.dtype("int64")
         pd_dataframe = dask_df.compute()
         assert pd_dataframe.empty is True
 
@@ -169,10 +174,15 @@ def test_needs_rational(db):
             "c": [True, True, False, True, True],
         }
     )
-    df = df.append(
+    df = pd.concat(
         [
-            {"a": "x", "b": now + d * 1000, "c": None},
-            {"a": None, "b": now + d * 1001, "c": None},
+            df,
+            pd.DataFrame(
+                [
+                    {"a": "x", "b": now + d * 1000, "c": None},
+                    {"a": None, "b": now + d * 1001, "c": None},
+                ]
+            ),
         ]
     )
     with tmpfile() as f:
@@ -237,7 +247,7 @@ def test_npartitions(db):
         "test",
         db,
         columns=list(df.columns),
-        bytes_per_chunk=2 ** 30,
+        bytes_per_chunk=2**30,
         index_col="number",
         head_rows=1,
     )
@@ -327,7 +337,7 @@ def test_no_meta_no_head_rows(db):
         read_sql_table("test", db, index_col="number", head_rows=0, npartitions=1)
 
 
-def test_range(db):
+def test_limits(db):
     data = read_sql_table("test", db, npartitions=2, index_col="number", limits=[1, 4])
     assert data.index.min().compute() == 1
     assert data.index.max().compute() == 4
@@ -351,78 +361,86 @@ def test_datetimes():
         assert_eq(data.map_partitions(lambda x: x.sort_index()), df2.sort_index())
 
 
-def test_with_func(db):
-    from sqlalchemy import sql
-
-    index = sql.func.abs(sql.column("negish")).label("abs")
-
-    # function for the index, get all columns
-    data = read_sql_table("test", db, npartitions=2, index_col=index)
-    assert data.divisions[0] == 0
-    part = data.get_partition(0).compute()
-    assert (part.index == 0).all()
-
-    # now an arith op for one column too; it's name will be 'age'
+def test_extra_connection_engine_keywords(caplog, db):
     data = read_sql_table(
-        "test",
-        db,
-        npartitions=2,
-        index_col=index,
-        columns=[index, -(sql.column("age"))],
-    )
-    assert (data.age.compute() < 0).all()
-
-    # a column that would have no name, give it a label
-    index = (-(sql.column("negish"))).label("index")
+        "test", db, npartitions=2, index_col="number", engine_kwargs={"echo": False}
+    ).compute()
+    # no captured message from the stdout with the echo=False parameter (this is the default)
+    out = "\n".join(r.message for r in caplog.records)
+    assert out == ""
+    assert_eq(data, df)
+    # with the echo=True sqlalchemy parameter, you should get all SQL queries in the stdout
     data = read_sql_table(
-        "test", db, npartitions=2, index_col=index, columns=["negish", "age"]
-    )
-    d = data.compute()
-    assert (-d.index == d["negish"]).all()
+        "test", db, npartitions=2, index_col="number", engine_kwargs={"echo": True}
+    ).compute()
+    out = "\n".join(r.message for r in caplog.records)
+    assert "WHERE" in out
+    assert "FROM" in out
+    assert "SELECT" in out
+    assert "AND" in out
+    assert ">= ?" in out
+    assert "< ?" in out
+    assert "<= ?" in out
+    assert_eq(data, df)
 
 
-def test_no_nameless_index(db):
+def test_query(db):
+    import sqlalchemy as sa
     from sqlalchemy import sql
 
-    index = -(sql.column("negish"))
-    with pytest.raises(ValueError):
-        read_sql_table(
-            "test", db, npartitions=2, index_col=index, columns=["negish", "age", index]
-        )
+    s1 = sql.select([sql.column("number"), sql.column("name")]).select_from(
+        sql.table("test")
+    )
+    out = read_sql_query(s1, db, npartitions=2, index_col="number")
+    assert_eq(out, df[["name"]])
 
-    index = sql.func.abs(sql.column("negish"))
+    s2 = (
+        sql.select(
+            [
+                sa.cast(sql.column("number"), sa.types.BigInteger).label("number"),
+                sql.column("name"),
+            ]
+        )
+        .where(sql.column("number") >= 5)
+        .select_from(sql.table("test"))
+    )
 
-    # function for the index, get all columns
-    with pytest.raises(ValueError):
-        read_sql_table("test", db, npartitions=2, index_col=index)
+    out = read_sql_query(s2, db, npartitions=2, index_col="number")
+    assert_eq(out, df.loc[5:, ["name"]])
 
 
-def test_select_from_select(db):
+def test_query_index_from_query(db):
     from sqlalchemy import sql
 
-    s1 = sql.select([sql.column("number"), sql.column("name")]).select_from(
+    number = sql.column("number")
+    name = sql.column("name")
+    s1 = sql.select([number, name, sql.func.length(name).label("lenname")]).select_from(
         sql.table("test")
     )
-    out = read_sql_table(s1, db, npartitions=2, index_col="number")
-    assert_eq(out, df[["name"]])
+    out = read_sql_query(s1, db, npartitions=2, index_col="lenname")
 
+    lenname_df = df.copy()
+    lenname_df["lenname"] = lenname_df["name"].str.len()
+    lenname_df = lenname_df.reset_index().set_index("lenname")
+    assert_eq(out, lenname_df.loc[:, ["number", "name"]])
 
-def test_extra_connection_engine_keywords(capsys, db):
-    data = read_sql_table(
-        "test", db, npartitions=2, index_col="number", engine_kwargs={"echo": False}
-    ).compute()
-    # no captured message from the stdout with the echo=False parameter (this is the default)
-    out, err = capsys.readouterr()
-    assert "SELECT" not in out
-    assert_eq(data, df)
-    # with the echo=True sqlalchemy parameter, you should get all SQL queries in the stdout
-    data = read_sql_table(
-        "test", db, npartitions=2, index_col="number", engine_kwargs={"echo": True}
-    ).compute()
-    out, err = capsys.readouterr()
-    assert "WHERE test.number >= ? AND test.number < ?" in out
-    assert "WHERE test.number >= ? AND test.number <= ?" in out
-    assert_eq(data, df)
+
+def test_query_with_meta(db):
+    from sqlalchemy import sql
+
+    data = {
+        "name": pd.Series([], name="name", dtype="str"),
+        "age": pd.Series([], name="age", dtype="int"),
+    }
+    index = pd.Index([], name="number", dtype="int")
+    meta = pd.DataFrame(data, index=index)
+
+    s1 = sql.select(
+        [sql.column("number"), sql.column("name"), sql.column("age")]
+    ).select_from(sql.table("test"))
+    out = read_sql_query(s1, db, npartitions=2, index_col="number", meta=meta)
+    # Don't check dtype for windows https://github.com/dask/dask/issues/8620
+    assert_eq(out, df[["name", "age"]], check_dtype=sys.platform != "win32")
 
 
 def test_no_character_index_without_divisions(db):
@@ -432,6 +450,21 @@ def test_no_character_index_without_divi
         read_sql_table("test", db, npartitions=2, index_col="name", divisions=None)
 
 
+def test_read_sql(db):
+    from sqlalchemy import sql
+
+    s = sql.select([sql.column("number"), sql.column("name")]).select_from(
+        sql.table("test")
+    )
+    out = read_sql(s, db, npartitions=2, index_col="number")
+    assert_eq(out, df[["name"]])
+
+    data = read_sql_table("test", db, npartitions=2, index_col="number").compute()
+    assert (data.name == df.name).all()
+    assert data.index.name == "number"
+    assert_eq(data, df)
+
+
 @contextmanager
 def tmp_db_uri():
     with tmpfile() as f:
@@ -519,3 +552,20 @@ def test_to_sql_kwargs():
             TypeError, match="to_sql\\(\\) got an unexpected keyword argument 'unknown'"
         ):
             ddf.to_sql("test", uri, unknown=None)
+
+
+def test_to_sql_engine_kwargs(caplog):
+    ddf = dd.from_pandas(df, 2)
+    with tmp_db_uri() as uri:
+        ddf.to_sql("test", uri, engine_kwargs={"echo": False})
+        logs = "\n".join(r.message for r in caplog.records)
+        assert logs == ""
+        assert_eq(df, read_sql_table("test", uri, "number"))
+
+    with tmp_db_uri() as uri:
+        ddf.to_sql("test", uri, engine_kwargs={"echo": True})
+        logs = "\n".join(r.message for r in caplog.records)
+        assert "CREATE" in logs
+        assert "INSERT" in logs
+
+        assert_eq(df, read_sql_table("test", uri, "number"))
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/io/utils.py 2022.02.0+dfsg-1/dask/dataframe/io/utils.py
--- 2022.01.0+dfsg-1/dask/dataframe/io/utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/io/utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,7 +1,20 @@
 import json
 from uuid import uuid4
 
+import fsspec
 import pandas as pd
+from fsspec.implementations.local import LocalFileSystem
+from packaging.version import parse as parse_version
+
+try:
+    import fsspec.parquet as fsspec_parquet
+except ImportError:
+    fsspec_parquet = None
+
+
+def _is_local_fs(fs):
+    """Check if an fsspec file-system is local"""
+    return fs and isinstance(fs, LocalFileSystem)
 
 
 def _get_pyarrow_dtypes(schema, categories):
@@ -97,3 +110,88 @@ def _meta_from_dtypes(to_read_columns, f
 def _guid():
     """Simple utility function to get random hex string"""
     return uuid4().hex
+
+
+def _set_context(obj, stack):
+    """Helper function to place an object on a context stack"""
+    if stack is None:
+        return obj
+    return stack.enter_context(obj)
+
+
+def _open_input_files(
+    paths,
+    fs=None,
+    context_stack=None,
+    open_file_func=None,
+    precache_options=None,
+    **kwargs,
+):
+    """Return a list of open-file objects given
+    a list of input-file paths.
+
+    WARNING: This utility is experimental, and is meant
+    for internal ``dask.dataframe`` use only.
+
+    Parameters
+    ----------
+    paths : list(str)
+        Remote or local path of the parquet file
+    fs : fsspec object, optional
+        File-system instance to use for file handling
+    context_stack : contextlib.ExitStack, Optional
+        Context manager to use for open files.
+    open_file_func : callable, optional
+        Callable function to use for file opening. If this argument
+        is specified, ``open_file_func(path, **kwargs)`` will be used
+        to open each file in ``paths``. Default is ``fs.open``.
+    precache_options : dict, optional
+        Dictionary of key-word arguments to use for precaching.
+        If ``precache_options`` contains ``{"method": "parquet"}``,
+        ``fsspec.parquet.open_parquet_file`` will be used for remote
+        storage.
+    **kwargs :
+        Key-word arguments to pass to the appropriate open function
+    """
+    # Use call-back function if specified
+    if open_file_func is not None:
+        return [
+            _set_context(open_file_func(path, **kwargs), context_stack)
+            for path in paths
+        ]
+
+    # Check if we are using `fsspec.parquet`.
+    # In the future, fsspec should be able to handle
+    # `{"method": "parquet"}`. However, for now we
+    # will redirect to `open_parquet_file` manually
+    precache_options = (precache_options or {}).copy()
+    precache = precache_options.pop("method", None)
+    if (
+        precache == "parquet"
+        and fs is not None
+        and not _is_local_fs(fs)
+        and parse_version(fsspec.__version__) > parse_version("2021.11.0")
+    ):
+        kwargs.update(precache_options)
+        row_groups = kwargs.pop("row_groups", None) or ([None] * len(paths))
+        cache_type = kwargs.pop("cache_type", "parts")
+        if cache_type != "parts":
+            raise ValueError(
+                f"'parts' `cache_type` required for 'parquet' precaching,"
+                f" got {cache_type}."
+            )
+        return [
+            _set_context(
+                fsspec_parquet.open_parquet_file(
+                    path,
+                    fs=fs,
+                    row_groups=rgs,
+                    **kwargs,
+                ),
+                context_stack,
+            )
+            for path, rgs in zip(paths, row_groups)
+        ]
+    elif fs is not None:
+        return [_set_context(fs.open(path, **kwargs), context_stack) for path in paths]
+    return [_set_context(open(path, **kwargs), context_stack) for path in paths]
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/methods.py 2022.02.0+dfsg-1/dask/dataframe/methods.py
--- 2022.01.0+dfsg-1/dask/dataframe/methods.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/methods.py	2022-02-11 16:21:12.000000000 +0000
@@ -101,7 +101,7 @@ def boundary_slice(df, start, stop, righ
         kind = kind or "loc"
         kind_opts = {"kind": kind}
 
-    if kind == "loc" and not df.index.is_monotonic:
+    if kind == "loc" and not df.index.is_monotonic_increasing:
         # Pandas treats missing keys differently for label-slicing
         # on monotonic vs. non-monotonic indexes
         # If the index is monotonic, `df.loc[start:stop]` is fine.
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/optimize.py 2022.02.0+dfsg-1/dask/dataframe/optimize.py
--- 2022.01.0+dfsg-1/dask/dataframe/optimize.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/optimize.py	2022-02-11 16:21:12.000000000 +0000
@@ -47,18 +47,14 @@ def optimize(dsk, keys, **kwargs):
 
 
 def optimize_dataframe_getitem(dsk, keys):
-    # This optimization looks for all `DataFrameLayer` instances,
+    # This optimization looks for all `DataFrameIOLayer` instances,
     # and calls `project_columns` on any layers that directly precede
-    # a (qualified) `getitem` operation. In the future, we can
-    # search for `getitem` operations instead, and work backwards
-    # through multiple adjacent `DataFrameLayer`s. This approach
-    # may become beneficial once `DataFrameLayer` is made a base
-    # type for all relevant DataFrame operations.
+    # a (qualified) `getitem` operation.
 
-    from ..layers import DataFrameLayer
+    from ..layers import DataFrameIOLayer
 
     dataframe_blockwise = [
-        k for k, v in dsk.layers.items() if isinstance(v, DataFrameLayer)
+        k for k, v in dsk.layers.items() if isinstance(v, DataFrameIOLayer)
     ]
 
     layers = dsk.layers.copy()
@@ -108,7 +104,7 @@ def optimize_dataframe_getitem(dsk, keys
         # Project columns and update blocks
         if column_projection:
             old = layers[k]
-            new = old.project_columns(columns)[0]
+            new = old.project_columns(columns)
             if new.name != old.name:
                 columns = list(columns)
                 assert len(update_blocks)
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/shuffle.py 2022.02.0+dfsg-1/dask/dataframe/shuffle.py
--- 2022.01.0+dfsg-1/dask/dataframe/shuffle.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/shuffle.py	2022-02-11 16:21:12.000000000 +0000
@@ -114,6 +114,19 @@ def sort_values(
             sort_function, **sort_function_kwargs
         )
 
+    if not isinstance(ascending, bool):
+        # support [True] as input
+        if (
+            isinstance(ascending, list)
+            and len(ascending) == 1
+            and isinstance(ascending[0], bool)
+        ):
+            ascending = ascending[0]
+        else:
+            raise NotImplementedError(
+                f"Dask currently only supports a single boolean for ascending. You passed {str(ascending)}"
+            )
+
     if (
         all(not pd.isna(x) for x in divisions)
         and mins == sorted(mins, reverse=not ascending)
@@ -666,7 +679,7 @@ def rearrange_by_column_tasks(
     else:
         k = n
 
-    inputs = [tuple(digit(i, j, k) for j in range(stages)) for i in range(k ** stages)]
+    inputs = [tuple(digit(i, j, k) for j in range(stages)) for i in range(k**stages)]
 
     npartitions_orig = df.npartitions
     token = tokenize(df, stages, column, n, k)
@@ -883,7 +896,7 @@ def shuffle_group(df, cols, stage, k, np
     typ = np.min_scalar_type(npartitions * 2)
 
     c = np.mod(c, npartitions).astype(typ, copy=False)
-    np.floor_divide(c, k ** stage, out=c)
+    np.floor_divide(c, k**stage, out=c)
     np.mod(c, k, out=c)
 
     return group_split_dispatch(df, c, k, ignore_index=ignore_index)
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_arithmetics_reduction.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_arithmetics_reduction.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_arithmetics_reduction.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_arithmetics_reduction.py	2022-02-11 16:21:12.000000000 +0000
@@ -134,9 +134,9 @@ def test_deterministic_arithmetic_names(
     df = pd.DataFrame({"x": [1, 2, 3, 4], "y": [5, 6, 7, 8]})
     a = dd.from_pandas(df, npartitions=2)
 
-    assert sorted((a.x + a.y ** 2).dask) == sorted((a.x + a.y ** 2).dask)
-    assert sorted((a.x + a.y ** 2).dask) != sorted((a.x + a.y ** 3).dask)
-    assert sorted((a.x + a.y ** 2).dask) != sorted((a.x - a.y ** 2).dask)
+    assert sorted((a.x + a.y**2).dask) == sorted((a.x + a.y**2).dask)
+    assert sorted((a.x + a.y**2).dask) != sorted((a.x + a.y**3).dask)
+    assert sorted((a.x + a.y**2).dask) != sorted((a.x - a.y**2).dask)
 
 
 @pytest.mark.slow
@@ -277,7 +277,7 @@ def check_series_arithmetics(l, r, el, e
     assert_eq(l - r, el - er)
     assert_eq(l / r, el / er)
     assert_eq(l // r, el // er)
-    assert_eq(l ** r, el ** er)
+    assert_eq(l**r, el**er)
     assert_eq(l % r, el % er)
 
     if allow_comparison_ops:
@@ -306,7 +306,7 @@ def check_series_arithmetics(l, r, el, e
     assert_eq(l | True, el | True)
     assert_eq(l ^ True, el ^ True)
     assert_eq(l // 2, el // 2)
-    assert_eq(l ** 2, el ** 2)
+    assert_eq(l**2, el**2)
     assert_eq(l % 2, el % 2)
     assert_eq(l > 2, el > 2)
     assert_eq(l < 2, el < 2)
@@ -323,7 +323,7 @@ def check_series_arithmetics(l, r, el, e
     assert_eq(True | r, True | er)
     assert_eq(True ^ r, True ^ er)
     assert_eq(2 // r, 2 // er)
-    assert_eq(2 ** r, 2 ** er)
+    assert_eq(2**r, 2**er)
     assert_eq(2 % r, 2 % er)
     assert_eq(2 > r, 2 > er)
     assert_eq(2 < r, 2 < er)
@@ -361,7 +361,7 @@ def check_frame_arithmetics(l, r, el, er
     assert_eq(l - r, el - er)
     assert_eq(l / r, el / er)
     assert_eq(l // r, el // er)
-    assert_eq(l ** r, el ** er)
+    assert_eq(l**r, el**er)
     assert_eq(l % r, el % er)
 
     if allow_comparison_ops:
@@ -390,7 +390,7 @@ def check_frame_arithmetics(l, r, el, er
     assert_eq(l | True, el | True)
     assert_eq(l ^ True, el ^ True)
     assert_eq(l // 2, el // 2)
-    assert_eq(l ** 2, el ** 2)
+    assert_eq(l**2, el**2)
     assert_eq(l % 2, el % 2)
     assert_eq(l > 2, el > 2)
     assert_eq(l < 2, el < 2)
@@ -407,7 +407,7 @@ def check_frame_arithmetics(l, r, el, er
     assert_eq(True | l, True | el)
     assert_eq(True ^ l, True ^ el)
     assert_eq(2 // l, 2 // el)
-    assert_eq(2 ** l, 2 ** el)
+    assert_eq(2**l, 2**el)
     assert_eq(2 % l, 2 % el)
     assert_eq(2 > l, 2 > el)
     assert_eq(2 < l, 2 < el)
@@ -448,7 +448,7 @@ def test_scalar_arithmetics():
     assert_eq(l - r, el - er)
     assert_eq(l / r, el / er)
     assert_eq(l // r, el // er)
-    assert_eq(l ** r, el ** er)
+    assert_eq(l**r, el**er)
     assert_eq(l % r, el % er)
 
     assert_eq(l & r, el & er)
@@ -469,7 +469,7 @@ def test_scalar_arithmetics():
     assert_eq(l | True, el | True)
     assert_eq(l ^ True, el ^ True)
     assert_eq(l // 2, el // 2)
-    assert_eq(l ** 2, el ** 2)
+    assert_eq(l**2, el**2)
     assert_eq(l % 2, el % 2)
     assert_eq(l > 2, el > 2)
     assert_eq(l < 2, el < 2)
@@ -486,7 +486,7 @@ def test_scalar_arithmetics():
     assert_eq(True | r, True | er)
     assert_eq(True ^ r, True ^ er)
     assert_eq(2 // r, 2 // er)
-    assert_eq(2 ** r, 2 ** er)
+    assert_eq(2**r, 2**er)
     assert_eq(2 % r, 2 % er)
     assert_eq(2 > r, 2 > er)
     assert_eq(2 < r, 2 < er)
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_dataframe.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_dataframe.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_dataframe.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_dataframe.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,3 +1,4 @@
+import contextlib
 import warnings
 import weakref
 import xml.etree.ElementTree
@@ -16,7 +17,13 @@ import dask.dataframe.groupby
 from dask.base import compute_as_if_collection
 from dask.blockwise import fuse_roots
 from dask.dataframe import _compat, methods
-from dask.dataframe._compat import PANDAS_GT_110, PANDAS_GT_120, tm
+from dask.dataframe._compat import (
+    PANDAS_GT_110,
+    PANDAS_GT_120,
+    PANDAS_GT_140,
+    PANDAS_GT_150,
+    tm,
+)
 from dask.dataframe.core import (
     Scalar,
     _concat,
@@ -355,8 +362,8 @@ def test_rename_series_method_2():
     ds = dd.from_pandas(s, 2)
 
     for is_sorted in [True, False]:
-        res = ds.rename(lambda x: x ** 2, sorted_index=is_sorted)
-        assert_eq(res, s.rename(lambda x: x ** 2))
+        res = ds.rename(lambda x: x**2, sorted_index=is_sorted)
+        assert_eq(res, s.rename(lambda x: x**2))
         assert res.known_divisions == is_sorted
 
         res = ds.rename(s, sorted_index=is_sorted)
@@ -372,13 +379,13 @@ def test_rename_series_method_2():
     assert not res.known_divisions
 
     ds2 = ds.clear_divisions()
-    res = ds2.rename(lambda x: x ** 2, sorted_index=True)
-    assert_eq(res, s.rename(lambda x: x ** 2))
+    res = ds2.rename(lambda x: x**2, sorted_index=True)
+    assert_eq(res, s.rename(lambda x: x**2))
     assert not res.known_divisions
 
-    res = ds.rename(lambda x: x ** 2, inplace=True, sorted_index=True)
+    res = ds.rename(lambda x: x**2, inplace=True, sorted_index=True)
     assert res is ds
-    s.rename(lambda x: x ** 2, inplace=True)
+    s.rename(lambda x: x**2, inplace=True)
     assert_eq(ds, s)
 
 
@@ -3005,6 +3012,16 @@ def test_apply_warns():
     assert "int64" in str(w[0].message)
 
 
+def test_apply_warns_with_invalid_meta():
+    df = pd.DataFrame({"x": [1, 2, 3, 4], "y": [10, 20, 30, 40]})
+    ddf = dd.from_pandas(df, npartitions=2)
+
+    func = lambda row: row["x"] + row["y"]
+
+    with pytest.warns(FutureWarning, match="Meta is not valid"):
+        ddf.apply(func, axis=1, meta=int)
+
+
 def test_applymap():
     df = pd.DataFrame({"x": [1, 2, 3, 4], "y": [10, 20, 30, 40]})
     ddf = dd.from_pandas(df, npartitions=2)
@@ -3699,8 +3716,8 @@ def test_inplace_operators():
 
     ddf.y **= 0.5
 
-    assert_eq(ddf.y, df.y ** 0.5)
-    assert_eq(ddf, df.assign(y=df.y ** 0.5))
+    assert_eq(ddf.y, df.y**0.5)
+    assert_eq(ddf, df.assign(y=df.y**0.5))
 
 
 @pytest.mark.parametrize("skipna", [True, False])
@@ -4164,7 +4181,8 @@ def test_dataframe_mode():
     ddf = dd.from_pandas(df, npartitions=3)
 
     assert_eq(ddf.mode(), df.mode())
-    assert_eq(ddf.Name.mode(), df.Name.mode())
+    # name is not preserved in older pandas
+    assert_eq(ddf.Name.mode(), df.Name.mode(), check_names=PANDAS_GT_140)
 
     # test empty
     df = pd.DataFrame(columns=["a", "b"])
@@ -4765,6 +4783,27 @@ def test_nunique(dropna, axis):
     )
 
 
+def test_view():
+    data = {
+        "x": pd.Series(range(5), dtype="int8"),
+        "y": pd.Series(
+            [
+                "2021-11-27 00:05:02.175274",
+                "2021-11-27 00:05:05.205596",
+                "2021-11-27 00:05:29.212572",
+                "2021-11-27 00:05:25.708343",
+                "2021-11-27 00:05:47.714958",
+            ],
+            dtype="datetime64[ns]",
+        ),
+    }
+
+    df = pd.DataFrame(data)
+    ddf = dd.from_pandas(df, npartitions=2)
+    assert_eq(ddf["x"].view("uint8"), df["x"].view("uint8"))
+    assert_eq(ddf["y"].view("int64"), df["y"].view("int64"))
+
+
 def test_simple_map_partitions():
     data = {"col_0": [9, -3, 0, -1, 5], "col_1": [-2, -7, 6, 8, -5]}
     df = pd.DataFrame(data)
@@ -4862,12 +4901,13 @@ def test_dask_layers():
     assert dds.__dask_layers__() == (dds._name,)
     ddi = dds.min()
     assert ddi.key[1:] == (0,)
-    assert ddi.dask.layers.keys() == {ddf._name, dds._name, ddi.key[0]}
-    assert ddi.dask.dependencies == {
-        ddf._name: set(),
-        dds._name: {ddf._name},
-        ddi.key[0]: {dds._name},
-    }
+    # Note that the `min` operation will use two layers
+    # now that ACA uses uses HLG
+    assert {ddf._name, dds._name, ddi.key[0]}.issubset(ddi.dask.layers.keys())
+    assert len(ddi.dask.layers) == 4
+    assert ddi.dask.dependencies[ddf._name] == set()
+    assert ddi.dask.dependencies[dds._name] == {ddf._name}
+    assert len(ddi.dask.dependencies) == 4
     assert ddi.__dask_layers__() == (ddi.key[0],)
 
 
@@ -4969,11 +5009,26 @@ def test_use_of_weakref_proxy():
     isinstance(res.compute(), pd.Series)
 
 
+@contextlib.contextmanager
+def check_is_monotonic_warning():
+    # `is_monotonic` was deprecated starting in `pandas=1.5.0`
+    if not PANDAS_GT_150:
+        with contextlib.nullcontext() as ctx:
+            yield ctx
+    else:
+        with pytest.warns(FutureWarning, match="is_monotonic is deprecated") as ctx:
+            yield ctx
+
+
 def test_is_monotonic_numeric():
     s = pd.Series(range(20))
     ds = dd.from_pandas(s, npartitions=5)
     assert_eq(s.is_monotonic_increasing, ds.is_monotonic_increasing)
-    assert_eq(s.is_monotonic, ds.is_monotonic)
+    with check_is_monotonic_warning():
+        expected = s.is_monotonic
+    with check_is_monotonic_warning():
+        result = ds.is_monotonic
+    assert_eq(expected, result)
 
     s_2 = pd.Series(range(20, 0, -1))
     ds_2 = dd.from_pandas(s_2, npartitions=5)
@@ -4999,7 +5054,11 @@ def test_index_is_monotonic_numeric():
     s = pd.Series(1, index=range(20))
     ds = dd.from_pandas(s, npartitions=5, sort=False)
     assert_eq(s.index.is_monotonic_increasing, ds.index.is_monotonic_increasing)
-    assert_eq(s.index.is_monotonic, ds.index.is_monotonic)
+    with check_is_monotonic_warning():
+        expected = s.index.is_monotonic
+    with check_is_monotonic_warning():
+        result = ds.index.is_monotonic
+    assert_eq(expected, result)
 
     s_2 = pd.Series(1, index=range(20, 0, -1))
     ds_2 = dd.from_pandas(s_2, npartitions=5, sort=False)
@@ -5019,3 +5078,34 @@ def test_index_is_monotonic_dt64():
     s_2 = pd.Series(1, index=list(reversed(s)))
     ds_2 = dd.from_pandas(s_2, npartitions=5, sort=False)
     assert_eq(s_2.index.is_monotonic_decreasing, ds_2.index.is_monotonic_decreasing)
+
+
+def test_custom_map_reduce():
+    # Make sure custom map-reduce workflows can use
+    # the universal ACA code path with metadata
+    # that is not DataFrame-like.
+    # See: https://github.com/dask/dask/issues/8636
+
+    df = pd.DataFrame(columns=["a"], data=[[2], [4], [8]], index=[0, 1, 2])
+    ddf = dd.from_pandas(df, npartitions=2)
+
+    def map_fn(x):
+        return {"x": x, "y": x}
+
+    def reduce_fn(series):
+        merged = None
+        for mapped in series:
+            if merged is None:
+                merged = mapped.copy()
+            else:
+                merged["x"] += mapped["x"]
+                merged["y"] *= mapped["y"]
+        return merged
+
+    result = (
+        ddf["a"]
+        .map(map_fn, meta=("data", "object"))
+        .reduction(reduce_fn, aggregate=reduce_fn, meta=("data", "object"))
+        .compute()[0]
+    )
+    assert result == {"x": 14, "y": 64}
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_groupby.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_groupby.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_groupby.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_groupby.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,5 +1,6 @@
 import collections
 import operator
+import pickle
 import warnings
 
 import numpy as np
@@ -136,11 +137,12 @@ def test_full_groupby():
     def func(df):
         return df.assign(b=df.b - df.b.mean())
 
-    with warnings.catch_warnings():
-        warnings.simplefilter("ignore")
+    expected = df.groupby("a").apply(func)
+
+    with pytest.warns(UserWarning, match="`meta` is not specified"):
         assert ddf.groupby("a").apply(func)._name.startswith("func")
 
-        assert_eq(df.groupby("a").apply(func), ddf.groupby("a").apply(func))
+        assert_eq(expected, ddf.groupby("a").apply(func))
 
 
 def test_full_groupby_apply_multiarg():
@@ -161,10 +163,7 @@ def test_full_groupby_apply_multiarg():
     c_delayed = dask.delayed(lambda: c)()
     d_delayed = dask.delayed(lambda: d)()
 
-    meta = df.groupby("a").apply(func, c)
-
-    with warnings.catch_warnings():
-        warnings.simplefilter("ignore")
+    with pytest.warns(UserWarning, match="`meta` is not specified"):
         assert_eq(
             df.groupby("a").apply(func, c, d=d),
             ddf.groupby("a").apply(func, c, d=d_scalar),
@@ -182,15 +181,17 @@ def test_full_groupby_apply_multiarg():
             check_dtype=False,
         )
 
-        assert_eq(
-            df.groupby("a").apply(func, c),
-            ddf.groupby("a").apply(func, c_scalar, meta=meta),
-        )
+    meta = df.groupby("a").apply(func, c)
 
-        assert_eq(
-            df.groupby("a").apply(func, c, d=d),
-            ddf.groupby("a").apply(func, c, d=d_scalar, meta=meta),
-        )
+    assert_eq(
+        df.groupby("a").apply(func, c),
+        ddf.groupby("a").apply(func, c_scalar, meta=meta),
+    )
+
+    assert_eq(
+        df.groupby("a").apply(func, c, d=d),
+        ddf.groupby("a").apply(func, c, d=d_scalar, meta=meta),
+    )
 
     # Delayed arguments work, but only if metadata is provided
     with pytest.raises(ValueError) as exc:
@@ -243,10 +244,7 @@ def test_full_groupby_multilevel(grouper
     def func(df):
         return df.assign(b=df.b - df.b.mean())
 
-    # last one causes a DeprecationWarning from pandas.
-    # See https://github.com/pandas-dev/pandas/issues/16481
-    with warnings.catch_warnings():
-        warnings.simplefilter("ignore")
+    with pytest.warns(UserWarning, match="`meta` is not specified"):
         assert_eq(
             df.groupby(grouper(df)).apply(func), ddf.groupby(grouper(ddf)).apply(func)
         )
@@ -433,6 +431,19 @@ def test_series_groupby_propagates_names
     assert_eq(result, expected)
 
 
+@pytest.mark.parametrize("npartitions", (1, 2))
+@pytest.mark.parametrize("func", ("cumsum", "cumprod", "cumcount"))
+def test_series_groupby_cumfunc_with_named_index(npartitions, func):
+    df = pd.DataFrame(
+        {"x": [1, 2, 3, 4, 5, 6, 7], "y": [8, 9, 6, 2, 3, 5, 6]}
+    ).set_index("x")
+    ddf = dd.from_pandas(df, npartitions)
+    assert ddf.npartitions == npartitions
+    expected = getattr(df["y"].groupby("x"), func)()
+    result = getattr(ddf["y"].groupby("x"), func)()
+    assert_eq(result, expected)
+
+
 def test_series_groupby():
     s = pd.Series([1, 2, 2, 1, 1])
     pd_group = s.groupby(s)
@@ -989,7 +1000,14 @@ def test_aggregate_build_agg_args__reuse
 def test_aggregate_dask():
     dask_holder = collections.namedtuple("dask_holder", ["dask"])
     get_agg_dask = lambda obj: dask_holder(
-        {k: v for (k, v) in obj.dask.items() if k[0].startswith("aggregate")}
+        {
+            k: v
+            for (k, v) in obj.dask.items()
+            # Skip "chunk" tasks, because they include
+            # SubgraphCallable object with non-deterministic
+            # (uuid-based) function names
+            if (k[0].startswith("aggregate") and "-chunk-" not in k[0])
+        }
     )
 
     specs = [
@@ -1002,6 +1020,8 @@ def test_aggregate_dask():
             "max",
             "count",
             "size",
+        ],
+        [
             "std",
             "var",
             "first",
@@ -1044,14 +1064,20 @@ def test_aggregate_dask():
         assert_max_deps(agg_dask1, 2)
         assert_max_deps(agg_dask2, 2)
 
-        # check for deterministic key names and values
-        assert agg_dask1 == agg_dask2
+        # check for deterministic key names and values.
+        # Require pickle since "partial" concat functions
+        # used in tree-reduction cannot be compared
+        assert pickle.dumps(agg_dask1[0]) == pickle.dumps(agg_dask2[0])
 
         # the length of the dask does not depend on the passed spec
         for other_spec in specs:
-            other = ddf.groupby(["a", "b"]).agg(other_spec, split_every=2)
-            assert len(other.dask) == len(result1.dask)
-            assert len(other.dask) == len(result2.dask)
+            # Note: List-based aggregation specs may result in
+            # an extra delayed layer. This is because a "long" list
+            # arg will be detected in `dask.array.core.normalize_arg`.
+            if isinstance(spec, list) == isinstance(other_spec, list):
+                other = ddf.groupby(["a", "b"]).agg(other_spec, split_every=2)
+                assert len(other.dask) == len(result1.dask)
+                assert len(other.dask) == len(result2.dask)
 
 
 @pytest.mark.parametrize(
@@ -1597,7 +1623,8 @@ def test_groupby_column_and_index_agg_fu
 
 @pytest.mark.parametrize("group_args", [["idx", "a"], ["a", "idx"], ["idx"], "idx"])
 @pytest.mark.parametrize(
-    "apply_func", [np.min, np.mean, lambda s: np.max(s) - np.mean(s)]
+    "apply_func",
+    [np.min, np.mean, lambda s, axis=None: np.max(s.values) - np.mean(s.values)],
 )
 def test_groupby_column_and_index_apply(group_args, apply_func):
     df = pd.DataFrame(
@@ -1608,32 +1635,32 @@ def test_groupby_column_and_index_apply(
     ddf_no_divs = dd.from_pandas(df, npartitions=df.index.nunique(), sort=False)
 
     # Expected result
-    expected = df.groupby(group_args).apply(apply_func)
+    expected = df.groupby(group_args).apply(apply_func, axis=0)
 
-    with warnings.catch_warnings():
-        warnings.simplefilter("ignore")
+    # Compute on dask DataFrame with divisions (no shuffling)
+    result = ddf.groupby(group_args).apply(apply_func, axis=0, meta=expected)
+    assert_eq(expected, result, check_divisions=False)
+
+    # Check that partitioning is preserved
+    assert ddf.divisions == result.divisions
 
-        # Compute on dask DataFrame with divisions (no shuffling)
-        result = ddf.groupby(group_args).apply(apply_func)
-        assert_eq(expected, result, check_divisions=False)
+    # Check that no shuffling occurred.
+    # The groupby operation should add only 1 task per partition
+    assert len(result.dask) == (len(ddf.dask) + ddf.npartitions)
 
-        # Check that partitioning is preserved
-        assert ddf.divisions == result.divisions
+    expected = df.groupby(group_args).apply(apply_func, axis=0)
 
-        # Check that no shuffling occurred.
-        # The groupby operation should add only 1 task per partition
-        assert len(result.dask) == (len(ddf.dask) + ddf.npartitions)
+    # Compute on dask DataFrame without divisions (requires shuffling)
+    result = ddf_no_divs.groupby(group_args).apply(apply_func, axis=0, meta=expected)
 
-        # Compute on dask DataFrame without divisions (requires shuffling)
-        result = ddf_no_divs.groupby(group_args).apply(apply_func)
-        assert_eq(expected, result, check_divisions=False)
+    assert_eq(expected, result, check_divisions=False)
 
-        # Check that divisions were preserved (all None in this case)
-        assert ddf_no_divs.divisions == result.divisions
+    # Check that divisions were preserved (all None in this case)
+    assert ddf_no_divs.divisions == result.divisions
 
-        # Crude check to see if shuffling was performed.
-        # The groupby operation should add only more than 1 task per partition
-        assert len(result.dask) > (len(ddf_no_divs.dask) + ddf_no_divs.npartitions)
+    # Crude check to see if shuffling was performed.
+    # The groupby operation should add only more than 1 task per partition
+    assert len(result.dask) > (len(ddf_no_divs.dask) + ddf_no_divs.npartitions)
 
 
 custom_mean = dd.Aggregation(
@@ -2345,7 +2372,7 @@ def test_groupby_large_ints_exception(ba
     else:
         data_frame = dd.from_pandas
     max = np.iinfo(np.uint64).max
-    sqrt = max ** 0.5
+    sqrt = max**0.5
     series = data_source.Series(
         np.concatenate([sqrt * np.arange(5), np.arange(35)])
     ).astype("int64")
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_multi.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_multi.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_multi.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_multi.py	2022-02-11 16:21:12.000000000 +0000
@@ -6,7 +6,7 @@ import pytest
 
 import dask.dataframe as dd
 from dask.base import compute_as_if_collection
-from dask.dataframe._compat import tm
+from dask.dataframe._compat import PANDAS_GT_140, tm
 from dask.dataframe.core import _Frame
 from dask.dataframe.methods import concat
 from dask.dataframe.multi import (
@@ -2009,6 +2009,26 @@ def test_concat_datetimeindex():
     assert_eq(result, expected)
 
 
+def check_append_with_warning(dask_obj, dask_append, pandas_obj, pandas_append):
+
+    if PANDAS_GT_140:
+        with pytest.warns() as record:
+            expected = pandas_obj.append(pandas_append)
+            result = dask_obj.append(dask_append)
+            assert_eq(result, expected)
+        for w in record:
+            assert w.category == FutureWarning
+            assert "append method is deprecated" in str(w.message)
+    else:
+        with warnings.catch_warnings():
+            warnings.simplefilter("error")
+            expected = pandas_obj.append(pandas_append)
+            result = dask_obj.append(dask_append)
+            assert_eq(result, expected)
+
+    return result
+
+
 def test_append():
     df = pd.DataFrame({"a": [1, 2, 3, 4, 5, 6], "b": [1, 2, 3, 4, 5, 6]})
     df2 = pd.DataFrame(
@@ -2024,29 +2044,20 @@ def test_append():
 
     s = pd.Series([7, 8], name=6, index=["a", "b"])
 
-    def check_with_warning(dask_obj, dask_append, pandas_obj, pandas_append):
-        with warnings.catch_warnings():
-            warnings.simplefilter("ignore", FutureWarning)
-            expected = pandas_obj.append(pandas_append)
-
-            result = dask_obj.append(dask_append)
-
-        assert_eq(result, expected)
-
-    check_with_warning(ddf, s, df, s)
-    check_with_warning(ddf, ddf2, df, df2)
-    check_with_warning(ddf.a, ddf2.a, df.a, df2.a)
+    check_append_with_warning(ddf, s, df, s)
+    check_append_with_warning(ddf, ddf2, df, df2)
+    check_append_with_warning(ddf.a, ddf2.a, df.a, df2.a)
 
     # different columns
-    check_with_warning(ddf, ddf3, df, df3)
-    check_with_warning(ddf.a, ddf3.b, df.a, df3.b)
+    check_append_with_warning(ddf, ddf3, df, df3)
+    check_append_with_warning(ddf.a, ddf3.b, df.a, df3.b)
 
     # dask + pandas
-    check_with_warning(ddf, df2, df, df2)
-    check_with_warning(ddf.a, df2.a, df.a, df2.a)
+    check_append_with_warning(ddf, df2, df, df2)
+    check_append_with_warning(ddf.a, df2.a, df.a, df2.a)
 
-    check_with_warning(ddf, df3, df, df3)
-    check_with_warning(ddf.a, df3.b, df.a, df3.b)
+    check_append_with_warning(ddf, df3, df, df3)
+    check_append_with_warning(ddf.a, df3.b, df.a, df3.b)
 
 
 def test_append2():
@@ -2057,6 +2068,7 @@ def test_append2():
     }
     meta = make_meta({"a": "i8", "b": "i8"}, parent_meta=pd.DataFrame())
     ddf1 = dd.DataFrame(dsk, "x", meta, [None, None])
+    df1 = ddf1.compute()
 
     dsk = {
         ("y", 0): pd.DataFrame({"a": [10, 20, 30], "b": [40, 50, 60]}),
@@ -2064,6 +2076,7 @@ def test_append2():
         ("y", 2): pd.DataFrame({"a": [70, 80, 90], "b": [0, 0, 0]}),
     }
     ddf2 = dd.DataFrame(dsk, "y", meta, [None, None])
+    df2 = ddf2.compute()
 
     dsk = {
         ("y", 0): pd.DataFrame({"b": [10, 20, 30], "c": [40, 50, 60]}),
@@ -2071,29 +2084,22 @@ def test_append2():
     }
     meta = make_meta({"b": "i8", "c": "i8"}, parent_meta=pd.DataFrame())
     ddf3 = dd.DataFrame(dsk, "y", meta, [None, None])
+    df3 = ddf3.compute()
 
-    assert_eq(ddf1.append(ddf2), ddf1.compute().append(ddf2.compute(), sort=False))
-    assert_eq(ddf2.append(ddf1), ddf2.compute().append(ddf1.compute(), sort=False))
+    check_append_with_warning(ddf1, ddf2, df1, df2)
+    check_append_with_warning(ddf2, ddf1, df2, df1)
 
     # different columns
-    assert_eq(ddf1.append(ddf3), ddf1.compute().append(ddf3.compute(), sort=False))
-    assert_eq(ddf3.append(ddf1), ddf3.compute().append(ddf1.compute(), sort=False))
+    check_append_with_warning(ddf1, ddf3, df1, df3)
+    check_append_with_warning(ddf3, ddf1, df3, df1)
 
     # Dask + pandas
-    assert_eq(
-        ddf1.append(ddf2.compute()), ddf1.compute().append(ddf2.compute(), sort=False)
-    )
-    assert_eq(
-        ddf2.append(ddf1.compute()), ddf2.compute().append(ddf1.compute(), sort=False)
-    )
+    check_append_with_warning(ddf1, df2, df1, df2)
+    check_append_with_warning(ddf2, df1, df2, df1)
 
     # different columns
-    assert_eq(
-        ddf1.append(ddf3.compute()), ddf1.compute().append(ddf3.compute(), sort=False)
-    )
-    assert_eq(
-        ddf3.append(ddf1.compute()), ddf3.compute().append(ddf1.compute(), sort=False)
-    )
+    check_append_with_warning(ddf1, df3, df1, df3)
+    check_append_with_warning(ddf3, df1, df3, df1)
 
 
 def test_append_categorical():
@@ -2131,18 +2137,16 @@ def test_append_categorical():
             )
         ddf1, ddf2 = dframes
 
-        res = ddf1.append(ddf2)
-        assert_eq(res, df1.append(df2))
+        res = check_append_with_warning(ddf1, ddf2, df1, df2)
+
         assert has_known_categories(res.index) == known
         assert has_known_categories(res.y) == known
 
-        res = ddf1.y.append(ddf2.y)
-        assert_eq(res, df1.y.append(df2.y))
+        res = check_append_with_warning(ddf1.y, ddf2.y, df1.y, df2.y)
         assert has_known_categories(res.index) == known
         assert has_known_categories(res) == known
 
-        res = ddf1.index.append(ddf2.index)
-        assert_eq(res, df1.index.append(df2.index))
+        res = check_append_with_warning(ddf1.index, ddf2.index, df1.index, df2.index)
         assert has_known_categories(res) == known
 
 
@@ -2150,9 +2154,8 @@ def test_append_lose_divisions():
     df = pd.DataFrame({"x": [1, 2, 3, 4]}, index=[1, 2, 3, 4])
     ddf = dd.from_pandas(df, npartitions=2)
 
-    ddf2 = ddf.append(ddf)
-    df2 = df.append(df)
-    assert_eq(ddf2, df2)
+    res = check_append_with_warning(ddf, ddf, df, df)
+    assert res.known_divisions is False
 
 
 def test_singleton_divisions():
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_reshape.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_reshape.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_reshape.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_reshape.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,9 +1,10 @@
 import numpy as np
 import pandas as pd
 import pytest
+from packaging.version import parse as parse_version
 
 import dask.dataframe as dd
-from dask.dataframe._compat import tm
+from dask.dataframe._compat import PANDAS_VERSION, tm
 from dask.dataframe.utils import assert_eq, make_meta
 
 
@@ -59,14 +60,10 @@ def test_get_dummies_kwargs():
     ds = dd.from_pandas(s, 2)
     res = dd.get_dummies(ds, prefix="X", prefix_sep="-")
     assert_eq(res, exp)
-    tm.assert_index_equal(res.columns, pd.Index(["X-1", "X-2", "X-3", "X-4"]))
 
     exp = pd.get_dummies(s, drop_first=True)
-
-    ds = dd.from_pandas(s, 2)
     res = dd.get_dummies(ds, drop_first=True)
     assert_eq(res, exp)
-    tm.assert_index_equal(res.columns, exp.columns)
 
     # nan
     s = pd.Series([1, 1, 1, 2, np.nan, 3, np.nan, 5], dtype="category")
@@ -75,17 +72,36 @@ def test_get_dummies_kwargs():
     ds = dd.from_pandas(s, 2)
     res = dd.get_dummies(ds)
     assert_eq(res, exp)
-    tm.assert_index_equal(res.columns, exp.columns)
 
     # dummy_na
     exp = pd.get_dummies(s, dummy_na=True)
-
-    ds = dd.from_pandas(s, 2)
     res = dd.get_dummies(ds, dummy_na=True)
     assert_eq(res, exp)
-    tm.assert_index_equal(res.columns, pd.Index([1, 2, 3, 5, np.nan]))
 
 
+def check_pandas_issue_45618_warning(test_func):
+    # Check for FutureWarning raised in `pandas=1.4.0`-only.
+    # This can be removed when `pandas=1.4.0` is no longer supported (PANDAS_GT_140).
+    # See https://github.com/pandas-dev/pandas/issues/45618 for more details.
+
+    def decorator():
+        with pytest.warns(None) as record:
+            test_func()
+
+        if PANDAS_VERSION == parse_version("1.4.0"):
+            assert len(record)
+            assert all(r.category is FutureWarning for r in record)
+            assert all(
+                "In a future version, passing a SparseArray" in str(r.message)
+                for r in record
+            )
+        else:
+            assert len(record) == 0
+
+    return decorator
+
+
+@check_pandas_issue_45618_warning
 def test_get_dummies_sparse():
     s = pd.Series(pd.Categorical(["a", "b", "a"], categories=["a", "b", "c"]))
     ds = dd.from_pandas(s, 2)
@@ -103,6 +119,7 @@ def test_get_dummies_sparse():
     assert pd.api.types.is_sparse(res.a_a.compute())
 
 
+@check_pandas_issue_45618_warning
 def test_get_dummies_sparse_mix():
     df = pd.DataFrame(
         {
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_shuffle.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_shuffle.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_shuffle.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_shuffle.py	2022-02-11 16:21:12.000000000 +0000
@@ -588,7 +588,7 @@ def test_set_index_sorts():
         dfs.append(pd.DataFrame({"timestamp": vals[lo:hi]}, index=range(lo, hi)))
 
     ddf = dd.concat(dfs).clear_divisions()
-    assert ddf.set_index("timestamp").index.compute().is_monotonic is True
+    assert ddf.set_index("timestamp").index.compute().is_monotonic_increasing is True
 
 
 @pytest.mark.parametrize(
@@ -1248,10 +1248,13 @@ def test_sort_values(nelem, nparts, by,
     "data",
     [
         {
-            "a": list(range(50)) + [None] * 50 + list(range(50, 100)),
-            "b": [None] * 100 + list(range(100, 150)),
+            "a": list(range(50)) + [None] * 50 + list(range(50, 100)),  # type: ignore
+            "b": [None] * 100 + list(range(100, 150)),  # type: ignore
+        },
+        {
+            "a": list(range(15)) + [None] * 5,  # type: ignore
+            "b": list(reversed(range(20))),
         },
-        {"a": list(range(15)) + [None] * 5, "b": list(reversed(range(20)))},
     ],
 )
 def test_sort_values_with_nulls(data, nparts, by, ascending, na_position):
@@ -1281,3 +1284,12 @@ def test_sort_values_custom_function(by,
         )
     expect = df.sort_values(by=by)
     dd.assert_eq(got, expect, check_index=False)
+
+
+def test_sort_values_bool_ascending():
+    df = pd.DataFrame({"a": [1, 2, 3] * 20, "b": [4, 5, 6, 7] * 15})
+    ddf = dd.from_pandas(df, npartitions=10)
+
+    # attempt to sort with list of ascending booleans
+    with pytest.raises(NotImplementedError):
+        ddf.sort_values(by="a", ascending=[True, False])
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/tests/test_utils_dataframe.py 2022.02.0+dfsg-1/dask/dataframe/tests/test_utils_dataframe.py
--- 2022.01.0+dfsg-1/dask/dataframe/tests/test_utils_dataframe.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/tests/test_utils_dataframe.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,9 +1,11 @@
 import re
+from typing import Iterable
 
 import numpy as np
 import pandas as pd
 import pytest
 
+import dask
 import dask.dataframe as dd
 from dask.dataframe._compat import tm
 from dask.dataframe.core import apply_and_enforce
@@ -21,6 +23,7 @@ from dask.dataframe.utils import (
     raise_on_meta_error,
     shard_df_on_index,
 )
+from dask.local import get_sync
 
 
 def test_shard_df_on_index():
@@ -67,7 +70,7 @@ def test_make_meta():
     assert (meta.dtypes == df.dtypes).all()
     assert isinstance(meta.index, pd.RangeIndex)
 
-    # Iterable
+    # List
     meta = make_meta([("a", "i8"), ("c", "f8"), ("b", "O")])
     assert (meta.columns == ["a", "c", "b"]).all()
     assert len(meta) == 0
@@ -81,6 +84,31 @@ def test_make_meta():
     assert meta.dtype == "i8"
     assert meta.name == "a"
 
+    # Iterable
+    class CustomMetadata(Iterable):
+        """Custom class iterator returning pandas types."""
+
+        def __init__(self, max=0):
+            self.types = [("a", "i8"), ("c", "f8"), ("b", "O")]
+
+        def __iter__(self):
+            self.n = 0
+            return self
+
+        def __next__(self):
+            if self.n < len(self.types):
+                ret = self.types[self.n]
+                self.n += 1
+                return ret
+            else:
+                raise StopIteration
+
+    meta = make_meta(CustomMetadata())
+    assert (meta.columns == ["a", "c", "b"]).all()
+    assert len(meta) == 0
+    assert (meta.dtypes == df.dtypes[meta.dtypes.index]).all()
+    assert isinstance(meta.index, pd.RangeIndex)
+
     # With index
     idx = pd.Index([1, 2], name="foo")
     meta = make_meta(
@@ -521,3 +549,31 @@ def test_assert_eq_sorts():
     assert_eq(df1, df2_r, check_index=False)
     with pytest.raises(AssertionError):
         assert_eq(df1, df2_r)
+
+
+def test_assert_eq_scheduler():
+    using_custom_scheduler = False
+
+    def custom_scheduler(*args, **kwargs):
+        nonlocal using_custom_scheduler
+        try:
+            using_custom_scheduler = True
+            return get_sync(*args, **kwargs)
+        finally:
+            using_custom_scheduler = False
+
+    def check_custom_scheduler(part: pd.DataFrame) -> pd.DataFrame:
+        assert using_custom_scheduler, "not using custom scheduler"
+        return part + 1
+
+    df = pd.DataFrame({"x": [1, 2, 3, 4]})
+    ddf = dd.from_pandas(df, npartitions=2)
+    ddf2 = ddf.map_partitions(check_custom_scheduler, meta=ddf)
+
+    with pytest.raises(AssertionError, match="not using custom scheduler"):
+        # NOTE: we compare `ddf2` to itself in order to test both sides of the `assert_eq` logic.
+        assert_eq(ddf2, ddf2)
+
+    assert_eq(ddf2, ddf2, scheduler=custom_scheduler)
+    with dask.config.set(scheduler=custom_scheduler):
+        assert_eq(ddf2, ddf2, scheduler=None)
diff -pruN 2022.01.0+dfsg-1/dask/dataframe/utils.py 2022.02.0+dfsg-1/dask/dataframe/utils.py
--- 2022.01.0+dfsg-1/dask/dataframe/utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/dataframe/utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,20 +1,21 @@
+from __future__ import annotations
+
 import math
-import numbers
 import re
 import sys
 import textwrap
 import traceback
 from collections.abc import Iterator, Mapping
 from contextlib import contextmanager
+from numbers import Number
 
 import numpy as np
 import pandas as pd
 from pandas.api.types import is_scalar  # noqa: F401
 from pandas.api.types import is_categorical_dtype, is_dtype_equal
 
-from ..base import is_dask_collection
+from ..base import get_scheduler, is_dask_collection
 from ..core import get_deps
-from ..local import get_sync
 from ..utils import is_arraylike  # noqa: F401
 from ..utils import asciitable
 from ..utils import is_dataframe_like as dask_is_dataframe_like
@@ -28,7 +29,7 @@ from .dispatch import make_meta  # noqa
 from .dispatch import make_meta_obj, meta_nonempty  # noqa : F401
 from .extensions import make_scalar
 
-meta_object_types = (pd.Series, pd.DataFrame, pd.Index, pd.MultiIndex)
+meta_object_types: tuple[type, ...] = (pd.Series, pd.DataFrame, pd.Index, pd.MultiIndex)
 try:
     import scipy.sparse as sp
 
@@ -445,7 +446,7 @@ def index_summary(idx, name=None):
 ###############################################################
 
 
-def _check_dask(dsk, check_names=True, check_dtypes=True, result=None):
+def _check_dask(dsk, check_names=True, check_dtypes=True, result=None, scheduler=None):
     import dask.dataframe as dd
 
     if hasattr(dsk, "__dask_graph__"):
@@ -453,7 +454,7 @@ def _check_dask(dsk, check_names=True, c
         if hasattr(graph, "validate"):
             graph.validate()
         if result is None:
-            result = dsk.compute(scheduler="sync")
+            result = dsk.compute(scheduler=scheduler)
         if isinstance(dsk, dd.Index):
             assert "Index" in type(result).__name__, type(result)
             # assert type(dsk._meta) == type(result), type(dsk._meta)
@@ -529,19 +530,24 @@ def assert_eq(
     check_dtype=True,
     check_divisions=True,
     check_index=True,
+    scheduler="sync",
     **kwargs,
 ):
     if check_divisions:
-        assert_divisions(a)
-        assert_divisions(b)
+        assert_divisions(a, scheduler=scheduler)
+        assert_divisions(b, scheduler=scheduler)
         if hasattr(a, "divisions") and hasattr(b, "divisions"):
             at = type(np.asarray(a.divisions).tolist()[0])  # numpy to python
             bt = type(np.asarray(b.divisions).tolist()[0])  # scalar conversion
             assert at == bt, (at, bt)
     assert_sane_keynames(a)
     assert_sane_keynames(b)
-    a = _check_dask(a, check_names=check_names, check_dtypes=check_dtype)
-    b = _check_dask(b, check_names=check_names, check_dtypes=check_dtype)
+    a = _check_dask(
+        a, check_names=check_names, check_dtypes=check_dtype, scheduler=scheduler
+    )
+    b = _check_dask(
+        b, check_names=check_names, check_dtypes=check_dtype, scheduler=scheduler
+    )
     if hasattr(a, "to_pandas"):
         a = a.to_pandas()
     if hasattr(b, "to_pandas"):
@@ -585,7 +591,7 @@ def assert_dask_graph(dask, label):
     raise AssertionError(f"given dask graph doesn't contain label: {label}")
 
 
-def assert_divisions(ddf):
+def assert_divisions(ddf, scheduler=None):
     if not hasattr(ddf, "divisions"):
         return
 
@@ -602,7 +608,8 @@ def assert_divisions(ddf):
         except AttributeError:
             return x.index
 
-    results = get_sync(ddf.dask, ddf.__dask_keys__())
+    get = get_scheduler(scheduler=scheduler, collections=[type(ddf)])
+    results = get(ddf.dask, ddf.__dask_keys__())
     for i, df in enumerate(results[:-1]):
         if len(df):
             assert index(df).min() >= ddf.divisions[i]
@@ -693,11 +700,11 @@ def valid_divisions(divisions):
     for i, x in enumerate(divisions[:-2]):
         if x >= divisions[i + 1]:
             return False
-        if isinstance(x, numbers.Number) and math.isnan(x):
+        if isinstance(x, Number) and math.isnan(x):
             return False
 
     for x in divisions[-2:]:
-        if isinstance(x, numbers.Number) and math.isnan(x):
+        if isinstance(x, Number) and math.isnan(x):
             return False
 
     if divisions[-2] > divisions[-1]:
diff -pruN 2022.01.0+dfsg-1/dask/diagnostics/profile_visualize.py 2022.02.0+dfsg-1/dask/diagnostics/profile_visualize.py
--- 2022.01.0+dfsg-1/dask/diagnostics/profile_visualize.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/diagnostics/profile_visualize.py	2022-02-11 16:21:12.000000000 +0000
@@ -9,6 +9,14 @@ from tlz import accumulate, groupby, plu
 from ..core import istask
 from ..utils import apply, funcname, import_required
 
+
+def BOKEH_VERSION():
+    import bokeh
+    from packaging.version import parse as parse_version
+
+    return parse_version(bokeh.__version__)
+
+
 _BOKEH_MISSING_MSG = "Diagnostics plots require `bokeh` to be installed"
 
 
@@ -200,11 +208,17 @@ def visualize(
             f.x_range = top.x_range
             f.title = None
             f.min_border_top = 20
-            f.plot_height -= 30
+            if BOKEH_VERSION().major < 3:
+                f.plot_height -= 30
+            else:
+                f.height -= 30
         for f in figs[:-1]:
             f.xaxis.axis_label = None
             f.min_border_bottom = 20
-            f.plot_height -= 30
+            if BOKEH_VERSION().major < 3:
+                f.plot_height -= 30
+            else:
+                f.height -= 30
         for f in figs:
             f.min_border_left = 75
             f.min_border_right = 75
@@ -354,8 +368,12 @@ def plot_resources(results, palette="Vir
     # Support plot_width and plot_height for backwards compatibility
     if "plot_width" in kwargs:
         kwargs["width"] = kwargs.pop("plot_width")
+        if BOKEH_VERSION().major >= 3:
+            warnings.warn("Use width instead of plot_width with Bokeh >= 3")
     if "plot_height" in kwargs:
         kwargs["height"] = kwargs.pop("plot_height")
+        if BOKEH_VERSION().major >= 3:
+            warnings.warn("Use height instead of plot_height with Bokeh >= 3")
 
     # Drop `label_size` to match `plot_cache` and `plot_tasks` kwargs
     if "label_size" in kwargs:
@@ -448,8 +466,12 @@ def plot_cache(
     # Support plot_width and plot_height for backwards compatibility
     if "plot_width" in kwargs:
         kwargs["width"] = kwargs.pop("plot_width")
+        if BOKEH_VERSION().major >= 3:
+            warnings.warn("Use width instead of plot_width with Bokeh >= 3")
     if "plot_height" in kwargs:
         kwargs["height"] = kwargs.pop("plot_height")
+        if BOKEH_VERSION().major >= 3:
+            warnings.warn("Use height instead of plot_height with Bokeh >= 3")
     defaults.update(**kwargs)
 
     if results:
diff -pruN 2022.01.0+dfsg-1/dask/diagnostics/progress.py 2022.02.0+dfsg-1/dask/diagnostics/progress.py
--- 2022.01.0+dfsg-1/dask/diagnostics/progress.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/diagnostics/progress.py	2022-02-11 16:21:12.000000000 +0000
@@ -37,6 +37,10 @@ class ProgressBar(Callback):
         Width of the bar
     dt : float, optional
         Update resolution in seconds, default is 0.1 seconds
+    out : file object, optional
+        File object to which the progress bar will be written
+        It can be ``sys.stdout``, ``sys.stderr`` or any other file object able to write ``str`` objects
+        Default is ``sys.stdout``
 
     Examples
     --------
diff -pruN 2022.01.0+dfsg-1/dask/diagnostics/tests/test_profiler.py 2022.02.0+dfsg-1/dask/diagnostics/tests/test_profiler.py
--- 2022.01.0+dfsg-1/dask/diagnostics/tests/test_profiler.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/diagnostics/tests/test_profiler.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,13 +1,14 @@
 import contextlib
 import os
 from operator import add, mul
-from time import sleep
 
 import pytest
 
 from dask.diagnostics import CacheProfiler, Profiler, ResourceProfiler
+from dask.diagnostics.profile_visualize import BOKEH_VERSION
 from dask.threaded import get
 from dask.utils import apply, tmpfile
+from dask.utils_test import slowadd
 
 try:
     import bokeh
@@ -21,10 +22,8 @@ except ImportError:
 
 prof = Profiler()
 
-
 dsk = {"a": 1, "b": 2, "c": (add, "a", "b"), "d": (mul, "a", "b"), "e": (mul, "c", "d")}
-
-dsk2 = {"a": 1, "b": 2, "c": (lambda a, b: sleep(0.1) or (a + b), "a", "b")}
+dsk2 = {"a": 1, "b": 2, "c": (slowadd, "a", "b")}
 
 
 def test_profiler():
@@ -220,15 +219,19 @@ def test_profiler_plot():
     with prof:
         get(dsk, "e")
     p = prof.visualize(
-        plot_width=500,
-        plot_height=300,
+        width=500,
+        height=300,
         tools="hover",
         title="Not the default",
         show=False,
         save=False,
     )
-    assert p.plot_width == 500
-    assert p.plot_height == 300
+    if BOKEH_VERSION().major < 3:
+        assert p.plot_width == 500
+        assert p.plot_height == 300
+    else:
+        assert p.width == 500
+        assert p.height == 300
     assert len(p.tools) == 1
     assert isinstance(p.tools[0], bokeh.models.HoverTool)
     assert p.title.text == "Not the default"
@@ -246,15 +249,19 @@ def test_resource_profiler_plot():
     with ResourceProfiler(dt=0.01) as rprof:
         get(dsk2, "c")
     p = rprof.visualize(
-        plot_width=500,
-        plot_height=300,
+        width=500,
+        height=300,
         tools="hover",
         title="Not the default",
         show=False,
         save=False,
     )
-    assert p.plot_width == 500
-    assert p.plot_height == 300
+    if BOKEH_VERSION().major < 3:
+        assert p.plot_width == 500
+        assert p.plot_height == 300
+    else:
+        assert p.width == 500
+        assert p.height == 300
     assert len(p.tools) == 1
     assert isinstance(p.tools[0], bokeh.models.HoverTool)
     assert p.title.text == "Not the default"
@@ -280,15 +287,19 @@ def test_cache_profiler_plot():
     with CacheProfiler(metric_name="non-standard") as cprof:
         get(dsk, "e")
     p = cprof.visualize(
-        plot_width=500,
-        plot_height=300,
+        width=500,
+        height=300,
         tools="hover",
         title="Not the default",
         show=False,
         save=False,
     )
-    assert p.plot_width == 500
-    assert p.plot_height == 300
+    if BOKEH_VERSION().major < 3:
+        assert p.plot_width == 500
+        assert p.plot_height == 300
+    else:
+        assert p.width == 500
+        assert p.height == 300
     assert len(p.tools) == 1
     assert isinstance(p.tools[0], bokeh.models.HoverTool)
     assert p.title.text == "Not the default"
diff -pruN 2022.01.0+dfsg-1/dask/graph_manipulation.py 2022.02.0+dfsg-1/dask/graph_manipulation.py
--- 2022.01.0+dfsg-1/dask/graph_manipulation.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/graph_manipulation.py	2022-02-11 16:21:12.000000000 +0000
@@ -2,19 +2,11 @@
 output collections produced by this module are typically not functionally equivalent to
 their inputs.
 """
+from __future__ import annotations
+
 import uuid
-from numbers import Number
-from typing import (
-    AbstractSet,
-    Callable,
-    Dict,
-    Hashable,
-    Optional,
-    Set,
-    Tuple,
-    TypeVar,
-    Union,
-)
+from collections.abc import Callable, Hashable, Set
+from typing import TYPE_CHECKING, TypeVar
 
 from .base import (
     clone_key,
@@ -29,18 +21,18 @@ from .core import flatten
 from .delayed import Delayed, delayed
 from .highlevelgraph import HighLevelGraph, Layer, MaterializedLayer
 
+if TYPE_CHECKING:
+    from typing_extensions import Literal
+
 __all__ = ("bind", "checkpoint", "clone", "wait_on")
 
 T = TypeVar("T")
-try:
-    from typing import Literal  # Python >= 3.8
-
-    SplitEvery = Union[Number, Literal[False], None]
-except ImportError:
-    SplitEvery = Union[Number, bool, None]  # type: ignore
 
 
-def checkpoint(*collections, split_every: SplitEvery = None) -> Delayed:
+def checkpoint(
+    *collections,
+    split_every: float | Literal[False] | None = None,
+) -> Delayed:
     """Build a :doc:`delayed` which waits until all chunks of the input collection(s)
     have been computed before returning None.
 
@@ -125,7 +117,7 @@ def _checkpoint_one(collection, split_ev
     return Delayed(name, dsk)
 
 
-def _can_apply_blockwise(collection):
+def _can_apply_blockwise(collection) -> bool:
     """Return True if _map_blocks can be sped up via blockwise operations; False
     otherwise.
 
@@ -159,7 +151,7 @@ def _build_map_layer(
     prev_name: str,
     new_name: str,
     collection,
-    dependencies: Tuple[Delayed, ...] = (),
+    dependencies: tuple[Delayed, ...] = (),
 ) -> Layer:
     """Apply func to all keys of collection. Create a Blockwise layer whenever possible;
     fall back to MaterializedLayer otherwise.
@@ -219,7 +211,7 @@ def bind(
     omit=None,
     seed: Hashable = None,
     assume_layers: bool = True,
-    split_every: SplitEvery = None,
+    split_every: float | Literal[False] | None = None,
 ) -> T:
     """
     Make ``children`` collection(s), optionally omitting sub-collections, dependent on
@@ -317,9 +309,9 @@ def bind(
 
 def _bind_one(
     child: T,
-    blocker: Optional[Delayed],
-    omit_layers: Set[str],
-    omit_keys: Set[Hashable],
+    blocker: Delayed | None,
+    omit_layers: set[str],
+    omit_keys: set[Hashable],
     seed: Hashable,
 ) -> T:
     prev_coll_names = get_collection_names(child)
@@ -329,8 +321,8 @@ def _bind_one(
         return child
 
     dsk = child.__dask_graph__()  # type: ignore
-    new_layers: Dict[str, Layer] = {}
-    new_deps: Dict[str, AbstractSet[str]] = {}
+    new_layers: dict[str, Layer] = {}
+    new_deps: dict[str, Set[str]] = {}
 
     if isinstance(dsk, HighLevelGraph):
         try:
@@ -465,7 +457,10 @@ def clone(*collections, omit=None, seed:
     return out[0] if len(collections) == 1 else out
 
 
-def wait_on(*collections, split_every: SplitEvery = None):
+def wait_on(
+    *collections,
+    split_every: float | Literal[False] | None = None,
+):
     """Ensure that all chunks of all input collections have been computed before
     computing the dependents of any of the chunks.
 
diff -pruN 2022.01.0+dfsg-1/dask/highlevelgraph.py 2022.02.0+dfsg-1/dask/highlevelgraph.py
--- 2022.01.0+dfsg-1/dask/highlevelgraph.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/highlevelgraph.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,25 +1,17 @@
+from __future__ import annotations
+
 import abc
-import collections.abc
 import copy
 import html
-from typing import (
-    AbstractSet,
-    Any,
-    Dict,
-    Hashable,
-    Iterable,
-    Mapping,
-    MutableMapping,
-    Optional,
-    Tuple,
-)
+from collections.abc import Hashable, Iterable, Mapping, MutableMapping, Set
+from typing import Any
 
 import tlz as toolz
 
 from . import config
 from .base import clone_key, flatten, is_dask_collection
 from .core import keys_in_tasks, reverse_dict
-from .utils import _deprecated, ensure_dict, key_split, stringify
+from .utils import ensure_dict, key_split, stringify
 from .utils_test import add, inc  # noqa: F401
 from .widgets import get_template
 
@@ -41,7 +33,7 @@ def compute_layer_dependencies(layers):
     return ret
 
 
-class Layer(collections.abc.Mapping):
+class Layer(Mapping):
     """High level graph layer
 
     This abstract class establish a protocol for high level graph layers.
@@ -57,8 +49,8 @@ class Layer(collections.abc.Mapping):
     implementations.
     """
 
-    annotations: Optional[Mapping[str, Any]]
-    collection_annotations: Optional[Mapping[str, Any]]
+    annotations: Mapping[str, Any] | None
+    collection_annotations: Mapping[str, Any] | None
 
     def __init__(
         self,
@@ -92,7 +84,7 @@ class Layer(collections.abc.Mapping):
         return True
 
     @abc.abstractmethod
-    def get_output_keys(self) -> AbstractSet:
+    def get_output_keys(self) -> Set:
         """Return a set of all output keys
 
         Output keys are all keys in the layer that might be referenced by
@@ -103,14 +95,14 @@ class Layer(collections.abc.Mapping):
 
         Returns
         -------
-        keys: AbstractSet
+        keys: Set
             All output keys
         """
         return self.keys()  # this implementation will materialize the graph
 
     def cull(
         self, keys: set, all_hlg_keys: Iterable
-    ) -> Tuple["Layer", Mapping[Hashable, set]]:
+    ) -> tuple[Layer, Mapping[Hashable, set]]:
         """Remove unnecessary tasks from the layer
 
         In other words, return a new Layer with only the tasks required to
@@ -172,8 +164,8 @@ class Layer(collections.abc.Mapping):
         return keys_in_tasks(all_hlg_keys, [self[key]])
 
     def __dask_distributed_annotations_pack__(
-        self, annotations: Mapping[str, Any] = None
-    ) -> Optional[Mapping[str, Any]]:
+        self, annotations: Mapping[str, Any] | None = None
+    ) -> Mapping[str, Any] | None:
         """Packs Layer annotations for transmission to scheduler
 
         Callables annotations are fully expanded over Layer keys, while
@@ -202,7 +194,7 @@ class Layer(collections.abc.Mapping):
     @staticmethod
     def __dask_distributed_annotations_unpack__(
         annotations: MutableMapping[str, Any],
-        new_annotations: Optional[Mapping[str, Any]],
+        new_annotations: Mapping[str, Any] | None,
         keys: Iterable[Hashable],
     ) -> None:
         """
@@ -252,7 +244,7 @@ class Layer(collections.abc.Mapping):
         keys: set,
         seed: Hashable,
         bind_to: Hashable = None,
-    ) -> "tuple[Layer, bool]":
+    ) -> tuple[Layer, bool]:
         """Clone selected keys in the layer, as well as references to keys in other
         layers
 
@@ -439,7 +431,7 @@ class Layer(collections.abc.Mapping):
         state: Any,
         dsk: Mapping[str, Any],
         dependencies: Mapping[str, set],
-    ) -> Dict:
+    ) -> dict:
         """Unpack the state of a layer previously packed by __dask_distributed_pack__()
 
         This method is called by the scheduler in Distributed in order to unpack
@@ -622,16 +614,16 @@ class HighLevelGraph(Mapping):
     """
 
     layers: Mapping[str, Layer]
-    dependencies: Mapping[str, AbstractSet]
-    key_dependencies: Dict[Hashable, AbstractSet]
+    dependencies: Mapping[str, Set]
+    key_dependencies: dict[Hashable, Set]
     _to_dict: dict
     _all_external_keys: set
 
     def __init__(
         self,
         layers: Mapping[str, Mapping],
-        dependencies: Mapping[str, AbstractSet],
-        key_dependencies: Optional[Dict[Hashable, AbstractSet]] = None,
+        dependencies: Mapping[str, Set],
+        key_dependencies: dict[Hashable, Set] | None = None,
     ):
         self.dependencies = dependencies
         self.key_dependencies = key_dependencies or {}
@@ -757,7 +749,7 @@ class HighLevelGraph(Mapping):
             out = self._to_dict = ensure_dict(self)
             return out
 
-    def keys(self) -> AbstractSet:
+    def keys(self) -> Set:
         """Get all keys of all the layers.
 
         This will in many cases materialize layers, which makes it a relatively
@@ -765,11 +757,6 @@ class HighLevelGraph(Mapping):
         """
         return self.to_dict().keys()
 
-    @_deprecated(use_instead="HighLevelGraph.keys")
-    def keyset(self) -> AbstractSet:
-        # Backwards compatibility for now
-        return self.keys()
-
     def get_all_external_keys(self) -> set:
         """Get all output keys of all layers
 
@@ -799,7 +786,7 @@ class HighLevelGraph(Mapping):
     def values(self):
         return self.to_dict().values()
 
-    def get_all_dependencies(self) -> Dict[Hashable, AbstractSet]:
+    def get_all_dependencies(self) -> dict[Hashable, Set]:
         """Get dependencies of all keys
 
         This will in most cases materialize all layers, which makes
@@ -822,12 +809,6 @@ class HighLevelGraph(Mapping):
     def dependents(self):
         return reverse_dict(self.dependencies)
 
-    @property
-    @_deprecated(use_instead="HighLevelGraph.layers")
-    def dicts(self):
-        # Backwards compatibility for now
-        return self.layers
-
     def copy(self):
         return HighLevelGraph(
             ensure_dict(self.layers, copy=True),
@@ -925,7 +906,7 @@ class HighLevelGraph(Mapping):
                     ready.append(rdep)
         return ret
 
-    def cull(self, keys: Iterable) -> "HighLevelGraph":
+    def cull(self, keys: Iterable) -> HighLevelGraph:
         """Return new HighLevelGraph with only the tasks required to calculate keys.
 
         In other words, remove unnecessary tasks from dask.
@@ -979,7 +960,7 @@ class HighLevelGraph(Mapping):
 
         return HighLevelGraph(ret_layers, ret_dependencies, ret_key_deps)
 
-    def cull_layers(self, layers: Iterable[str]) -> "HighLevelGraph":
+    def cull_layers(self, layers: Iterable[str]) -> HighLevelGraph:
         """Return a new HighLevelGraph with only the given layers and their
         dependencies. Internally, layers are not modified.
 
@@ -1099,11 +1080,11 @@ class HighLevelGraph(Mapping):
         Returns
         -------
         unpacked-graph: dict
-            dsk: Dict[str, Any]
+            dsk: dict[str, Any]
                 Materialized (stringified) graph of all nodes in the high level graph
-            deps: Dict[str, set]
+            deps: dict[str, set]
                 Dependencies of each key in `dsk`
-            annotations: Dict[str, Any]
+            annotations: dict[str, Any]
                 Annotations for `dsk`
         """
         from distributed.protocol.serialize import import_allowed_module
@@ -1195,7 +1176,6 @@ def to_graphviz(
             "BroadcastJoinLayer": ["#D9F2FF", False],  # blue
             "Blockwise": ["#D9FFE6", False],  # green
             "BlockwiseLayer": ["#D9FFE6", False],  # green
-            "BlockwiseCreateArray": ["#D9FFE6", False],  # green
             "MaterializedLayer": ["#DBDEE5", False],  # gray
         }
 
diff -pruN 2022.01.0+dfsg-1/dask/layers.py 2022.02.0+dfsg-1/dask/layers.py
--- 2022.01.0+dfsg-1/dask/layers.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/layers.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,8 +1,11 @@
+from __future__ import annotations
+
+import functools
+import math
 import operator
 from collections import defaultdict
-from functools import partial
 from itertools import product
-from typing import List, Optional, Tuple
+from typing import Any
 
 import tlz as toolz
 from tlz.curried import map
@@ -11,7 +14,14 @@ from .base import tokenize
 from .blockwise import Blockwise, BlockwiseDep, BlockwiseDepDict, blockwise_token
 from .core import flatten, keys_in_tasks
 from .highlevelgraph import Layer
-from .utils import apply, concrete, insert, stringify, stringify_collection_keys
+from .utils import (
+    apply,
+    cached_cumsum,
+    concrete,
+    insert,
+    stringify,
+    stringify_collection_keys,
+)
 
 #
 ##
@@ -43,19 +53,26 @@ class CallableLazyImport:
 #
 
 
-class CreateArrayDeps(BlockwiseDep):
-    """Index-chunk mapping for BlockwiseCreateArray"""
+class ArrayBlockwiseDep(BlockwiseDep):
+    """
+    Blockwise dep for array-likes, which only needs chunking
+    information to compute its data.
+    """
 
-    def __init__(self, chunks: tuple):
+    chunks: tuple[tuple[int, ...], ...]
+    numblocks: tuple[int, ...]
+    produces_tasks: bool = False
+
+    def __init__(self, chunks: tuple[tuple[int, ...], ...]):
         self.chunks = chunks
         self.numblocks = tuple(len(chunk) for chunk in chunks)
         self.produces_tasks = False
 
-    def __getitem__(self, idx: tuple):
-        return tuple(chunk[i] for i, chunk in zip(idx, self.chunks))
+    def __getitem__(self, idx: tuple[int, ...]):
+        raise NotImplementedError("Subclasses must implement __getitem__")
 
     def __dask_distributed_pack__(
-        self, required_indices: Optional[List[Tuple[int, ...]]] = None
+        self, required_indices: list[tuple[int, ...]] | None = None
     ):
         return {"chunks": self.chunks}
 
@@ -64,43 +81,25 @@ class CreateArrayDeps(BlockwiseDep):
         return cls(**state)
 
 
-class BlockwiseCreateArray(Blockwise):
-    """
-    Specialized Blockwise Layer for array creation routines.
+class ArrayChunkShapeDep(ArrayBlockwiseDep):
+    """Produce chunk shapes given a chunk index"""
 
-    Enables HighLevelGraph optimizations.
+    def __getitem__(self, idx: tuple[int, ...]):
+        return tuple(chunk[i] for i, chunk in zip(idx, self.chunks))
 
-    Parameters
-    ----------
-    name: string
-        The output name.
-    func : callable
-        Function to apply to populate individual blocks. This function should take
-        an iterable containing the dimensions of the given block.
-    shape: iterable
-        Iterable containing the overall shape of the array.
-    chunks: iterable
-        Iterable containing the chunk sizes along each dimension of array.
-    """
 
-    def __init__(
-        self,
-        name,
-        func,
-        shape,
-        chunks,
-    ):
-        # Define "blockwise" graph
-        dsk = {name: (func, blockwise_token(0))}
+class ArraySliceDep(ArrayBlockwiseDep):
+    """Produce slice(s) into the full-sized array given a chunk index"""
 
-        out_ind = tuple(range(len(shape)))
-        super().__init__(
-            output=name,
-            output_indices=out_ind,
-            dsk=dsk,
-            indices=[(CreateArrayDeps(chunks), out_ind)],
-            numblocks={},
-        )
+    starts: tuple[tuple[int, ...], ...]
+
+    def __init__(self, chunks: tuple[tuple[int, ...], ...]):
+        super().__init__(chunks)
+        self.starts = tuple(cached_cumsum(c, initial_zero=True) for c in chunks)
+
+    def __getitem__(self, idx: tuple):
+        loc = tuple((start[i], start[i + 1]) for i, start in zip(idx, self.starts))
+        return tuple(slice(*s, None) for s in loc)
 
 
 class ArrayOverlapLayer(Layer):
@@ -200,7 +199,9 @@ class ArrayOverlapLayer(Layer):
             from dask.array.core import concatenate3
 
         dims = list(map(len, chunks))
-        expand_key2 = partial(_expand_keys_around_center, dims=dims, axes=axes)
+        expand_key2 = functools.partial(
+            _expand_keys_around_center, dims=dims, axes=axes
+        )
 
         # Make keys for each of the surrounding sub-arrays
         interior_keys = toolz.pipe(
@@ -342,26 +343,7 @@ def fractional_slice(task, axes):
 #
 
 
-class DataFrameLayer(Layer):
-    """DataFrame-based HighLevelGraph Layer"""
-
-    def project_columns(self, output_columns):
-        """Produce a column projection for this layer.
-        Given a list of required output columns, this method
-        returns a tuple with the projected layer, and any column
-        dependencies for this layer.  A value of ``None`` for
-        ``output_columns`` means that the current layer (and
-        any dependent layers) cannot be projected. This method
-        should be overridden by specialized DataFrame layers
-        to enable column projection.
-        """
-
-        # Default behavior.
-        # Return: `projected_layer`, `dep_columns`
-        return self, None
-
-
-class SimpleShuffleLayer(DataFrameLayer):
+class SimpleShuffleLayer(Layer):
     """Simple HighLevelGraph Shuffle layer
 
     High-level graph layer for a simple shuffle operation in which
@@ -852,7 +834,7 @@ class ShuffleLayer(SimpleShuffleLayer):
         return dsk
 
 
-class BroadcastJoinLayer(DataFrameLayer):
+class BroadcastJoinLayer(Layer):
     """Broadcast-based Join Layer
 
     High-level graph layer for a join operation requiring the
@@ -1147,7 +1129,7 @@ class BroadcastJoinLayer(DataFrameLayer)
         return dsk
 
 
-class DataFrameIOLayer(Blockwise, DataFrameLayer):
+class DataFrameIOLayer(Blockwise):
     """DataFrame-based Blockwise Layer with IO
 
     Parameters
@@ -1173,6 +1155,11 @@ class DataFrameIOLayer(Blockwise, DataFr
         contain a nested task. This argument in only used for
         serialization purposes, and will be deprecated in the
         future. Default is False.
+    creation_info: dict (optional)
+        Dictionary containing the callable function ('func'),
+        positional arguments ('args'), and key-word arguments
+        ('kwargs') used to produce the dask collection with
+        this underlying ``DataFrameIOLayer``.
     annotations: dict (optional)
         Layer annotations to pass through to Blockwise.
     """
@@ -1185,6 +1172,7 @@ class DataFrameIOLayer(Blockwise, DataFr
         io_func,
         label=None,
         produces_tasks=False,
+        creation_info=None,
         annotations=None,
     ):
         self.name = name
@@ -1194,6 +1182,7 @@ class DataFrameIOLayer(Blockwise, DataFr
         self.label = label
         self.produces_tasks = produces_tasks
         self.annotations = annotations
+        self.creation_info = creation_info
 
         # Define mapping between key index and "part"
         io_arg_map = BlockwiseDepDict(
@@ -1213,7 +1202,10 @@ class DataFrameIOLayer(Blockwise, DataFr
         )
 
     def project_columns(self, columns):
-        # Method inherited from `DataFrameLayer.project_columns`
+        """Produce a column projection for this IO layer.
+        Given a list of required output columns, this method
+        returns the projected layer.
+        """
         if columns and (self.columns is None or columns < set(self.columns)):
 
             # Apply column projection in IO function
@@ -1227,15 +1219,325 @@ class DataFrameIOLayer(Blockwise, DataFr
                 list(columns),
                 self.inputs,
                 io_func,
+                label=self.label,
                 produces_tasks=self.produces_tasks,
                 annotations=self.annotations,
             )
-            return layer, None
+            return layer
         else:
             # Default behavior
-            return self, None
+            return self
 
     def __repr__(self):
         return "DataFrameIOLayer<name='{}', n_parts={}, columns={}>".format(
             self.name, len(self.inputs), self.columns
         )
+
+
+class DataFrameTreeReduction(Layer):
+    """DataFrame Tree-Reduction Layer
+
+    Parameters
+    ----------
+    name : str
+        Name to use for the constructed layer.
+    name_input : str
+        Name of the input layer that is being reduced.
+    npartitions_input : str
+        Number of partitions in the input layer.
+    concat_func : callable
+        Function used by each tree node to reduce a list of inputs
+        into a single output value. This function must accept only
+        a list as its first positional argument.
+    tree_node_func : callable
+        Function used on the output of ``concat_func`` in each tree
+        node. This function must accept the output of ``concat_func``
+        as its first positional argument.
+    finalize_func : callable, optional
+        Function used in place of ``tree_node_func`` on the final tree
+        node(s) to produce the final output for each split. By default,
+        ``tree_node_func`` will be used.
+    split_every : int, optional
+        This argument specifies the maximum number of input nodes
+        to be handled by any one task in the tree. Defaults to 32.
+    split_out : int, optional
+        This argument specifies the number of output nodes in the
+        reduction tree. If ``split_out`` is set to an integer >=1, the
+        input tasks must contain data that can be indexed by a ``getitem``
+        operation with a key in the range ``[0, split_out)``.
+    output_partitions : list, optional
+        List of required output partitions. This parameter is used
+        internally by Dask for high-level culling.
+    tree_node_name : str, optional
+        Name to use for intermediate tree-node tasks.
+    """
+
+    name: str
+    name_input: str
+    npartitions_input: str
+    concat_func: callable
+    tree_node_func: callable
+    finalize_func: callable | None
+    split_every: int
+    split_out: int
+    output_partitions: list[int]
+    tree_node_name: str
+    widths: list[int]
+    height: int
+
+    def __init__(
+        self,
+        name: str,
+        name_input: str,
+        npartitions_input: str,
+        concat_func: callable,
+        tree_node_func: callable,
+        finalize_func: callable | None = None,
+        split_every: int = 32,
+        split_out: int | None = None,
+        output_partitions: list[int] | None = None,
+        tree_node_name: str | None = None,
+        annotations: dict[str, Any] | None = None,
+    ):
+        super().__init__(annotations=annotations)
+        self.name = name
+        self.name_input = name_input
+        self.npartitions_input = npartitions_input
+        self.concat_func = concat_func
+        self.tree_node_func = tree_node_func
+        self.finalize_func = finalize_func
+        self.split_every = split_every
+        self.split_out = split_out
+        self.output_partitions = (
+            list(range(self.split_out or 1))
+            if output_partitions is None
+            else output_partitions
+        )
+        self.tree_node_name = tree_node_name or "tree_node-" + self.name
+
+        # Calculate tree widths and height
+        # (Used to get output keys without materializing)
+        parts = self.npartitions_input
+        self.widths = [parts]
+        while parts > 1:
+            parts = math.ceil(parts / self.split_every)
+            self.widths.append(parts)
+        self.height = len(self.widths)
+
+    def _make_key(self, *name_parts, split=0):
+        # Helper function construct a key
+        # with a "split" element when
+        # bool(split_out) is True
+        return name_parts + (split,) if self.split_out else name_parts
+
+    def _define_task(self, input_keys, final_task=False):
+        # Define nested concatenation and func task
+        if final_task and self.finalize_func:
+            outer_func = self.finalize_func
+        else:
+            outer_func = self.tree_node_func
+        return (toolz.pipe, input_keys, self.concat_func, outer_func)
+
+    def _construct_graph(self):
+        """Construct graph for a tree reduction."""
+
+        dsk = {}
+        if not self.output_partitions:
+            return dsk
+
+        # Deal with `bool(split_out) == True`.
+        # These cases require that the input tasks
+        # return a type that enables getitem operation
+        # with indices: [0, split_out)
+        # Therefore, we must add "getitem" tasks to
+        # select the appropriate element for each split
+        name_input_use = self.name_input
+        if self.split_out:
+            name_input_use += "-split"
+            for s in self.output_partitions:
+                for p in range(self.npartitions_input):
+                    dsk[self._make_key(name_input_use, p, split=s)] = (
+                        operator.getitem,
+                        (self.name_input, p),
+                        s,
+                    )
+
+        if self.height >= 2:
+            # Loop over output splits
+            for s in self.output_partitions:
+                # Loop over reduction levels
+                for depth in range(1, self.height):
+                    # Loop over reduction groups
+                    for group in range(self.widths[depth]):
+                        # Calculate inputs for the current group
+                        p_max = self.widths[depth - 1]
+                        lstart = self.split_every * group
+                        lstop = min(lstart + self.split_every, p_max)
+                        if depth == 1:
+                            # Input nodes are from input layer
+                            input_keys = [
+                                self._make_key(name_input_use, p, split=s)
+                                for p in range(lstart, lstop)
+                            ]
+                        else:
+                            # Input nodes are tree-reduction nodes
+                            input_keys = [
+                                self._make_key(
+                                    self.tree_node_name, p, depth - 1, split=s
+                                )
+                                for p in range(lstart, lstop)
+                            ]
+
+                        # Define task
+                        if depth == self.height - 1:
+                            # Final Node (Use fused `self.tree_finalize` task)
+                            assert (
+                                group == 0
+                            ), f"group = {group}, not 0 for final tree reduction task"
+                            dsk[(self.name, s)] = self._define_task(
+                                input_keys, final_task=True
+                            )
+                        else:
+                            # Intermediate Node
+                            dsk[
+                                self._make_key(
+                                    self.tree_node_name, group, depth, split=s
+                                )
+                            ] = self._define_task(input_keys, final_task=False)
+        else:
+            # Deal with single-partition case
+            for s in self.output_partitions:
+                input_keys = [self._make_key(name_input_use, 0, split=s)]
+                dsk[(self.name, s)] = self._define_task(input_keys, final_task=True)
+
+        return dsk
+
+    def __repr__(self):
+        return "DataFrameTreeReduction<name='{}', input_name={}, split_out={}>".format(
+            self.name, self.name_input, self.split_out
+        )
+
+    def _output_keys(self):
+        return {(self.name, s) for s in self.output_partitions}
+
+    def get_output_keys(self):
+        if hasattr(self, "_cached_output_keys"):
+            return self._cached_output_keys
+        else:
+            output_keys = self._output_keys()
+            self._cached_output_keys = output_keys
+        return self._cached_output_keys
+
+    def is_materialized(self):
+        return hasattr(self, "_cached_dict")
+
+    @property
+    def _dict(self):
+        """Materialize full dict representation"""
+        if hasattr(self, "_cached_dict"):
+            return self._cached_dict
+        else:
+            dsk = self._construct_graph()
+            self._cached_dict = dsk
+        return self._cached_dict
+
+    def __getitem__(self, key):
+        return self._dict[key]
+
+    def __iter__(self):
+        return iter(self._dict)
+
+    def __len__(self):
+        # Start with "base" tree-reduction size
+        tree_size = (sum(self.widths[1:]) or 1) * (self.split_out or 1)
+        if self.split_out:
+            # Add on "split-*" tasks used for `getitem` ops
+            return tree_size + self.npartitions_input * len(self.output_partitions)
+        return tree_size
+
+    def _keys_to_output_partitions(self, keys):
+        """Simple utility to convert keys to output partition indices."""
+        splits = set()
+        for key in keys:
+            try:
+                _name, _split = key
+            except ValueError:
+                continue
+            if _name != self.name:
+                continue
+            splits.add(_split)
+        return splits
+
+    def _cull(self, output_partitions):
+        return DataFrameTreeReduction(
+            self.name,
+            self.name_input,
+            self.npartitions_input,
+            self.concat_func,
+            self.tree_node_func,
+            finalize_func=self.finalize_func,
+            split_every=self.split_every,
+            split_out=self.split_out,
+            output_partitions=output_partitions,
+            tree_node_name=self.tree_node_name,
+            annotations=self.annotations,
+        )
+
+    def cull(self, keys, all_keys):
+        """Cull a DataFrameTreeReduction HighLevelGraph layer"""
+        deps = {
+            (self.name, 0): {
+                (self.name_input, i) for i in range(self.npartitions_input)
+            }
+        }
+        output_partitions = self._keys_to_output_partitions(keys)
+        if output_partitions != set(self.output_partitions):
+            culled_layer = self._cull(output_partitions)
+            return culled_layer, deps
+        else:
+            return self, deps
+
+    def __dask_distributed_pack__(self, *args, **kwargs):
+        from distributed.protocol.serialize import to_serialize
+
+        # Pickle the (possibly) user-defined functions here
+        _concat_func = to_serialize(self.concat_func)
+        _tree_node_func = to_serialize(self.tree_node_func)
+        if self.finalize_func:
+            _finalize_func = to_serialize(self.finalize_func)
+        else:
+            _finalize_func = None
+
+        return {
+            "name": self.name,
+            "name_input": self.name_input,
+            "npartitions_input": self.npartitions_input,
+            "concat_func": _concat_func,
+            "tree_node_func": _tree_node_func,
+            "finalize_func": _finalize_func,
+            "split_out": self.split_out,
+            "output_partitions": self.output_partitions,
+            "tree_node_name": self.tree_node_name,
+        }
+
+    @classmethod
+    def __dask_distributed_unpack__(cls, state, dsk, dependencies):
+        from distributed.protocol.serialize import to_serialize
+
+        # Materialize the layer
+        raw = cls(**state)._construct_graph()
+
+        # Convert all keys to strings and dump tasks
+        raw = {stringify(k): stringify_collection_keys(v) for k, v in raw.items()}
+        keys = raw.keys() | dsk.keys()
+        deps = {k: keys_in_tasks(keys, [v]) for k, v in raw.items()}
+
+        # Must use `to_serialize` on the entire task.
+        # This is required because the task-tuples contain `Serialized`
+        # function objects instead of real functions. Using `dumps_task`
+        # may or may not correctly wrap the entire tuple in `to_serialize`.
+        # So we use `to_serialize` here to be explicit. When the task
+        # arrives at a worker, both the `Serialized` task-tuples and the
+        # `Serialized` functions nested within them should be deserialzed
+        # automatically by the comm.
+        return {"dsk": toolz.valmap(to_serialize, raw), "deps": deps}
diff -pruN 2022.01.0+dfsg-1/dask/local.py 2022.02.0+dfsg-1/dask/local.py
--- 2022.01.0+dfsg-1/dask/local.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/local.py	2022-02-11 16:21:12.000000000 +0000
@@ -127,7 +127,6 @@ if os.name == "nt":
             except Empty:
                 pass
 
-
 else:
 
     def queue_get(q):
diff -pruN 2022.01.0+dfsg-1/dask/multiprocessing.py 2022.02.0+dfsg-1/dask/multiprocessing.py
--- 2022.01.0+dfsg-1/dask/multiprocessing.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/multiprocessing.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,3 +1,5 @@
+from __future__ import annotations
+
 import copyreg
 import multiprocessing
 import multiprocessing.pool
@@ -69,10 +71,10 @@ class RemoteException(Exception):
             return getattr(self.exception, key)
 
 
-exceptions = dict()
+exceptions: dict[type[Exception], type[Exception]] = {}
 
 
-def remote_exception(exc, tb):
+def remote_exception(exc: Exception, tb) -> Exception:
     """Metaclass that wraps exception type in RemoteException"""
     if type(exc) in exceptions:
         typ = exceptions[type(exc)]
@@ -98,13 +100,12 @@ try:
     def _pack_traceback(tb):
         return tb
 
-
 except ImportError:
 
     def _pack_traceback(tb):
         return "".join(traceback.format_tb(tb))
 
-    def reraise(exc, tb):
+    def reraise(exc, tb=None):
         exc = remote_exception(exc, tb)
         raise exc
 
diff -pruN 2022.01.0+dfsg-1/dask/sizeof.py 2022.02.0+dfsg-1/dask/sizeof.py
--- 2022.01.0+dfsg-1/dask/sizeof.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/sizeof.py	2022-02-11 16:21:12.000000000 +0000
@@ -10,7 +10,7 @@ try:  # PyPy does not support sys.getsiz
     getsizeof = sys.getsizeof
 except (AttributeError, TypeError):  # Monkey patch
 
-    def getsizeof(x):
+    def getsizeof(x):  # type: ignore
         return 100
 
 
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_base.py 2022.02.0+dfsg-1/dask/tests/test_base.py
--- 2022.01.0+dfsg-1/dask/tests/test_base.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_base.py	2022-02-11 16:21:12.000000000 +0000
@@ -5,6 +5,7 @@ import sys
 import time
 from collections import OrderedDict
 from concurrent.futures import Executor
+from dataclasses import dataclass
 from operator import add, mul
 
 import pytest
@@ -509,13 +510,9 @@ def test_is_dask_collection():
     assert not is_dask_collection(DummyCollection)
 
 
-try:
-    import dataclasses
-
-    # Avoid @dataclass decorator as Python < 3.7 fail to interpret the type hints
-    ADataClass = dataclasses.make_dataclass("ADataClass", [("a", int)])
-except ImportError:
-    dataclasses = None
+@dataclass
+class ADataClass:
+    a: int
 
 
 def test_unpack_collections():
@@ -539,9 +536,8 @@ def test_unpack_collections():
             iterator,
         )  # Iterator
 
-        if dataclasses is not None:
-            t[2]["f"] = ADataClass(a=a)
-            t[2]["g"] = (ADataClass, a)
+        t[2]["f"] = ADataClass(a=a)
+        t[2]["g"] = (ADataClass, a)
 
         return t
 
@@ -606,15 +602,12 @@ def test_get_collection_names():
     h1 = object()
     h2 = object()
     # __dask_keys__() returns a nested list
-    assert (
-        get_collection_names(
-            DummyCollection(
-                {("a-1", h1): 1, ("a-1", h2): 2, "b-2": 3, "c": 4},
-                [[[("a-1", h1), ("a-1", h2), "b-2", "c"]]],
-            )
+    assert get_collection_names(
+        DummyCollection(
+            {("a-1", h1): 1, ("a-1", h2): 2, "b-2": 3, "c": 4},
+            [[[("a-1", h1), ("a-1", h2), "b-2", "c"]]],
         )
-        == {"a-1", "b-2", "c"}
-    )
+    ) == {"a-1", "b-2", "c"}
 
 
 def test_get_name_from_key():
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_compatibility.py 2022.02.0+dfsg-1/dask/tests/test_compatibility.py
--- 2022.01.0+dfsg-1/dask/tests/test_compatibility.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_compatibility.py	1970-01-01 00:00:00.000000000 +0000
@@ -1,10 +0,0 @@
-import pytest
-
-import dask
-
-
-def test_PY_VERSION_deprecated():
-
-    with pytest.warns(FutureWarning, match="removed in a future release"):
-        from dask.compatibility import PY_VERSION
-    assert PY_VERSION is dask.compatibility._PY_VERSION
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_config.py 2022.02.0+dfsg-1/dask/tests/test_config.py
--- 2022.01.0+dfsg-1/dask/tests/test_config.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_config.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,13 +1,16 @@
 import os
+import site
 import stat
 import sys
 from collections import OrderedDict
 from contextlib import contextmanager
 
 import pytest
+import yaml
 
 import dask.config
 from dask.config import (
+    _get_paths,
     canonical_name,
     collect,
     collect_env,
@@ -21,14 +24,11 @@ from dask.config import (
     refresh,
     rename,
     serialize,
-    set,
     update,
     update_defaults,
 )
 from dask.utils import tmpfile
 
-yaml = pytest.importorskip("yaml")
-
 
 def test_canonical_name():
     c = {"foo-bar": 1, "fizz_buzz": 2}
@@ -237,47 +237,47 @@ def test_ensure_file(tmpdir):
 
 
 def test_set():
-    with set(abc=123):
+    with dask.config.set(abc=123):
         assert config["abc"] == 123
-        with set(abc=456):
+        with dask.config.set(abc=456):
             assert config["abc"] == 456
         assert config["abc"] == 123
 
     assert "abc" not in config
 
-    with set({"abc": 123}):
+    with dask.config.set({"abc": 123}):
         assert config["abc"] == 123
     assert "abc" not in config
 
-    with set({"abc.x": 1, "abc.y": 2, "abc.z.a": 3}):
+    with dask.config.set({"abc.x": 1, "abc.y": 2, "abc.z.a": 3}):
         assert config["abc"] == {"x": 1, "y": 2, "z": {"a": 3}}
     assert "abc" not in config
 
     d = {}
-    set({"abc.x": 123}, config=d)
+    dask.config.set({"abc.x": 123}, config=d)
     assert d["abc"]["x"] == 123
 
 
 def test_set_kwargs():
-    with set(foo__bar=1, foo__baz=2):
+    with dask.config.set(foo__bar=1, foo__baz=2):
         assert config["foo"] == {"bar": 1, "baz": 2}
     assert "foo" not in config
 
     # Mix kwargs and dict, kwargs override
-    with set({"foo.bar": 1, "foo.baz": 2}, foo__buzz=3, foo__bar=4):
+    with dask.config.set({"foo.bar": 1, "foo.baz": 2}, foo__buzz=3, foo__bar=4):
         assert config["foo"] == {"bar": 4, "baz": 2, "buzz": 3}
     assert "foo" not in config
 
     # Mix kwargs and nested dict, kwargs override
-    with set({"foo": {"bar": 1, "baz": 2}}, foo__buzz=3, foo__bar=4):
+    with dask.config.set({"foo": {"bar": 1, "baz": 2}}, foo__buzz=3, foo__bar=4):
         assert config["foo"] == {"bar": 4, "baz": 2, "buzz": 3}
     assert "foo" not in config
 
 
 def test_set_nested():
-    with set({"abc": {"x": 123}}):
+    with dask.config.set({"abc": {"x": 123}}):
         assert config["abc"] == {"x": 123}
-        with set({"abc.y": 456}):
+        with dask.config.set({"abc.y": 456}):
             assert config["abc"] == {"x": 123, "y": 456}
         assert config["abc"] == {"x": 123}
     assert "abc" not in config
@@ -286,8 +286,8 @@ def test_set_nested():
 def test_set_hard_to_copyables():
     import threading
 
-    with set(x=threading.Lock()):
-        with set(y=1):
+    with dask.config.set(x=threading.Lock()):
+        with dask.config.set(y=1):
             pass
 
 
@@ -503,3 +503,45 @@ def test_config_inheritance():
         {"DASK_INTERNAL_INHERIT_CONFIG": serialize({"array": {"svg": {"size": 150}}})}
     )
     assert dask.config.get("array.svg.size", config=config) == 150
+
+
+def test__get_paths(monkeypatch):
+    # These settings are used by Dask's config system. We temporarily
+    # remove them to avoid interference from the machine where tests
+    # are being run.
+    monkeypatch.delenv("DASK_CONFIG", raising=False)
+    monkeypatch.delenv("DASK_ROOT_CONFIG", raising=False)
+    monkeypatch.setattr(site, "PREFIXES", [])
+
+    expected = [
+        "/etc/dask",
+        os.path.join(sys.prefix, "etc", "dask"),
+        os.path.join(os.path.expanduser("~"), ".config", "dask"),
+    ]
+    paths = _get_paths()
+    assert paths == expected
+    assert len(paths) == len(set(paths))  # No duplicate paths
+
+    with monkeypatch.context() as m:
+        m.setenv("DASK_CONFIG", "foo-bar")
+        paths = _get_paths()
+        assert paths == expected + ["foo-bar"]
+        assert len(paths) == len(set(paths))
+
+    with monkeypatch.context() as m:
+        m.setenv("DASK_ROOT_CONFIG", "foo-bar")
+        paths = _get_paths()
+        assert paths == ["foo-bar"] + expected[1:]
+        assert len(paths) == len(set(paths))
+
+    with monkeypatch.context() as m:
+        prefix = os.path.join("include", "this", "path")
+        m.setattr(site, "PREFIXES", site.PREFIXES + [prefix])
+        paths = _get_paths()
+        assert os.path.join(prefix, "etc", "dask") in paths
+        assert len(paths) == len(set(paths))
+
+
+def test_default_search_paths():
+    # Ensure _get_paths() is used for default paths
+    assert dask.config.paths == _get_paths()
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_context.py 2022.02.0+dfsg-1/dask/tests/test_context.py
--- 2022.01.0+dfsg-1/dask/tests/test_context.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_context.py	2022-02-11 16:21:12.000000000 +0000
@@ -36,7 +36,7 @@ def bar():
 
 class Foo:
     @globalmethod(key="f")
-    def f():
+    def f():  # type: ignore
         return 1
 
     g = globalmethod(foo, key="g", falsey=bar)
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_delayed.py 2022.02.0+dfsg-1/dask/tests/test_delayed.py
--- 2022.01.0+dfsg-1/dask/tests/test_delayed.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_delayed.py	2022-02-11 16:21:12.000000000 +0000
@@ -2,7 +2,7 @@ import pickle
 import types
 from collections import namedtuple
 from functools import partial
-from operator import add, setitem
+from operator import add, matmul, setitem
 from random import random
 
 import cloudpickle
@@ -16,11 +16,6 @@ from dask.delayed import Delayed, delaye
 from dask.highlevelgraph import HighLevelGraph
 from dask.utils_test import inc
 
-try:
-    from operator import matmul
-except ImportError:
-    matmul = None
-
 
 class Tuple:
     __dask_scheduler__ = staticmethod(dask.threaded.get)
@@ -128,7 +123,7 @@ def test_operators():
     assert (1 + a).compute() == 11
     assert (a >> 1).compute() == 5
     assert (a > 2).compute()
-    assert (a ** 2).compute() == 100
+    assert (a**2).compute() == 100
 
     if matmul:
 
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_distributed.py 2022.02.0+dfsg-1/dask/tests/test_distributed.py
--- 2022.01.0+dfsg-1/dask/tests/test_distributed.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_distributed.py	2022-02-11 16:21:12.000000000 +0000
@@ -181,9 +181,15 @@ async def test_serializable_groupby_agg(
     df = pd.DataFrame({"x": [1, 2, 3, 4], "y": [1, 0, 1, 0]})
     ddf = dd.from_pandas(df, npartitions=2)
 
-    result = ddf.groupby("y").agg("count")
+    result = ddf.groupby("y").agg("count", split_out=2)
 
-    await c.compute(result)
+    # Check Culling and Compute
+    agg0 = await c.compute(result.partitions[0])
+    agg1 = await c.compute(result.partitions[1])
+    dd.utils.assert_eq(
+        pd.concat([agg0, agg1]),
+        pd.DataFrame({"x": [2, 2], "y": [0, 1]}).set_index("y"),
+    )
 
 
 def test_futures_in_graph(c):
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_graph_manipulation.py 2022.02.0+dfsg-1/dask/tests/test_graph_manipulation.py
--- 2022.01.0+dfsg-1/dask/tests/test_graph_manipulation.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_graph_manipulation.py	2022-02-11 16:21:12.000000000 +0000
@@ -115,8 +115,8 @@ def demo_tuples(layers: bool) -> "tuple[
         {"c": set(), "d": set()},
     )
     if not layers:
-        dsk1 = dsk1.to_dict()
-        dsk2 = dsk2.to_dict()
+        dsk1 = dsk1.to_dict()  # type: ignore
+        dsk2 = dsk2.to_dict()  # type: ignore
 
     return Tuple(dsk1, list(dsk1)), Tuple(dsk2, list(dsk2)), cnt
 
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_highgraph.py 2022.02.0+dfsg-1/dask/tests/test_highgraph.py
--- 2022.01.0+dfsg-1/dask/tests/test_highgraph.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_highgraph.py	2022-02-11 16:21:12.000000000 +0000
@@ -49,20 +49,6 @@ def test_keys_values_items_to_dict_metho
     assert hg.to_dict() == dict(hg)
 
 
-def test_keyset_deprecated():
-    a = {"x": 1, "y": (inc, "x")}
-    hg = HighLevelGraph({"a": a}, {"a": set()})
-    with pytest.warns(FutureWarning, match="HighLevelGraph.keys"):
-        assert hg.keyset() == hg.keys()
-
-
-def test_dicts_deprecated():
-    a = {"x": 1, "y": (inc, "x")}
-    hg = HighLevelGraph({"a": a}, {"a": set()})
-    with pytest.warns(FutureWarning, match="HighLevelGraph.layers"):
-        assert hg.dicts == hg.layers
-
-
 def test_getitem():
     hg = HighLevelGraph(
         {"a": {"a": 1, ("a", 0): 2, "b": 3}, "b": {"c": 4}}, {"a": set(), "b": set()}
@@ -248,13 +234,6 @@ def test_blockwise_cull(flat):
         assert not layer.is_materialized()
 
 
-def test_highlevelgraph_dicts_deprecation():
-    with pytest.warns(FutureWarning):
-        layers = {"a": MaterializedLayer({"x": 1, "y": (inc, "x")})}
-        hg = HighLevelGraph(layers, {"a": set()})
-        assert hg.dicts == layers
-
-
 def test_len_does_not_materialize():
     a = {"x": 1}
     b = Blockwise(
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_layers.py 2022.02.0+dfsg-1/dask/tests/test_layers.py
--- 2022.01.0+dfsg-1/dask/tests/test_layers.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_layers.py	2022-02-11 16:21:12.000000000 +0000
@@ -10,7 +10,7 @@ from operator import getitem
 from distributed import Client, SchedulerPlugin
 from distributed.utils_test import cluster, loop  # noqa F401
 
-from dask.layers import fractional_slice
+from dask.layers import ArrayChunkShapeDep, ArraySliceDep, fractional_slice
 
 
 class SchedulerImportCheck(SchedulerPlugin):
@@ -32,6 +32,43 @@ class SchedulerImportCheck(SchedulerPlug
                 sys.modules.pop(mod)
 
 
+def test_array_chunk_shape_dep():
+    dac = pytest.importorskip("dask.array.core")
+    d = 2  # number of chunks in x,y
+    chunk = (2, 3)  # chunk shape
+    shape = tuple(d * n for n in chunk)  # array shape
+    chunks = dac.normalize_chunks(chunk, shape)
+    array_deps = ArrayChunkShapeDep(chunks)
+
+    def check(i, j):
+        chunk_shape = array_deps[(i, j)]
+        assert chunk_shape == chunk
+
+    for i in range(d):
+        for j in range(d):
+            check(i, j)
+
+
+def test_array_slice_deps():
+    dac = pytest.importorskip("dask.array.core")
+    d = 2  # number of chunks in x,y
+    chunk = (2, 3)  # chunk shape
+    shape = tuple(d * n for n in chunk)  # array shape
+    chunks = dac.normalize_chunks(chunk, shape)
+    array_deps = ArraySliceDep(chunks)
+
+    def check(i, j):
+        slices = array_deps[(i, j)]
+        assert slices == (
+            slice(chunk[0] * i, chunk[0] * (i + 1), None),
+            slice(chunk[1] * j, chunk[1] * (j + 1), None),
+        )
+
+    for i in range(d):
+        for j in range(d):
+            check(i, j)
+
+
 def _dataframe_shuffle(tmpdir):
     pd = pytest.importorskip("pandas")
     dd = pytest.importorskip("dask.dataframe")
@@ -41,6 +78,15 @@ def _dataframe_shuffle(tmpdir):
     return dd.from_pandas(df, npartitions=2).shuffle("a", shuffle="tasks")
 
 
+def _dataframe_tree_reduction(tmpdir):
+    pd = pytest.importorskip("pandas")
+    dd = pytest.importorskip("dask.dataframe")
+
+    # Perform a computation using an HLG-based tree reduction
+    df = pd.DataFrame({"a": range(10), "b": range(10, 20)})
+    return dd.from_pandas(df, npartitions=2).mean()
+
+
 def _dataframe_broadcast_join(tmpdir):
     pd = pytest.importorskip("pandas")
     dd = pytest.importorskip("dask.dataframe")
@@ -104,21 +150,13 @@ def _pq_pyarrow(tmpdir):
     ddf1 = dd.read_parquet(str(tmpdir), engine="pyarrow", filters=filters)
     if pa_ds:
         # Need to test that layer serialization succeeds
-        # with "pyarrow-dataset" filtering (whether or not
-        # `large_graph_objects=True` is specified)
+        # with "pyarrow-dataset" filtering
         ddf2 = dd.read_parquet(
             str(tmpdir),
             engine="pyarrow-dataset",
             filters=filters,
-            large_graph_objects=True,
-        )
-        ddf3 = dd.read_parquet(
-            str(tmpdir),
-            engine="pyarrow-dataset",
-            filters=filters,
-            large_graph_objects=False,
         )
-        return (ddf1, ddf2, ddf3)
+        return (ddf1, ddf2)
     else:
         return ddf1
 
@@ -150,6 +188,7 @@ def _read_csv(tmpdir):
     "op,lib",
     [
         (_dataframe_shuffle, "pandas."),
+        (_dataframe_tree_reduction, "pandas."),
         (_dataframe_broadcast_join, "pandas."),
         (_pq_pyarrow, "pandas."),
         (_pq_fastparquet, "pandas."),
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_threaded.py 2022.02.0+dfsg-1/dask/tests/test_threaded.py
--- 2022.01.0+dfsg-1/dask/tests/test_threaded.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_threaded.py	2022-02-11 16:21:12.000000000 +0000
@@ -159,7 +159,7 @@ def test_interrupt():
     else:
         main_thread = threading.get_ident()
 
-        def interrupt_main():
+        def interrupt_main() -> None:
             signal.pthread_kill(main_thread, signal.SIGINT)
 
     # 7 seconds is is how long the test will take when you factor in teardown.
diff -pruN 2022.01.0+dfsg-1/dask/tests/test_utils.py 2022.02.0+dfsg-1/dask/tests/test_utils.py
--- 2022.01.0+dfsg-1/dask/tests/test_utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/tests/test_utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -15,6 +15,7 @@ from dask.utils import (
     SerializableLock,
     _deprecated,
     asciitable,
+    cached_cumsum,
     derived_from,
     ensure_dict,
     extra_titles,
@@ -22,14 +23,12 @@ from dask.utils import (
     funcname,
     getargspec,
     has_keyword,
-    ignoring,
     is_arraylike,
     itemgetter,
     iter_chunks,
     memory_repr,
     methodcaller,
     ndeepmap,
-    noop_context,
     parse_bytes,
     parse_timedelta,
     partial_by_order,
@@ -219,7 +218,7 @@ def test_random_state_data():
 
 def test_memory_repr():
     for power, mem_repr in enumerate(["1.0 bytes", "1.0 KB", "1.0 MB", "1.0 GB"]):
-        assert memory_repr(1024 ** power) == mem_repr
+        assert memory_repr(1024**power) == mem_repr
 
 
 def test_method_caller():
@@ -558,6 +557,8 @@ def test_derived_from_dask_dataframe():
     assert "not supported" in axis_arg.lower()
     assert "dask" in axis_arg.lower()
 
+    assert "Object with missing values filled" in dd.DataFrame.ffill.__doc__
+
 
 def test_parse_bytes():
     assert parse_bytes("100") == 100
@@ -566,7 +567,7 @@ def test_parse_bytes():
     assert parse_bytes("5kB") == 5000
     assert parse_bytes("5.4 kB") == 5400
     assert parse_bytes("1kiB") == 1024
-    assert parse_bytes("1Mi") == 2 ** 20
+    assert parse_bytes("1Mi") == 2**20
     assert parse_bytes("1e6") == 1000000
     assert parse_bytes("1e6 kB") == 1000000000
     assert parse_bytes("MB") == 1000000
@@ -686,15 +687,15 @@ def test_stringify_collection_keys():
         (0, "0 B"),
         (920, "920 B"),
         (930, "0.91 kiB"),
-        (921.23 * 2 ** 10, "921.23 kiB"),
-        (931.23 * 2 ** 10, "0.91 MiB"),
-        (921.23 * 2 ** 20, "921.23 MiB"),
-        (931.23 * 2 ** 20, "0.91 GiB"),
-        (921.23 * 2 ** 30, "921.23 GiB"),
-        (931.23 * 2 ** 30, "0.91 TiB"),
-        (921.23 * 2 ** 40, "921.23 TiB"),
-        (931.23 * 2 ** 40, "0.91 PiB"),
-        (2 ** 60, "1024.00 PiB"),
+        (921.23 * 2**10, "921.23 kiB"),
+        (931.23 * 2**10, "0.91 MiB"),
+        (921.23 * 2**20, "921.23 MiB"),
+        (931.23 * 2**20, "0.91 GiB"),
+        (921.23 * 2**30, "921.23 GiB"),
+        (931.23 * 2**30, "0.91 TiB"),
+        (921.23 * 2**40, "921.23 TiB"),
+        (931.23 * 2**40, "0.91 PiB"),
+        (2**60, "1024.00 PiB"),
     ],
 )
 def test_format_bytes(n, expect):
@@ -754,18 +755,6 @@ def test_deprecated_message():
     assert str(record[0].message) == "woohoo"
 
 
-def test_ignoring_deprecated():
-    with pytest.warns(FutureWarning, match="contextlib.suppress"):
-        with ignoring(ValueError):
-            pass
-
-
-def test_noop_context_deprecated():
-    with pytest.warns(FutureWarning, match="contextlib.nullcontext"):
-        with noop_context():
-            pass
-
-
 def test_typename():
     assert typename(HighLevelGraph) == "dask.highlevelgraph.HighLevelGraph"
     assert typename(HighLevelGraph, short=True) == "dask.HighLevelGraph"
@@ -778,3 +767,27 @@ class MyType:
 def test_typename_on_instances():
     instance = MyType()
     assert typename(instance) == typename(MyType)
+
+
+def test_cached_cumsum():
+    a = (1, 2, 3, 4)
+    x = cached_cumsum(a)
+    y = cached_cumsum(a, initial_zero=True)
+    assert x == (1, 3, 6, 10)
+    assert y == (0, 1, 3, 6, 10)
+
+
+def test_cached_cumsum_nan():
+    np = pytest.importorskip("numpy")
+    a = (1, np.nan, 3)
+    x = cached_cumsum(a)
+    y = cached_cumsum(a, initial_zero=True)
+    np.testing.assert_equal(x, (1, np.nan, np.nan))
+    np.testing.assert_equal(y, (0, 1, np.nan, np.nan))
+
+
+def test_cached_cumsum_non_tuple():
+    a = [1, 2, 3]
+    assert cached_cumsum(a) == (1, 3, 6)
+    a[1] = 4
+    assert cached_cumsum(a) == (1, 5, 8)
diff -pruN 2022.01.0+dfsg-1/dask/threaded.py 2022.02.0+dfsg-1/dask/threaded.py
--- 2022.01.0+dfsg-1/dask/threaded.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/threaded.py	2022-02-11 16:21:12.000000000 +0000
@@ -3,12 +3,14 @@ A threaded shared-memory scheduler
 
 See local.py
 """
+from __future__ import annotations
+
 import atexit
 import multiprocessing.pool
 import sys
 import threading
 from collections import defaultdict
-from concurrent.futures import ThreadPoolExecutor
+from concurrent.futures import Executor, ThreadPoolExecutor
 from threading import Lock, current_thread
 
 from . import config
@@ -22,8 +24,8 @@ def _thread_get_id():
 
 
 main_thread = current_thread()
-default_pool = None
-pools = defaultdict(dict)
+default_pool: Executor | None = None
+pools: defaultdict[threading.Thread, dict[int, Executor]] = defaultdict(dict)
 pools_lock = Lock()
 
 
diff -pruN 2022.01.0+dfsg-1/dask/utils.py 2022.02.0+dfsg-1/dask/utils.py
--- 2022.01.0+dfsg-1/dask/utils.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/utils.py	2022-02-11 16:21:12.000000000 +0000
@@ -9,16 +9,16 @@ import sys
 import tempfile
 import uuid
 import warnings
-from _thread import RLock
-from collections.abc import Iterable, Iterator, Mapping
+from collections.abc import Hashable, Iterable, Iterator, Mapping
 from contextlib import contextmanager, nullcontext, suppress
 from datetime import datetime, timedelta
 from errno import ENOENT
 from functools import lru_cache
 from importlib import import_module
 from numbers import Integral, Number
-from threading import Lock
-from typing import TypeVar
+from operator import add
+from threading import Lock, RLock
+from typing import ClassVar, TypeVar
 from weakref import WeakValueDictionary
 
 import tlz as toolz
@@ -153,15 +153,6 @@ def ndeepmap(n, func, seq):
         return func(seq)
 
 
-@_deprecated(
-    version="2021.06.1", use_instead="contextlib.suppress from the standard library"
-)
-@contextmanager
-def ignoring(*exceptions):
-    with suppress(*exceptions):
-        yield
-
-
 def import_required(mod_name, error_msg):
     """Attempt to import a required dependency.
 
@@ -282,15 +273,6 @@ def tmp_cwd(dir=None):
             yield dirname
 
 
-@_deprecated(
-    version="2021.06.1", use_instead="contextlib.nullcontext from the standard library"
-)
-@contextmanager
-def noop_context():
-    with nullcontext():
-        yield
-
-
 class IndexCallable:
     """Provide getitem syntax for functions
 
@@ -750,6 +732,14 @@ def _derived_from(cls, method, ua_args=N
     if doc is None:
         doc = ""
 
+    # pandas DataFrame/Series sometimes override methods without setting __doc__
+    if not doc and cls.__name__ in {"DataFrame", "Series"}:
+        for obj in cls.mro():
+            obj_method = getattr(obj, method.__name__, None)
+            if obj_method is not None and obj_method.__doc__:
+                doc = obj_method.__doc__
+                break
+
     # Insert disclaimer that this is a copied docstring
     if doc:
         doc = ignore_warning(
@@ -934,7 +924,7 @@ def digit(n, k, base):
     >>> digit(1234, 3, 10)
     1
     """
-    return n // base ** k % base
+    return n // base**k % base
 
 
 def insert(tup, loc, val):
@@ -996,7 +986,7 @@ def put_lines(buf, lines):
     buf.write("\n".join(lines))
 
 
-_method_cache = {}
+_method_cache: dict[str, methodcaller] = {}
 
 
 class methodcaller:
@@ -1008,7 +998,12 @@ class methodcaller:
     """
 
     __slots__ = ("method",)
-    func = property(lambda self: self.method)  # For `funcname` to work
+    method: str
+
+    @property
+    def func(self) -> str:
+        # For `funcname` to work
+        return self.method
 
     def __new__(cls, method: str):
         try:
@@ -1060,16 +1055,18 @@ class MethodCache:
     True
     """
 
-    __getattr__ = staticmethod(methodcaller)
-    __dir__ = lambda self: list(_method_cache)
+    def __getattr__(self, item):
+        return methodcaller(item)
+
+    def __dir__(self):
+        return list(_method_cache)
 
 
 M = MethodCache()
 
 
 class SerializableLock:
-    _locks = WeakValueDictionary()
-    """ A Serializable per-process Lock
+    """A Serializable per-process Lock
 
     This wraps a normal ``threading.Lock`` object and satisfies the same
     interface.  However, this lock can also be serialized and sent to different
@@ -1096,7 +1093,11 @@ class SerializableLock:
     The creation of locks is itself not threadsafe.
     """
 
-    def __init__(self, token=None):
+    _locks: ClassVar[WeakValueDictionary[Hashable, Lock]] = WeakValueDictionary()
+    token: Hashable
+    lock: Lock
+
+    def __init__(self, token: Hashable | None = None):
         self.token = token or str(uuid.uuid4())
         if self.token in SerializableLock._locks:
             self.lock = SerializableLock._locks[self.token]
@@ -1403,16 +1404,16 @@ def parse_bytes(s):
 
 
 byte_sizes = {
-    "kB": 10 ** 3,
-    "MB": 10 ** 6,
-    "GB": 10 ** 9,
-    "TB": 10 ** 12,
-    "PB": 10 ** 15,
-    "KiB": 2 ** 10,
-    "MiB": 2 ** 20,
-    "GiB": 2 ** 30,
-    "TiB": 2 ** 40,
-    "PiB": 2 ** 50,
+    "kB": 10**3,
+    "MB": 10**6,
+    "GB": 10**9,
+    "TB": 10**12,
+    "PB": 10**15,
+    "KiB": 2**10,
+    "MiB": 2**20,
+    "GiB": 2**30,
+    "TiB": 2**40,
+    "PiB": 2**50,
     "B": 1,
     "": 1,
 }
@@ -1531,11 +1532,11 @@ def format_bytes(n: int) -> str:
     For all values < 2**60, the output is always <= 10 characters.
     """
     for prefix, k in (
-        ("Pi", 2 ** 50),
-        ("Ti", 2 ** 40),
-        ("Gi", 2 ** 30),
-        ("Mi", 2 ** 20),
-        ("ki", 2 ** 10),
+        ("Pi", 2**50),
+        ("Ti", 2**40),
+        ("Gi", 2**30),
+        ("Mi", 2**20),
+        ("ki", 2**10),
     ):
         if n >= k * 0.9:
             return f"{n / k:.2f} {prefix}B"
@@ -1809,11 +1810,11 @@ def stringify_collection_keys(obj):
 try:
     _cached_property = functools.cached_property
 except AttributeError:
-    # TODO: Copied from functools.cached_property in python 3.8. Remove when minimum
-    # supported python version is 3.8:
+    # TODO: Copied from functools.cached_property in python 3.8.
+    #       Remove when minimum supported python version is 3.8:
     _NOT_FOUND = object()
 
-    class _cached_property:
+    class _cached_property:  # type: ignore
         def __init__(self, func):
             self.func = func
             self.attrname = None
@@ -1868,3 +1869,62 @@ class cached_property(_cached_property):
     def __set__(self, instance, val):
         """Raise an error when attempting to set a cached property."""
         raise AttributeError("Can't set attribute")
+
+
+class _HashIdWrapper:
+    """Hash and compare a wrapped object by identity instead of value"""
+
+    def __init__(self, wrapped):
+        self.wrapped = wrapped
+
+    def __eq__(self, other):
+        if not isinstance(other, _HashIdWrapper):
+            return NotImplemented
+        return self.wrapped is other.wrapped
+
+    def __ne__(self, other):
+        if not isinstance(other, _HashIdWrapper):
+            return NotImplemented
+        return self.wrapped is not other.wrapped
+
+    def __hash__(self):
+        return id(self.wrapped)
+
+
+@functools.lru_cache()
+def _cumsum(seq, initial_zero):
+    if isinstance(seq, _HashIdWrapper):
+        seq = seq.wrapped
+    if initial_zero:
+        return tuple(toolz.accumulate(add, seq, 0))
+    else:
+        return tuple(toolz.accumulate(add, seq))
+
+
+def cached_cumsum(seq, initial_zero=False):
+    """Compute :meth:`toolz.accumulate` with caching.
+
+    Caching is by the identify of `seq` rather than the value. It is thus
+    important that `seq` is a tuple of immutable objects, and this function
+    is intended for use where `seq` is a value that will persist (generally
+    block sizes).
+
+    Parameters
+    ----------
+    seq : tuple
+        Values to cumulatively sum.
+    initial_zero : bool, optional
+        If true, the return value is prefixed with a zero.
+
+    Returns
+    -------
+    tuple
+    """
+    if isinstance(seq, tuple):
+        # Look up by identity first, to avoid a linear-time __hash__
+        # if we've seen this tuple object before.
+        result = _cumsum(_HashIdWrapper(seq), initial_zero)
+    else:
+        # Construct a temporary tuple, and look up by value.
+        result = _cumsum(tuple(seq), initial_zero)
+    return result
diff -pruN 2022.01.0+dfsg-1/dask/utils_test.py 2022.02.0+dfsg-1/dask/utils_test.py
--- 2022.01.0+dfsg-1/dask/utils_test.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/utils_test.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,6 +1,7 @@
 from __future__ import annotations
 
 import importlib
+import time
 from typing import TYPE_CHECKING
 
 if TYPE_CHECKING:
@@ -19,6 +20,11 @@ def add(x, y):
     return x + y
 
 
+def slowadd(a, b, delay=0.1):
+    time.sleep(delay)
+    return a + b
+
+
 class GetFunctionTestMixin:
     """
     The GetFunctionTestCase class can be imported and used to test foreign
diff -pruN 2022.01.0+dfsg-1/dask/_version.py 2022.02.0+dfsg-1/dask/_version.py
--- 2022.01.0+dfsg-1/dask/_version.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/_version.py	2022-02-11 16:21:12.000000000 +0000
@@ -49,7 +49,7 @@ class NotThisMethod(Exception):
     """Exception raised if a method is not valid for the current scenario."""
 
 
-LONG_VERSION_PY = {}
+LONG_VERSION_PY: dict = {}
 HANDLERS = {}
 
 
diff -pruN 2022.01.0+dfsg-1/dask/widgets/__init__.py 2022.02.0+dfsg-1/dask/widgets/__init__.py
--- 2022.01.0+dfsg-1/dask/widgets/__init__.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/dask/widgets/__init__.py	2022-02-11 16:21:12.000000000 +0000
@@ -9,11 +9,11 @@ except ImportError as e:
         '  python -m pip install "dask[diagnostics]" --upgrade  # or python -m pip install'
     )
     exception = e  # Explicit reference for e as it will be lost outside the try block
-    FILTERS = dict()
+    FILTERS = {}
     TEMPLATE_PATHS = []
 
-    def get_environment(*args, **kwargs):
+    def get_environment():
         raise ImportError(msg) from exception
 
-    def get_template(*args, **kwargs):
+    def get_template(name: str):
         raise ImportError(msg) from exception
diff -pruN 2022.01.0+dfsg-1/debian/changelog 2022.02.0+dfsg-1/debian/changelog
--- 2022.01.0+dfsg-1/debian/changelog	2022-02-21 01:11:53.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/changelog	2022-08-04 18:15:42.000000000 +0000
@@ -1,4 +1,4 @@
-dask (2022.01.0+dfsg-1) unstable; urgency=medium
+dask (2022.02.0+dfsg-1) unstable; urgency=medium
 
   * New upstream release
   * Refresh patches
@@ -10,9 +10,12 @@ dask (2022.01.0+dfsg-1) unstable; urgenc
   * Add python3-ipython as a build-dependency.
     - it provides a sphinx formatting extension.
   * Add patch pytest-futurewarning.patch for compatibility with our pytest
-  * Disable python3-sparse for tests since numba is broken (Closes: #1005962)
+  * Add scipy-1.8.patch (Closes: #1013080) for Scipy 1.8 compatibility
+    Thanks: Graham Inggs
+  * run autopkgtest on all supported versions
+  * Remove built-using on -doc package
 
- -- Diane Trout <diane@ghic.org>  Sun, 20 Feb 2022 17:11:53 -0800
+ -- Diane Trout <diane@ghic.org>  Thu, 04 Aug 2022 11:15:42 -0700
 
 dask (2021.09.1+dfsg-2) unstable; urgency=medium
 
@@ -20,7 +23,6 @@ dask (2021.09.1+dfsg-2) unstable; urgenc
     See https://github.com/dask/dask/issues/8169
   * Upstream was concerned the 32-bit patch might not work with older
     versions of pandas, so require at least pandas version 1.3
-  * Remove executable bit from dask/dataframe/io/orc/utils.py
 
  -- Diane Trout <diane@ghic.org>  Mon, 20 Dec 2021 21:39:22 -0800
 
diff -pruN 2022.01.0+dfsg-1/debian/control 2022.02.0+dfsg-1/debian/control
--- 2022.01.0+dfsg-1/debian/control	2022-02-21 01:11:53.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/control	2022-08-04 18:12:08.000000000 +0000
@@ -21,7 +21,7 @@ Build-Depends: debhelper-compat (= 13),
                python3-partd <!nodoc>,
                python3-scipy <!nodoc>,
                python3-setuptools,
-#               python3-sparse (>= 0.11) <!nocheck>,
+               python3-sparse (>= 0.11) <!nocheck>,
                python3-sphinx <!nodoc>,
                python3-sphinx-click <!nodoc>,
                python3-sphinx-remove-toctrees <!nodoc>,
@@ -81,7 +81,6 @@ Depends: libjs-mathjax,
          node-js-yaml,
          ${misc:Depends},
          ${sphinxdoc:Depends}
-Built-Using: ${sphinxdoc:Built-Using}
 Description: Minimal task scheduling abstraction documentation
  Dask is a flexible parallel computing library for analytics,
  containing two components.
diff -pruN 2022.01.0+dfsg-1/debian/patches/32bit-comatibility.patch 2022.02.0+dfsg-1/debian/patches/32bit-comatibility.patch
--- 2022.01.0+dfsg-1/debian/patches/32bit-comatibility.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/32bit-comatibility.patch	2022-07-14 04:24:15.000000000 +0000
@@ -5,7 +5,7 @@ Bug: https://github.com/dask/dask/issues
 
 --- a/dask/dataframe/backends.py
 +++ b/dask/dataframe/backends.py
-@@ -352,7 +352,7 @@
+@@ -354,7 +354,7 @@
  @group_split_dispatch.register((pd.DataFrame, pd.Series, pd.Index))
  def group_split_pandas(df, c, k, ignore_index=False):
      indexer, locations = pd._libs.algos.groupsort_indexer(
diff -pruN 2022.01.0+dfsg-1/debian/patches/disable-embedded-google-calendar.patch 2022.02.0+dfsg-1/debian/patches/disable-embedded-google-calendar.patch
--- 2022.01.0+dfsg-1/debian/patches/disable-embedded-google-calendar.patch	2022-02-20 19:50:34.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/disable-embedded-google-calendar.patch	1970-01-01 00:00:00.000000000 +0000
@@ -1,20 +0,0 @@
-Author: Diane Trout <diane@ghic.org>
-Subject: Remove embedded iframe google calendar
-Forwarded: no
-
---- a/docs/source/support.rst
-+++ b/docs/source/support.rst
-@@ -35,12 +35,7 @@
-     Meeting notes are available in
-     `this Google doc <https://docs.google.com/document/d/1UqNAP87a56ERH_xkQsS5Q_0PKYybd5Lj2WANy_hRzI0/edit>`_.
- 
--    .. raw:: html
--
--       <iframe id="calendariframe" src="https://calendar.google.com/calendar/embed?ctz=local&amp;src=4l0vts0c1cgdbq5jhcogj55sfs%40group.calendar.google.com" style="border: 0" width="800" height="600" frameborder="0" scrolling="no"></iframe>
--       <script>document.getElementById("calendariframe").src = document.getElementById("calendariframe").src.replace("ctz=local", "ctz=" + Intl.DateTimeFormat().resolvedOptions().timeZone)</script>
--
--    You can subscribe to this calendar to be notified of changes:
-+    You can subscribe to the Dask project calendar to be notified of changes:
- 
-     * `Google Calendar <https://calendar.google.com/calendar/u/0?cid=NGwwdnRzMGMxY2dkYnE1amhjb2dqNTVzZnNAZ3JvdXAuY2FsZW5kYXIuZ29vZ2xlLmNvbQ>`__
-     * `iCal <https://calendar.google.com/calendar/ical/4l0vts0c1cgdbq5jhcogj55sfs%40group.calendar.google.com/public/basic.ics>`__
diff -pruN 2022.01.0+dfsg-1/debian/patches/reproducible-config-autofunction.patch 2022.02.0+dfsg-1/debian/patches/reproducible-config-autofunction.patch
--- 2022.01.0+dfsg-1/debian/patches/reproducible-config-autofunction.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/reproducible-config-autofunction.patch	2022-07-14 04:24:15.000000000 +0000
@@ -7,26 +7,25 @@ Description: Call os.path.expanduser in
  Also I think ~ is a bit clearer to the user, at least Unix-like OS users.
 --- a/dask/config.py
 +++ b/dask/config.py
-@@ -16,15 +16,15 @@
- paths = [
-     os.getenv("DASK_ROOT_CONFIG", "/etc/dask"),
-     os.path.join(sys.prefix, "etc", "dask"),
--    os.path.join(os.path.expanduser("~"), ".config", "dask"),
--    os.path.join(os.path.expanduser("~"), ".dask"),
-+    os.path.join("~", ".config", "dask"),
-+    os.path.join("~", ".dask"),
- ]
- 
+@@ -30,7 +30,7 @@
+         os.getenv("DASK_ROOT_CONFIG", "/etc/dask"),
+         os.path.join(sys.prefix, "etc", "dask"),
+         *[os.path.join(prefix, "etc", "dask") for prefix in site.PREFIXES],
+-        os.path.join(os.path.expanduser("~"), ".config", "dask"),
++        os.path.join("~", ".config", "dask"),
+     ]
+     if "DASK_CONFIG" in os.environ:
+         paths.append(os.environ["DASK_CONFIG"])
+@@ -46,7 +46,7 @@
  if "DASK_CONFIG" in os.environ:
      PATH = os.environ["DASK_CONFIG"]
-     paths.append(PATH)
  else:
 -    PATH = os.path.join(os.path.expanduser("~"), ".config", "dask")
 +    PATH = os.path.join("~", ".config", "dask")
  
  
- global_config = config = {}
-@@ -133,6 +133,7 @@
+ config: dict = {}
+@@ -156,6 +156,7 @@
      # Find all paths
      file_paths = []
      for path in paths:
@@ -34,7 +33,7 @@ Description: Call os.path.expanduser in
          if os.path.exists(path):
              if os.path.isdir(path):
                  try:
-@@ -224,7 +225,7 @@
+@@ -246,7 +247,7 @@
          Whether or not to comment out the config file when copying.
      """
      if destination is None:
@@ -43,3 +42,14 @@ Description: Call os.path.expanduser in
  
      # destination is a file and already exists, never overwrite
      if os.path.isfile(destination):
+--- a/dask/tests/test_config.py
++++ b/dask/tests/test_config.py
+@@ -516,7 +516,7 @@
+     expected = [
+         "/etc/dask",
+         os.path.join(sys.prefix, "etc", "dask"),
+-        os.path.join(os.path.expanduser("~"), ".config", "dask"),
++        os.path.join("~", ".config", "dask"),
+     ]
+     paths = _get_paths()
+     assert paths == expected
diff -pruN 2022.01.0+dfsg-1/debian/patches/reproducible-version.patch 2022.02.0+dfsg-1/debian/patches/reproducible-version.patch
--- 2022.01.0+dfsg-1/debian/patches/reproducible-version.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/reproducible-version.patch	2022-07-14 04:24:15.000000000 +0000
@@ -11,7 +11,7 @@ I think this is a source of non-determin
 
 --- a/versioneer.py
 +++ b/versioneer.py
-@@ -1408,6 +1408,15 @@
+@@ -1404,6 +1404,15 @@
          # see the discussion in cmdclass.py:get_cmdclass()
          del sys.modules["versioneer"]
  
diff -pruN 2022.01.0+dfsg-1/debian/patches/scipy1.8.patch 2022.02.0+dfsg-1/debian/patches/scipy1.8.patch
--- 2022.01.0+dfsg-1/debian/patches/scipy1.8.patch	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/scipy1.8.patch	2022-07-14 04:24:15.000000000 +0000
@@ -0,0 +1,158 @@
+Description: Scipy 1.8.0 compat: copy private classes into
+ dask/array/stats.py (#8694)
+Origin: upstream, https://github.com/dask/dask/commit/a27b437da2594fd5c15f85736aa520feb726ddbd
+Author: Julia Signell <jsignell@gmail.com>
+Last-Update: 2022-03-03
+
+--- a/dask/array/stats.py
++++ b/dask/array/stats.py
+@@ -28,6 +28,7 @@
+ # out of the use of this software, even if advised of the possibility of
+ # such damage.
+ import math
++from collections import namedtuple
+ 
+ import numpy as np
+ 
+@@ -42,16 +43,27 @@
+     raise ImportError("`dask.array.stats` requires `scipy` to be installed.") from e
+ from scipy import special
+ from scipy.stats import distributions
+-from scipy.stats.stats import (
+-    F_onewayResult,
+-    KurtosistestResult,
+-    NormaltestResult,
+-    Power_divergenceResult,
+-    SkewtestResult,
+-    Ttest_1sampResult,
+-    Ttest_indResult,
+-    Ttest_relResult,
+-)
++
++# copied from https://github.com/scipy/scipy/blob/v1.8.0/scipy/stats/_stats_py.py since
++# these are all private after v1.8.0
++F_onewayResult = namedtuple("F_onewayResult", ("statistic", "pvalue"))
++KurtosistestResult = namedtuple("KurtosistestResult", ("statistic", "pvalue"))
++NormaltestResult = namedtuple("NormaltestResult", ("statistic", "pvalue"))
++Power_divergenceResult = namedtuple("Power_divergenceResult", ("statistic", "pvalue"))
++SkewtestResult = namedtuple("SkewtestResult", ("statistic", "pvalue"))
++Ttest_1sampResult = namedtuple("Ttest_1sampResult", ("statistic", "pvalue"))
++Ttest_indResult = namedtuple("Ttest_indResult", ("statistic", "pvalue"))
++Ttest_relResult = namedtuple("Ttest_relResult", ("statistic", "pvalue"))
++
++# Map from names to lambda_ values used in power_divergence().
++_power_div_lambda_names = {
++    "pearson": 1,
++    "log-likelihood": 0,
++    "freeman-tukey": -0.5,
++    "mod-log-likelihood": -1,
++    "neyman": -2,
++    "cressie-read": 2 / 3,
++}
+ 
+ __all__ = [
+     "ttest_ind",
+@@ -140,14 +152,13 @@
+ def power_divergence(f_obs, f_exp=None, ddof=0, axis=0, lambda_=None):
+ 
+     if isinstance(lambda_, str):
+-        # TODO: public api
+-        if lambda_ not in scipy.stats.stats._power_div_lambda_names:
+-            names = repr(list(scipy.stats.stats._power_div_lambda_names.keys()))[1:-1]
++        if lambda_ not in _power_div_lambda_names:
++            names = repr(list(_power_div_lambda_names.keys()))[1:-1]
+             raise ValueError(
+                 f"invalid string for lambda_: {lambda_!r}. "
+                 f"Valid strings are {names}"
+             )
+-        lambda_ = scipy.stats.stats._power_div_lambda_names[lambda_]
++        lambda_ = _power_div_lambda_names[lambda_]
+     elif lambda_ is None:
+         lambda_ = 1
+ 
+--- a/dask/array/tests/test_linearoperator.py
++++ /dev/null
+@@ -1,31 +0,0 @@
+-import pytest
+-
+-pytest.importorskip("scipy")
+-
+-import numpy as np
+-import scipy.sparse.linalg
+-
+-import dask.array as da
+-
+-
+-def test_LinearOperator():
+-    X = np.random.random(size=(3, 2))
+-    y = np.random.random(size=(2, 1))
+-    w = np.random.random(size=(3, 1))
+-    square = np.random.random(size=(2, 2))
+-
+-    dX = da.from_array(X, chunks=(2, 1))
+-
+-    npLO = scipy.sparse.linalg.aslinearoperator(X)
+-    daLO = scipy.sparse.linalg.interface.MatrixLinearOperator(dX)
+-
+-    functions = [lambda x, y: x.matvec(y), lambda x, y: x * y, lambda x, y: x.dot(y)]
+-    for func in functions:
+-        assert np.allclose(func(npLO, y), func(daLO, y))
+-
+-    assert np.allclose(npLO.matmat(square), daLO.matmat(square))
+-
+-    assert np.allclose(npLO.rmatvec(w), daLO.rmatvec(w))
+-
+-    assert npLO.dtype == daLO.dtype
+-    assert npLO.shape == daLO.shape
+--- a/docs/source/array-linear-operator.rst
++++ /dev/null
+@@ -1,28 +0,0 @@
+-LinearOperator
+-==============
+-
+-Dask Array implements the SciPy LinearOperator_ interface and it can be used
+-with any SciPy algorithm depending on that interface.
+-
+-Example
+--------
+-
+-.. code-block:: python
+-
+-   import dask.array as da
+-   x = da.random.random(size=(10000, 10000), chunks=(1000, 1000))
+-
+-   from scipy.sparse.linalg.interface import MatrixLinearOperator
+-   A = MatrixLinearOperator(x)
+-
+-   import numpy as np
+-   b = np.random.random(10000)
+-
+-   from scipy.sparse.linalg import gmres
+-   x = gmres(A, b)
+-
+-*Disclaimer: This is just a toy example and not necessarily the best way to
+-solve this problem for this data.*
+-
+-
+-.. _LinearOperator: https://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.linalg.LinearOperator.html
+--- a/docs/source/array-overlap.rst
++++ b/docs/source/array-overlap.rst
+@@ -114,7 +114,7 @@
+ 
+ .. code-block:: python
+ 
+-   >>> from scipy.ndimage.filters import gaussian_filter
++   >>> from scipy.ndimage import gaussian_filter
+    >>> def func(block):
+    ...    return gaussian_filter(block, sigma=1)
+ 
+--- a/docs/source/array.rst
++++ b/docs/source/array.rst
+@@ -12,7 +12,6 @@
+    array-design.rst
+    array-sparse.rst
+    array-stats.rst
+-   array-linear-operator.rst
+    array-slicing.rst
+    array-assignment.rst
+    array-stack.rst
diff -pruN 2022.01.0+dfsg-1/debian/patches/series 2022.02.0+dfsg-1/debian/patches/series
--- 2022.01.0+dfsg-1/debian/patches/series	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/series	2022-07-14 04:24:15.000000000 +0000
@@ -9,3 +9,4 @@ js-yaml.patch
 use-youtube-nocookie.patch
 reproducible-config-autofunction.patch
 32bit-comatibility.patch
+scipy1.8.patch
diff -pruN 2022.01.0+dfsg-1/debian/patches/use-debian-version.patch 2022.02.0+dfsg-1/debian/patches/use-debian-version.patch
--- 2022.01.0+dfsg-1/debian/patches/use-debian-version.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/use-debian-version.patch	2022-07-14 04:24:15.000000000 +0000
@@ -11,7 +11,7 @@ Forwarded: No
 
 --- a/versioneer.py
 +++ b/versioneer.py
-@@ -1398,6 +1398,12 @@
+@@ -1394,6 +1394,12 @@
  
      Returns dict with two keys: 'version' and 'full'.
      """
diff -pruN 2022.01.0+dfsg-1/debian/patches/use-local-intersphinx.patch 2022.02.0+dfsg-1/debian/patches/use-local-intersphinx.patch
--- 2022.01.0+dfsg-1/debian/patches/use-local-intersphinx.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/use-local-intersphinx.patch	2022-07-14 04:24:15.000000000 +0000
@@ -10,7 +10,7 @@ Forwarded: No
 
 --- a/docs/source/conf.py
 +++ b/docs/source/conf.py
-@@ -315,15 +315,15 @@
+@@ -317,15 +317,15 @@
      "python": ("https://docs.python.org/3/", None),
      "pandas": (
          "https://pandas.pydata.org/pandas-docs/stable/",
diff -pruN 2022.01.0+dfsg-1/debian/patches/use-local-reference-yaml.patch 2022.02.0+dfsg-1/debian/patches/use-local-reference-yaml.patch
--- 2022.01.0+dfsg-1/debian/patches/use-local-reference-yaml.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/use-local-reference-yaml.patch	2022-07-14 04:24:15.000000000 +0000
@@ -5,7 +5,7 @@ Forwarded: no
 
 --- a/docs/source/configuration.rst
 +++ b/docs/source/configuration.rst
-@@ -462,8 +462,8 @@
+@@ -467,8 +467,8 @@
  
  .. dask-config-block::
      :location: dask
@@ -16,7 +16,7 @@ Forwarded: no
  
  
  Distributed Client
-@@ -471,16 +471,16 @@
+@@ -476,16 +476,16 @@
  
  .. dask-config-block::
      :location: distributed.client
@@ -37,7 +37,7 @@ Forwarded: no
  
  
  Distributed Dashboard
-@@ -488,8 +488,8 @@
+@@ -493,8 +493,8 @@
  
  .. dask-config-block::
      :location: distributed.dashboard
@@ -48,7 +48,7 @@ Forwarded: no
  
  
  Distributed Deploy
-@@ -497,8 +497,8 @@
+@@ -502,8 +502,8 @@
  
  .. dask-config-block::
      :location: distributed.deploy
@@ -59,7 +59,7 @@ Forwarded: no
  
  
  Distributed Scheduler
-@@ -506,8 +506,8 @@
+@@ -511,8 +511,8 @@
  
  .. dask-config-block::
      :location: distributed.scheduler
@@ -70,7 +70,7 @@ Forwarded: no
  
  
  Distributed Worker
-@@ -515,8 +515,8 @@
+@@ -520,8 +520,8 @@
  
  .. dask-config-block::
      :location: distributed.worker
@@ -81,7 +81,7 @@ Forwarded: no
  
  
  Distributed Nanny
-@@ -524,8 +524,8 @@
+@@ -529,8 +529,8 @@
  
  .. dask-config-block::
      :location: distributed.nanny
@@ -92,7 +92,7 @@ Forwarded: no
  
  
  Distributed Admin
-@@ -533,8 +533,8 @@
+@@ -538,8 +538,8 @@
  
  .. dask-config-block::
      :location: distributed.admin
@@ -103,7 +103,7 @@ Forwarded: no
  
  
  Distributed RMM
-@@ -542,5 +542,5 @@
+@@ -547,5 +547,5 @@
  
  .. dask-config-block::
      :location: distributed.rmm
@@ -113,7 +113,7 @@ Forwarded: no
 +    :schema: file:///../debian/dask-config-blocks/distributed-schema.yaml
 --- a/docs/source/ext/dask_config_sphinx_ext.py
 +++ b/docs/source/ext/dask_config_sphinx_ext.py
-@@ -5,8 +5,13 @@
+@@ -6,8 +6,13 @@
  
  
  def get_remote_yaml(url):
diff -pruN 2022.01.0+dfsg-1/debian/patches/Use-the-packaged-MathJax.patch 2022.02.0+dfsg-1/debian/patches/Use-the-packaged-MathJax.patch
--- 2022.01.0+dfsg-1/debian/patches/Use-the-packaged-MathJax.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/Use-the-packaged-MathJax.patch	2022-07-14 04:24:15.000000000 +0000
@@ -9,7 +9,7 @@ Forwarded: not-needed
 
 --- a/docs/source/conf.py
 +++ b/docs/source/conf.py
-@@ -55,6 +55,8 @@
+@@ -57,6 +57,8 @@
  # See https://github.com/dask/dask/issues/8227.
  remove_from_toctrees = ["generated/*"]
  
diff -pruN 2022.01.0+dfsg-1/debian/patches/use-youtube-nocookie.patch 2022.02.0+dfsg-1/debian/patches/use-youtube-nocookie.patch
--- 2022.01.0+dfsg-1/debian/patches/use-youtube-nocookie.patch	2022-02-20 22:18:36.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/patches/use-youtube-nocookie.patch	2022-07-14 04:24:15.000000000 +0000
@@ -111,6 +111,15 @@ youtube and is a minor change.
             style="margin: 0 auto 20px auto; display: block;"
             frameborder="0"
             allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture"
+@@ -14,7 +14,7 @@
+ Dask Tutorial
+ -------------
+ `Dask Tutorial <https://tutorial.dask.org>`__ provides an overview of Dask and is typically delivered in 3 hours.
+-See `Parallel and Distributed Computing in Python with Dask <https://www.youtube.com/watch?v=EybGGLbLipI>`__ for the
++See `Parallel and Distributed Computing in Python with Dask <https://www.youtube-nocookie.com/watch?v=EybGGLbLipI>`__ for the
+ latest Dask Tutorial recording from SciPy 2020.
+ 
+ Dask Slides
 @@ -23,7 +23,7 @@
  
  Dask YouTube channel
@@ -120,7 +129,45 @@ youtube and is a minor change.
  
  .. contents:: :local:
  
-@@ -61,79 +61,79 @@
+@@ -32,108 +32,108 @@
+ 
+ * Dask Summit 2021
+ 
+-  * `Keynotes <https://www.youtube.com/playlist?list=PLJ0vO2F_f6OBymP5LtgOC6W4pxd9Mw3cE>`__ 
+-  * `Workshops and Tutorials <https://www.youtube.com/playlist?list=PLJ0vO2F_f6OBD1_iNeT1f7cpRoYwAuMPy>`__
+-  * `Talks <https://www.youtube.com/playlist?list=PLJ0vO2F_f6OBcisTDubrdEsQAhigkayjE>`__
++  * `Keynotes <https://www.youtube-nocookie.com/playlist?list=PLJ0vO2F_f6OBymP5LtgOC6W4pxd9Mw3cE>`__ 
++  * `Workshops and Tutorials <https://www.youtube-nocookie.com/playlist?list=PLJ0vO2F_f6OBD1_iNeT1f7cpRoYwAuMPy>`__
++  * `Talks <https://www.youtube-nocookie.com/playlist?list=PLJ0vO2F_f6OBcisTDubrdEsQAhigkayjE>`__
+ 
+ * PyCon US 2021
+ 
+-  * `Tutorial: Hacking Dask: Diving into Dask's Internals <https://www.youtube.com/watch?v=LQrgDhN-XOo>`__  (`materials <https://github.com/jrbourbeau/hacking-dask>`__)
+-  * `Dask-SQL: Empowering Pythonistas for Scalable End-to-End Data Engineering <https://www.youtube.com/watch?v=z7xKikaScxg>`__
++  * `Tutorial: Hacking Dask: Diving into Dask's Internals <https://www.youtube-nocookie.com/watch?v=LQrgDhN-XOo>`__  (`materials <https://github.com/jrbourbeau/hacking-dask>`__)
++  * `Dask-SQL: Empowering Pythonistas for Scalable End-to-End Data Engineering <https://www.youtube-nocookie.com/watch?v=z7xKikaScxg>`__
+ 
+ 
+ * BlazingSQL Webinars, May 2021
+ 
+-  * `Intro to distributed computing on GPUs with Dask in Python <https://www.youtube.com/watch?v=py1YPs6s6so>`__ (`materials <https://gist.github.com/jacobtomlinson/6f16abb716f50f81a6687bd67efd2f61>`__)
++  * `Intro to distributed computing on GPUs with Dask in Python <https://www.youtube-nocookie.com/watch?v=py1YPs6s6so>`__ (`materials <https://gist.github.com/jacobtomlinson/6f16abb716f50f81a6687bd67efd2f61>`__)
+ 
+ * PyData DC, August 2021
+ 
+-  * `Inside Dask <https://www.youtube.com/watch?v=X95WO41abXo>`__ (`materials <https://github.com/jsignell/inside-dask>`__)
++  * `Inside Dask <https://www.youtube-nocookie.com/watch?v=X95WO41abXo>`__ (`materials <https://github.com/jsignell/inside-dask>`__)
+ 
+ * PyCon US 2020
+ 
+-  * `Deploying Python at Scale with Dask <https://www.youtube.com/watch?v=deX0GlW4uew>`__
++  * `Deploying Python at Scale with Dask <https://www.youtube-nocookie.com/watch?v=deX0GlW4uew>`__
+ 
+ * PyCon Australia 2020
+ 
+-  * `dask-image: distributed image processing for large data <https://www.youtube.com/watch?v=MpjgzNeISeI>`__
++  * `dask-image: distributed image processing for large data <https://www.youtube-nocookie.com/watch?v=MpjgzNeISeI>`__
+ 
  * PyCon Korea 2019, August 2019
  
    * `Adapting from Spark to Dask: what to expect (18 minutes)
@@ -219,25 +266,25 @@ youtube and is a minor change.
    * `Dask Out of core NumPy:Pandas through Task Scheduling (16 minutes)
 -    <https://www.youtube.com/watch?v=1kkFZ4P-XHg>`__
 +    <https://www.youtube-nocookie.com/watch?v=1kkFZ4P-XHg>`__
---- a/docs/source/how-to/deploy-dask-clusters.rst
-+++ b/docs/source/how-to/deploy-dask-clusters.rst
-@@ -12,7 +12,7 @@
+--- a/docs/source/how-to/adaptive.rst
++++ b/docs/source/how-to/adaptive.rst
+@@ -36,7 +36,7 @@
  
     <iframe width="560"
             height="315"
--           src="https://www.youtube.com/embed/TQM9zIBzNBo"
-+           src="https://www.youtube-nocookie.com/embed/TQM9zIBzNBo"
+-           src="https://www.youtube.com/embed/dViyEqOMA8U"
++           src="https://www.youtube-nocookie.com/embed/dViyEqOMA8U"
             style="margin: 0 auto 20px auto; display: block;"
             frameborder="0"
             allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture"
---- a/docs/source/how-to/adaptive.rst
-+++ b/docs/source/how-to/adaptive.rst
-@@ -36,7 +36,7 @@
+--- a/docs/source/deploying.rst
++++ b/docs/source/deploying.rst
+@@ -37,7 +37,7 @@
  
     <iframe width="560"
             height="315"
--           src="https://www.youtube.com/embed/dViyEqOMA8U"
-+           src="https://www.youtube-nocookie.com/embed/dViyEqOMA8U"
+-           src="https://www.youtube.com/embed/TQM9zIBzNBo"
++           src="https://www.youtube-nocookie.com/embed/TQM9zIBzNBo"
             style="margin: 0 auto 20px auto; display: block;"
             frameborder="0"
             allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture"
diff -pruN 2022.01.0+dfsg-1/debian/rules 2022.02.0+dfsg-1/debian/rules
--- 2022.01.0+dfsg-1/debian/rules	2022-02-21 01:11:53.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/rules	2022-07-14 04:24:15.000000000 +0000
@@ -25,7 +25,6 @@ endif
 override_dh_auto_install:
 	dh_auto_install
 	dh_link
-	chmod a-x debian/python3-dask/usr/lib/python3*/dist-packages/dask/dataframe/io/orc/utils.py
 
 override_dh_installchangelogs:
 	dh_installchangelogs docs/source/changelog.rst
diff -pruN 2022.01.0+dfsg-1/debian/tests/run-test 2022.02.0+dfsg-1/debian/tests/run-test
--- 2022.01.0+dfsg-1/debian/tests/run-test	2022-02-19 19:52:55.000000000 +0000
+++ 2022.02.0+dfsg-1/debian/tests/run-test	2022-08-04 18:11:07.000000000 +0000
@@ -10,7 +10,7 @@ SKIP_TESTS=test_development_guidelines_m
 SKIP_TESTS="$SKIP_TESTS or test_describe_empty"
 
 cp conftest.py "$AUTOPKGTEST_TMP"
-for py in $(py3versions -r 2>/dev/null)
+for py in $(py3versions --supported 2>/dev/null)
 do cd "$AUTOPKGTEST_TMP"
   echo "Testing with $py:"
   http_proxy= $py -m pytest -k "not ( ${SKIP_TESTS} )" -m "not network" -v --pyargs dask
diff -pruN 2022.01.0+dfsg-1/docs/release-procedure.md 2022.02.0+dfsg-1/docs/release-procedure.md
--- 2022.01.0+dfsg-1/docs/release-procedure.md	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/release-procedure.md	2022-02-11 16:21:12.000000000 +0000
@@ -11,6 +11,9 @@ Release per project:
 
 *   Update release notes in docs/source/changelog.rst
 
+*   Update the distributed version in dask/setup.py and the dask version
+    in distributed/requirements.txt
+
 *   Commit
 
         git commit -a -m "bump version to x.x.x"
diff -pruN 2022.01.0+dfsg-1/docs/source/10-minutes-to-dask.rst 2022.02.0+dfsg-1/docs/source/10-minutes-to-dask.rst
--- 2022.01.0+dfsg-1/docs/source/10-minutes-to-dask.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/10-minutes-to-dask.rst	2022-02-11 16:21:12.000000000 +0000
@@ -507,7 +507,7 @@ on both single and multiple machines. Th
          <Client: 'tcp://127.0.0.1:41703' processes=4 threads=12, memory=31.08 GiB>
 
       There are a variety of ways to set up a remote cluster. Refer to
-      :doc:`how to deploy dask clusters <how-to/deploy-dask-clusters>` for more
+      :doc:`how to deploy dask clusters <deploying>` for more
       information.
 
 Once you create a client, any computation will run on the cluster that it points to.
diff -pruN 2022.01.0+dfsg-1/docs/source/changelog.rst 2022.02.0+dfsg-1/docs/source/changelog.rst
--- 2022.01.0+dfsg-1/docs/source/changelog.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/changelog.rst	2022-02-11 16:21:12.000000000 +0000
@@ -1,6 +1,122 @@
 Changelog
 =========
 
+.. _v2022.02.0:
+
+2022.02.0
+---------
+
+Released on February 11, 2022
+
+New Features
+^^^^^^^^^^^^
+- Add ``region`` to ``to_zarr`` when using existing array (:pr:`8590`) `Chris Roat`_
+- Add ``engine_kwargs`` support to ``dask.dataframe.to_sql`` (:pr:`8609`) `Amir Kadivar`_
+- Add ``include_path_column`` arg to ``read_json`` (:pr:`8603`) `Bryan W. Weber`_
+- Add ``expand_dims`` to Dask array (:pr:`8667) (:pr:`8687`) `Tom White`_
+
+Enhancements
+^^^^^^^^^^^^
+- Add scheduler option to ``assert_eq`` utilities (:pr:`8610`) `Xinrong Meng`_
+- Fix eye inconsistency with NumPy for ``dtype=None`` (:pr:`8669) (:pr:`8685`) `Tom White`_
+- Fix concatenate inconsistency with NumPy for ``axis=None`` (:pr:`8668) (:pr:`8686`) `Tom White`_
+- Type annotations, part 1 (:pr:`8295`) `Guido Imperiale`_
+- Really allow any iterable to be passed as a ``meta`` (:pr:`8629`) `Julia Signell`_
+- Use ``map_partitions`` (Blockwise) in ``to_parquet`` (:pr:`8487`) `Richard (Rick) Zamora`_
+
+Bug Fixes
+^^^^^^^^^
+- Result of reducing an array should not depend on its chunk-structure (:pr:`8637`) `ParticularMiner`_
+- Pass place-holder metadata to ``map_partitions`` in ACA code path (:pr:`8643`) `Richard (Rick) Zamora`_
+
+Deprecations
+^^^^^^^^^^^^
+- Deprecate ``is_monotonic`` (:pr:`8653`) `James Bourbeau`_
+- Remove some deprecations (:pr:`8605`) `James Bourbeau`_
+
+Documentation
+^^^^^^^^^^^^^
+- Add Domino Data Lab to Hosted / managed Dask clusters (:pr:`8675`) `Ray Bell`_
+- Fix inter-linking and remove deprecated function (:pr:`8715`) `Julia Signell`_
+- Fix imbalanced backticks. (:pr:`8693`) `Matthias Bussonnier`_
+- Add documentation for high level graph visualization (:pr:`8483`) `Genevieve Buckley`_
+- Update documentation of ``ProgressBar`` ``out`` parameter (:pr:`8604`) `Pedro Silva`_
+- Improve documentation of ``dask.config.set`` (:pr:`8705`) `Guido Imperiale`_
+- Revert mention to ``mypy`` among type checkers (:pr:`8699`) `Guido Imperiale`_
+
+Maintenance
+^^^^^^^^^^^
+- Update warning handling in ``get_dummies`` tests (:pr:`8651`) `James Bourbeau`_
+- Add a github changelog template (:pr:`8714`) `Julia Signell`_
+- Update year in LICENSE.txt (:pr:`8665`) `David Hoese`_
+- Update ``pre-commit`` version  (:pr:`8691`) `James Bourbeau`_
+- Include ``scipy`` in upstream CI build (:pr:`8681`) `James Bourbeau`_
+- Temporarily pin ``scipy < 1.8.0`` in CI (:pr:`8683`) `James Bourbeau`_
+- Pin ``scipy`` to less than 1.8.0 in GPU CI (:pr:`8698`) `Julia Signell`_
+- Avoid ``pytest.warns(None)`` in ``test_multi.py`` (:pr:`8678`) `James Bourbeau`_
+- Update GHA concurrent job cancellation (:pr:`8652`) `James Bourbeau`_
+- Make ``test__get_paths`` robust to ``site.PREFIXES`` being set (:pr:`8644`) `James Bourbeau`_
+- Bump gpuCI PYTHON_VER to 3.9 (:pr:`8642`) `Charles Blackmon-Luca`_
+
+
+.. _v2022.01.1:
+
+2022.01.1
+---------
+
+Released on January 28, 2022
+
+New Features
+^^^^^^^^^^^^
+- Add ``dask.dataframe.series.view()`` (:pr:`8533`) `Pavithra Eswaramoorthy`_
+
+Enhancements
+^^^^^^^^^^^^
+- Update ``tz`` for ``fastparquet`` + ``pandas`` 1.4.0 (:pr:`8626`) `Martin Durant`_
+- Cleaning up misc tests for ``pandas`` compat (:pr:`8623`) `Julia Signell`_
+- Moving to ``SQLAlchemy >= 1.4`` (:pr:`8158`) `McToel`_
+- Pandas compat: Filter sparse warnings (:pr:`8621`) `Julia Signell`_
+- Fail if ``meta`` is not a ``pandas`` object (:pr:`8563`) `Julia Signell`_
+- Use ``fsspec.parquet`` module for better remote-storage ``read_parquet`` performance (:pr:`8339`) `Richard (Rick) Zamora`_
+- Move DataFrame ACA aggregations to HLG (:pr:`8468`) `Richard (Rick) Zamora`_
+- Add optional information about originating function call in ``DataFrameIOLayer`` (:pr:`8453`) `Richard (Rick) Zamora`_
+- Blockwise array creation redux (:pr:`7417`) `Ian Rose`_
+- Refactor config default search path retrieval (:pr:`8573`) `James Bourbeau`_
+- Add ``optimize_graph`` flag to ``Bag.to_dataframe`` function (:pr:`8486`) `Maxim Lippeveld`_
+- Make sure that delayed output operations still return lists of paths (:pr:`8498`) `Julia Signell`_
+- Pandas compat: Fix ``to_frame`` ``name`` to not pass ``None`` (:pr:`8554`) `Julia Signell`_
+- Pandas compat: Fix ``axis=None`` warning (:pr:`8555`) `Julia Signell`_
+- Expand Dask YAML config search directories (:pr:`8531`) `abergou`_
+
+Bug Fixes
+^^^^^^^^^
+- Fix ``groupby.cumsum`` with series grouped by index (:pr:`8588`) `Julia Signell`_
+- Fix ``derived_from`` for ``pandas`` methods (:pr:`8612`) `Thomas J. Fan`_
+- Enforce boolean ``ascending`` for ``sort_values`` (:pr:`8440`) `Charles Blackmon-Luca`_
+- Fix parsing of ``__setitem__`` indices (:pr:`8601`) `David Hassell`_
+- Avoid divide by zero in slicing  (:pr:`8597`) `Doug Davis`_
+
+Deprecations
+^^^^^^^^^^^^
+- Downgrade ``meta`` error in (:pr:`8563`) to warning (:pr:`8628`) `Julia Signell`_
+- Pandas compat: Deprecate ``append`` when ``pandas >= 1.4.0`` (:pr:`8617`) `Julia Signell`_
+
+Documentation
+^^^^^^^^^^^^^
+- Replace outdated ``columns`` argument with ``meta`` in DataFrame constructor (:pr:`8614`) `kori73`_
+- Refactor deploying docs (:pr:`8602`) `Jacob Tomlinson`_
+
+Maintenance
+^^^^^^^^^^^
+- Pin ``coverage`` in CI (:pr:`8631`) `James Bourbeau`_
+- Move ``cached_cumsum`` imports to be from ``dask.utils`` (:pr:`8606`) `James Bourbeau`_
+- Update gpuCI ``RAPIDS_VER`` to ``22.04`` (:pr:`8600`)
+- Update cocstring for ``from_delayed`` function  (:pr:`8576`) `Kirito1397`_
+- Handle ``plot_width`` / ``plot_height`` deprecations (:pr:`8544`) `Bryan Van de Ven`_
+- Remove unnecessary ``pyyaml`` ``importorskip`` (:pr:`8562`) `James Bourbeau`_
+- Specify scheduler in DataFrame ``assert_eq`` (:pr:`8559`) `Gabe Joseph`_
+
+
 .. _v2022.01.0:
 
 2022.01.0
@@ -5039,3 +5155,12 @@ Other
 .. _`Deepyaman Datta`: https://github.com/deepyaman
 .. _`Maren Westermann`: https://github.com/marenwestermann
 .. _`Michael Delgado`: https://github.com/delgadom
+.. _`abergou`: https://github.com/abergou
+.. _`Pavithra Eswaramoorthy`: https://github.com/pavithraes
+.. _`Maxim Lippeveld`: https://github.com/MaximLippeveld
+.. _`Kirito1397`: https://github.com/Kirito1397
+.. _`Xinrong Meng`: https://github.com/xinrong-databricks
+.. _`Bryan W. Weber`: https://github.com/bryanwweber
+.. _`Amir Kadivar`: https://github.com/amirkdv
+.. _`Pedro Silva`: https://github.com/ppsbs
+.. _`neel iyer`: https://github.com/spiyer99
\ No newline at end of file
diff -pruN 2022.01.0+dfsg-1/docs/source/configuration.rst 2022.02.0+dfsg-1/docs/source/configuration.rst
--- 2022.01.0+dfsg-1/docs/source/configuration.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/configuration.rst	2022-02-11 16:21:12.000000000 +0000
@@ -78,7 +78,9 @@ These files can live in any of the follo
 
 1.  The ``~/.config/dask`` directory in the user's home directory
 2.  The ``{sys.prefix}/etc/dask`` directory local to Python
-3.  The root directory (specified by the ``DASK_ROOT_CONFIG`` environment
+3.  The ``{prefix}/etc/dask`` directories with ``{prefix}`` in `site.PREFIXES
+    <https://docs.python.org/3/library/site.html#site.PREFIXES>`_
+4.  The root directory (specified by the ``DASK_ROOT_CONFIG`` environment
     variable or ``/etc/dask/`` by default)
 
 Dask searches for *all* YAML files within each of these directories and merges
@@ -91,9 +93,6 @@ The contents of these YAML files are mer
 Dask subprojects like ``dask-kubernetes`` or ``dask-ml`` to manage configuration
 files separately, but have them merge into the same global configuration.
 
-*Note: for historical reasons we also look in the ``~/.dask`` directory for
-config files.  This is deprecated and will soon be removed.*
-
 
 Environment Variables
 ~~~~~~~~~~~~~~~~~~~~~
@@ -169,18 +168,24 @@ and interprets ``"."`` as nested access:
 
 .. code-block:: python
 
-   >>> dask.config.set({'scheduler.work-stealing': True})
+   >>> dask.config.set({'optimization.fuse.ave-width': 4})
 
 This function can also be used as a context manager for consistent cleanup:
 
 .. code-block:: python
 
-   with dask.config.set({'scheduler.work-stealing': True}):
-       ...
+   >>> with dask.config.set({'optimization.fuse.ave-width': 4}):
+   ...     arr2, = dask.optimize(arr)
 
 Note that the ``set`` function treats underscores and hyphens identically.
-For example, ``dask.config.set({'scheduler.work-stealing': True})`` is
-equivalent to ``dask.config.set({'scheduler.work_stealing': True})``.
+For example, ``dask.config.set({'optimization.fuse.ave_width': 4})`` is
+equivalent to ``dask.config.set({'optimization.fuse.ave-width': 4})``.
+
+Finally, note that persistent objects may acquire configuration settings when
+they are initialized. These settings may also be cached for performance reasons.
+This is particularly true for ``dask.distributed`` objects such as Client, Scheduler,
+Worker, and Nanny.
+
 
 Distributing configuration
 ~~~~~~~~~~~~~~~~~~~~~~~~~~
diff -pruN 2022.01.0+dfsg-1/docs/source/conf.py 2022.02.0+dfsg-1/docs/source/conf.py
--- 2022.01.0+dfsg-1/docs/source/conf.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/conf.py	2022-02-11 16:21:12.000000000 +0000
@@ -10,6 +10,8 @@
 # All configuration values have a default; values that are commented out
 # serve to show the default.
 
+from __future__ import annotations
+
 import os
 
 # Add any Sphinx extension module names here, as strings. They can be extensions
@@ -83,7 +85,7 @@ copyright = "2014-2018, Anaconda, Inc. a
 
 # List of patterns, relative to source directory, that match files and
 # directories to ignore when looking for source files.
-exclude_patterns = []
+exclude_patterns: list[str] = []
 
 # The reST default role (used for this markup: `text`) to use for all documents.
 # default_role = None
@@ -187,7 +189,7 @@ htmlhelp_basename = "daskdoc"
 
 # -- Options for LaTeX output --------------------------------------------------
 
-latex_elements = {
+latex_elements: dict[str, str] = {
     # The paper size ('letterpaper' or 'a4paper').
     #'papersize': 'letterpaper',
     # The font size ('10pt', '11pt' or '12pt').
@@ -365,16 +367,26 @@ redirect_files = [
     ("cite.html", "faq.html#how-do-I-cite-dask"),
     ("remote-data-services.html", "how-to/connect-to-remote-data.html"),
     ("debugging.html", "how-to/debug.html"),
-    ("setup.html", "how-to/deploy-dask-clusters.html"),
-    ("setup/cli.html", "how-to/deploy-dask/cli.html"),
-    ("setup/cloud.html", "how-to/deploy-dask/cloud.html"),
-    ("setup/docker.html", "how-to/deploy-dask/docker.html"),
-    ("setup/hpc.html", "how-to/deploy-dask/hpc.html"),
-    ("setup/kubernetes.html", "how-to/deploy-dask/kubernetes.html"),
-    ("setup/python-advanced.html", "how-to/deploy-dask/python-advanced.html"),
-    ("setup/single-distributed.html", "how-to/deploy-dask/single-distributed.html"),
-    ("setup/single-machine.html", "how-to/deploy-dask/single-machine.html"),
-    ("setup/ssh.html", "how-to/deploy-dask/ssh.html"),
+    ("setup.html", "deploying.html"),
+    ("how-to/deploy-dask-clusters.html", "deploying.html"),
+    ("setup/cli.html", "deploying-cli.html"),
+    ("how-to/deploy-dask/cli.html", "deploying-cli.html"),
+    ("setup/cloud.html", "deploying-cloud.html"),
+    ("how-to/deploy-dask/cloud.html", "deploying-cloud.html"),
+    ("setup/docker.html", "hdeploying-docker.html"),
+    ("how-to/deploy-dask/docker.html", "deploying-docker.html"),
+    ("setup/hpc.html", "deploying-hpc.html"),
+    ("how-to/deploy-dask/hpc.html", "deploying-hpc.html"),
+    ("setup/kubernetes.html", "deploying-kubernetes.html"),
+    ("how-to/deploy-dask/kubernetes.html", "deploying-kubernetes.html"),
+    ("setup/python-advanced.html", "deploying-python-advanced.html"),
+    ("how-to/deploy-dask/python-advanced.html", "deploying-python-advanced.html"),
+    ("setup/single-distributed.html", "deploying-python.html"),
+    ("how-to/deploy-dask/single-distributed.html", "deploying-python.html"),
+    ("setup/single-machine.html", "scheduling.html"),
+    ("how-to/deploy-dask/single-machine.html", "scheduling.html"),
+    ("setup/ssh.html", "deploying-ssh.html"),
+    ("how-to/deploy-dask/ssh.html", "deploying-ssh.html"),
     ("setup/adaptive.html", "how-to/adaptive.html"),
     ("setup/custom-startup.html", "how-to/customize-initialization.html"),
     ("setup/environment.html", "how-to/manage-environments.html"),
diff -pruN 2022.01.0+dfsg-1/docs/source/dataframe-api.rst 2022.02.0+dfsg-1/docs/source/dataframe-api.rst
--- 2022.01.0+dfsg-1/docs/source/dataframe-api.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/dataframe-api.rst	2022-02-11 16:21:12.000000000 +0000
@@ -376,6 +376,8 @@ Create DataFrames
    read_json
    read_orc
    read_sql_table
+   read_sql_query
+   read_sql
    from_array
    from_bcolz
    from_dask_array
diff -pruN 2022.01.0+dfsg-1/docs/source/dataframe-create.rst 2022.02.0+dfsg-1/docs/source/dataframe-create.rst
--- 2022.01.0+dfsg-1/docs/source/dataframe-create.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/dataframe-create.rst	2022-02-11 16:21:12.000000000 +0000
@@ -27,6 +27,8 @@ File Formats:
     read_orc
     read_json
     read_sql_table
+    read_sql_query
+    read_sql
     read_table
     read_fwf
     from_bcolz
@@ -113,7 +115,9 @@ information:
     corresponding to ``(name, i)`` should produce ``pandas.DataFrame`` objects
     that correspond to the columns and divisions information discussed below
 2.  Name: the special name used above
-3.  Columns: a list of column names
+3.  Meta: an empty pandas DataFrame with names, dtypes and index matching
+    the expected output. Can also be a list of tuples where each tuple defines
+    a ``(name, dtype)`` pair referring to one column.
 4.  Divisions: a list of index values that separate the different partitions.
     Alternatively, if you don't know the divisions (this is common), you can
     provide a list of ``[None, None, None, ...]`` with as many partitions as
@@ -130,13 +134,13 @@ The ``dd.read_csv`` function does this f
           ('mydf', 1): (pd.read_csv, 'data/2000-01-02.csv'),
           ('mydf', 2): (pd.read_csv, 'data/2000-01-03.csv')}
    name = 'mydf'
-   columns = ['price', 'name', 'id']
+   meta = [('price', float), ('name', str), ('id', int)]
    divisions = [Timestamp('2000-01-01 00:00:00'),
                 Timestamp('2000-01-02 00:00:00'),
                 Timestamp('2000-01-03 00:00:00'),
                 Timestamp('2000-01-03 23:59:59')]
 
-   df = dd.DataFrame(dsk, name, columns, divisions)
+   df = dd.DataFrame(dsk, name, meta, divisions)
 
 Storing
 -------
diff -pruN 2022.01.0+dfsg-1/docs/source/dataframe.rst 2022.02.0+dfsg-1/docs/source/dataframe.rst
--- 2022.01.0+dfsg-1/docs/source/dataframe.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/dataframe.rst	2022-02-11 16:21:12.000000000 +0000
@@ -171,5 +171,5 @@ Dask DataFrame as they are for Dask Arra
 the Pandas development team is actively working on releasing the GIL.
 
 When dealing with text data, you may see speedups by switching to the
-:doc:`distributed scheduler <how-to/deploy-dask/single-distributed>` either on a cluster or
+:doc:`distributed scheduler <deploying>` either on a cluster or
 single machine.
diff -pruN 2022.01.0+dfsg-1/docs/source/dataframe-sql.rst 2022.02.0+dfsg-1/docs/source/dataframe-sql.rst
--- 2022.01.0+dfsg-1/docs/source/dataframe-sql.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/dataframe-sql.rst	2022-02-11 16:21:12.000000000 +0000
@@ -75,14 +75,14 @@ You may find the dask API easier to use
 are already used to Pandas), and the diagnostic feedback more useful.
 These points can debatably be in Dask's favour.
 
-Loading from SQL with read_sql_table
-------------------------------------
+Loading from SQL with read_sql_table or read_sql_query
+------------------------------------------------------
 
 Dask allows you to build dataframes from SQL tables and queries using the
-function :func:`dask.dataframe.read_sql_table`, based on the `Pandas version`_,
-sharing most arguments, and using SQLAlchemy for the actual handling of the
-queries. You may need to install additional driver packages for your chosen
-database server.
+function :func:`dask.dataframe.read_sql_table` and :func:`dask.dataframe.read_sql_query`,
+based on the `Pandas version`_, sharing most arguments, and using SQLAlchemy
+for the actual handling of the queries. You may need to install additional
+driver packages for your chosen database server.
 
 .. _Pandas version: https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.read_sql_table.html
 
@@ -92,7 +92,7 @@ on a cluster, the following are the main
 - Dask does not support arbitrary text queries, only whole tables and SQLAlchemy
   `sql expressions`_
 
-- the engine argument must be a `URI string`_, not an SQLAlchemy engine/connection
+- the con argument must be a `URI string`_, not an SQLAlchemy engine/connection
 
 - partitioning information is *required*, which can be as simple as providing
   an index column argument, or can be more explicit (see below)
@@ -187,13 +187,13 @@ the point of execution.
             number, name, sql.func.length(name).label("lenname")
         ]
         ).select_from(sql.table("test"))
-    data = read_sql_table(
-        "test", db, npartitions=2, index_col=number
+    data = read_sql_query(
+        s1, db, npartitions=2, index_col=number
     )
 
 Here we have also demonstrated the use of the function ``length`` to
 perform an operation server-side. Note that it is necessary to *label* such
-operations, but you can use them for the index column (by name or expression),
+operations, but you can use them for the index column,
 so long as it is also
 in the set of selected columns. If using for the index/partitioning, the
 column should still be indexed in the database, for performance.
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-cli.rst 2022.02.0+dfsg-1/docs/source/deploying-cli.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-cli.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-cli.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,101 @@
+Command Line
+============
+
+This is the most fundamental way to deploy Dask on multiple machines.  In
+production environments this process is often automated by some other resource
+manager. Hence, it is rare that people need to follow these instructions
+explicitly.  Instead, these instructions are useful to help understand what
+*cluster managers* and other automated tooling is doing under the hood and to
+help users deploy onto platforms that have no automated tools today.
+
+A ``dask.distributed`` network consists of one ``dask-scheduler`` process and
+several ``dask-worker`` processes that connect to that scheduler.  These are
+normal Python processes that can be executed from the command line.  We launch
+the ``dask-scheduler`` executable in one process and the ``dask-worker``
+executable in several processes, possibly on different machines.
+
+To accomplish this, launch ``dask-scheduler`` on one node::
+
+   $ dask-scheduler
+   Scheduler at:   tcp://192.0.0.100:8786
+
+Then, launch ``dask-worker`` on the rest of the nodes, providing the address to
+the node that hosts ``dask-scheduler``::
+
+   $ dask-worker tcp://192.0.0.100:8786
+   Start worker at:  tcp://192.0.0.1:12345
+   Registered to:    tcp://192.0.0.100:8786
+
+   $ dask-worker tcp://192.0.0.100:8786
+   Start worker at:  tcp://192.0.0.2:40483
+   Registered to:    tcp://192.0.0.100:8786
+
+   $ dask-worker tcp://192.0.0.100:8786
+   Start worker at:  tcp://192.0.0.3:27372
+   Registered to:    tcp://192.0.0.100:8786
+
+The workers connect to the scheduler, which then sets up a long-running network
+connection back to the worker.  The workers will learn the location of other
+workers from the scheduler.
+
+
+Handling Ports
+--------------
+
+The scheduler and workers both need to accept TCP connections on an open port.
+By default, the scheduler binds to port ``8786`` and the worker binds to a
+random open port.  If you are behind a firewall then you may have to open
+particular ports or tell Dask to listen on particular ports with the ``--port``
+and ``--worker-port`` keywords.::
+
+   dask-scheduler --port 8000
+   dask-worker --dashboard-address 8000 --nanny-port 8001
+
+
+Nanny Processes
+---------------
+
+Dask workers are run within a nanny process that monitors the worker process
+and restarts it if necessary.
+
+
+Diagnostic Web Servers
+----------------------
+
+Additionally, Dask schedulers and workers host interactive diagnostic web
+servers using `Bokeh <https://docs.bokeh.org>`_.  These are optional, but
+generally useful to users.  The diagnostic server on the scheduler is
+particularly valuable, and is served on port ``8787`` by default (configurable
+with the ``--dashboard-address`` keyword).
+
+For more information about relevant ports, please take a look at the available
+:ref:`command line options <worker-scheduler-cli-options>`.
+
+Automated Tools
+---------------
+
+There are various mechanisms to deploy these executables on a cluster, ranging
+from manually SSH-ing into all of the machines to more automated systems like
+SGE/SLURM/Torque or Yarn/Mesos.  Additionally, cluster SSH tools exist to send
+the same commands to many machines.  We recommend searching online for "cluster
+ssh" or "cssh".
+
+
+.. _worker-scheduler-cli-options:
+
+CLI Options
+-----------
+
+.. note::
+
+   The command line documentation here may differ depending on your installed
+   version. We recommend referring to the output of ``dask-scheduler --help``
+   and ``dask-worker --help``.
+
+.. click:: distributed.cli.dask_scheduler:main
+   :prog: dask-scheduler
+   :show-nested:
+
+.. click:: distributed.cli.dask_worker:main
+   :prog: dask-worker
+   :show-nested:
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-cloud.rst 2022.02.0+dfsg-1/docs/source/deploying-cloud.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-cloud.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-cloud.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,75 @@
+Cloud
+=====
+
+There are a variety of ways to deploy Dask on cloud providers.
+Cloud providers provide managed services,
+like VMs, Kubernetes, Yarn, or custom APIs with which Dask can connect easily.
+You may want to consider the following options:
+
+1.  A managed Kubernetes service and Dask's
+    :doc:`Kubernetes integration <deploying-kubernetes>`.
+2.  A managed Yarn service,
+    like `Amazon EMR <https://aws.amazon.com/emr/>`_
+    or `Google Cloud DataProc <https://cloud.google.com/dataproc/>`_
+    and `Dask-Yarn <https://yarn.dask.org>`_.
+
+    Specific documentation for the popular Amazon EMR service can be found
+    `here <https://yarn.dask.org/en/latest/aws-emr.html>`_
+3.  Directly launching cloud resources such as VMs or containers via a cluster manager with
+    `Dask Cloud Provider <https://cloudprovider.dask.org/en/latest/>`_
+
+Cloud Deployment Example
+------------------------
+
+Using `Dask Cloud Provider <https://cloudprovider.dask.org/en/latest/>`_ to launch a cluster of
+VMs on a platform like `DigitalOcean <https://www.digitalocean.com/>`_ can be as convenient as
+launching a local cluster.
+
+.. code-block:: python
+
+    >>> import dask.config
+
+    >>> dask.config.set({"cloudprovider.digitalocean.token": "yourAPItoken"})
+
+    >>> from dask_cloudprovider.digitalocean import DropletCluster
+
+    >>> cluster = DropletCluster(n_workers=1)
+    Creating scheduler instance
+    Created droplet dask-38b817c1-scheduler
+    Waiting for scheduler to run
+    Scheduler is running
+    Creating worker instance
+    Created droplet dask-38b817c1-worker-dc95260d
+
+Many of the cluster managers in Dask Cloud Provider work by launching VMs with a startup script
+that pulls down the :doc:`Dask Docker image <deploying-docker>` and runs Dask components within that container.
+As with all cluster managers the VM resources, Docker image, etc are all configurable.
+
+You can then connect a client and work with the cluster as if it were on your local machine.
+
+.. code-block:: python
+
+    >>> from dask.distributed import Client
+
+    >>> client = Client(cluster)
+
+Data Access
+-----------
+
+You may want to install additional libraries in your Jupyter and worker images
+to access the object stores of each cloud:
+
+-  `s3fs <https://s3fs.readthedocs.io/>`_ for Amazon's S3
+-  `gcsfs <https://gcsfs.readthedocs.io/>`_ for Google's GCS
+-  `adlfs <https://github.com/dask/adlfs/>`_ for Microsoft's ADL
+
+Historical Libraries
+--------------------
+
+Dask previously maintained libraries for deploying Dask on
+Amazon's EC2 and Google GKE.
+Due to sporadic interest,
+and churn both within the Dask library and EC2 itself,
+these were not well maintained.
+They have since been deprecated in favor of the
+:doc:`Kubernetes and Helm <deploying-kubernetes-helm>` solution.
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-docker.rst 2022.02.0+dfsg-1/docs/source/deploying-docker.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-docker.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-docker.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,69 @@
+Docker Images
+=============
+
+Example docker images are maintained at https://github.com/dask/dask-docker
+and https://hub.docker.com/r/daskdev/ .
+
+Each image installs the full Dask conda environment (including the distributed
+scheduler), Numpy, and Pandas on top of a Miniconda installation on top of
+a Debian image.
+
+These images are large, around 1GB.
+
+-   ``daskdev/dask``: This a normal debian + miniconda image with the full Dask
+    conda package (including the distributed scheduler), Numpy, and Pandas.
+    This image is about 1GB in size.
+
+-   ``daskdev/dask-notebook``: This is based on the
+    `Jupyter base-notebook image <https://hub.docker.com/r/jupyter/base-notebook/>`_
+    and so it is suitable for use both normally as a Jupyter server, and also as
+    part of a JupyterHub deployment.  It also includes a matching Dask software
+    environment described above.  This image is about 2GB in size.
+
+Example
+-------
+
+Here is a simple example on the local host network
+
+.. code-block:: bash
+
+   docker run -it --network host daskdev/dask dask-scheduler  # start scheduler
+
+   docker run -it --network host daskdev/dask dask-worker localhost:8786 # start worker
+   docker run -it --network host daskdev/dask dask-worker localhost:8786 # start worker
+   docker run -it --network host daskdev/dask dask-worker localhost:8786 # start worker
+
+   docker run -it --network host daskdev/dask-notebook  # start Jupyter server
+
+
+Extensibility
+-------------
+
+Users can mildly customize the software environment by populating the
+environment variables ``EXTRA_APT_PACKAGES``, ``EXTRA_CONDA_PACKAGES``, and
+``EXTRA_PIP_PACKAGES``.  If these environment variables are set in the container,
+they will trigger calls to the following respectively::
+
+   apt-get install $EXTRA_APT_PACKAGES
+   conda install $EXTRA_CONDA_PACKAGES
+   python -m pip install $EXTRA_PIP_PACKAGES
+
+For example, the following ``conda`` installs the ``joblib`` package into
+the Dask worker software environment:
+
+.. code-block:: bash
+
+   docker run -it -e EXTRA_CONDA_PACKAGES="joblib" daskdev/dask dask-worker localhost:8786
+
+Note that using these can significantly delay the container from starting,
+especially when using ``apt``, or ``conda`` (``pip`` is relatively fast).
+
+Remember that it is important for software versions to match between Dask
+workers and Dask clients.  As a result, it is often useful to include the same
+extra packages in both Jupyter and Worker images.
+
+Source
+------
+
+Docker files are maintained at https://github.com/dask/dask-docker.
+This repository also includes a docker-compose configuration.
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-hpc.rst 2022.02.0+dfsg-1/docs/source/deploying-hpc.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-hpc.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-hpc.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,260 @@
+High Performance Computers
+==========================
+
+Relevant Machines
+-----------------
+
+This page includes instructions and guidelines when deploying Dask on high
+performance supercomputers commonly found in scientific and industry research
+labs.  These systems commonly have the following attributes:
+
+1.  Some mechanism to launch MPI applications or use job schedulers like
+    SLURM, SGE, TORQUE, LSF, DRMAA, PBS, or others
+2.  A shared network file system visible to all machines in the cluster
+3.  A high performance network interconnect, such as Infiniband
+4.  Little or no node-local storage
+
+
+Where to start
+--------------
+
+Most of this page documents various ways and best practices to use Dask on an
+HPC cluster.  This is technical and aimed both at users with some experience
+deploying Dask and also system administrators.
+
+The preferred and simplest way to run Dask on HPC systems today both for new,
+experienced users or administrator is to use
+`dask-jobqueue <https://jobqueue.dask.org>`_.
+
+However, dask-jobqueue is slightly oriented toward interactive analysis usage,
+and it might be better to use tools like dask-mpi in some routine batch
+production workloads.
+
+
+Dask-jobqueue and Dask-drmaa
+----------------------------
+
+`dask-jobqueue <https://jobqueue.dask.org>`_ provides cluster managers for PBS,
+SLURM, LSF, SGE and other resource managers. You can launch a Dask cluster on
+these systems like this.
+
+.. code-block:: python
+
+   from dask_jobqueue import PBSCluster
+
+   cluster = PBSCluster(cores=36,
+                        memory="100GB",
+                        project='P48500028',
+                        queue='premium',
+                        interface='ib0',
+                        walltime='02:00:00')
+
+   cluster.scale(100)  # Start 100 workers in 100 jobs that match the description above
+
+   from dask.distributed import Client
+   client = Client(cluster)    # Connect to that cluster
+
+Dask-jobqueue provides a lot of possibilities like adaptive dynamic scaling
+of workers, we recommend reading the `dask-jobqueue documentation
+<https://jobqueue.dask.org>`_ first to get a basic system running and then
+returning to this documentation for fine-tuning if necessary.
+
+
+Using MPI
+---------
+
+You can launch a Dask cluster using ``mpirun`` or ``mpiexec`` and the
+`dask-mpi <http://mpi.dask.org/en/latest/>`_ command line tool.
+
+.. code-block:: bash
+
+   mpirun --np 4 dask-mpi --scheduler-file /home/$USER/scheduler.json
+
+.. code-block:: python
+
+   from dask.distributed import Client
+   client = Client(scheduler_file='/path/to/scheduler.json')
+
+This depends on the `mpi4py <https://mpi4py.readthedocs.io/>`_ library.  It only
+uses MPI to start the Dask cluster and not for inter-node communication. MPI
+implementations differ: the use of ``mpirun --np 4`` is specific to the
+``mpich`` or ``open-mpi`` MPI implementation installed through conda and linked
+to mpi4py.
+
+.. code-block:: bash
+
+   conda install mpi4py
+
+It is not necessary to use exactly this implementation, but you may want to
+verify that your ``mpi4py`` Python library is linked against the proper
+``mpirun/mpiexec`` executable and that the flags used (like ``--np 4``) are
+correct for your system.  The system administrator of your cluster should be
+very familiar with these concerns and able to help.
+
+In some setups, MPI processes are not allowed to fork other processes. In this
+case, we recommend using ``--no-nanny`` option in order to prevent dask from
+using an additional nanny process to manage workers.
+
+Run ``dask-mpi --help`` to see more options for the ``dask-mpi`` command.
+
+
+Using a Shared Network File System and a Job Scheduler
+------------------------------------------------------
+
+.. note:: This section is not necessary if you use a tool like dask-jobqueue.
+
+Some clusters benefit from a shared File System (NFS, GPFS, Lustre or alike),
+and can use this to communicate the scheduler location to the workers::
+
+   dask-scheduler --scheduler-file /path/to/scheduler.json  # writes address to file
+
+   dask-worker --scheduler-file /path/to/scheduler.json  # reads file for address
+   dask-worker --scheduler-file /path/to/scheduler.json  # reads file for address
+
+.. code-block:: python
+
+   >>> client = Client(scheduler_file='/path/to/scheduler.json')
+
+This can be particularly useful when deploying ``dask-scheduler`` and
+``dask-worker`` processes using a job scheduler like
+SGE/SLURM/Torque/etc.  Here is an example using SGE's ``qsub`` command::
+
+    # Start a dask-scheduler somewhere and write the connection information to a file
+    qsub -b y /path/to/dask-scheduler --scheduler-file /home/$USER/scheduler.json
+
+    # Start 100 dask-worker processes in an array job pointing to the same file
+    qsub -b y -t 1-100 /path/to/dask-worker --scheduler-file /home/$USER/scheduler.json
+
+Note, the ``--scheduler-file`` option is *only* valuable if your scheduler and
+workers share a network file system.
+
+
+High Performance Network
+------------------------
+
+Many HPC systems have both standard Ethernet networks as well as
+high-performance networks capable of increased bandwidth.  You can instruct
+Dask to use the high-performance network interface by using the ``--interface``
+keyword with the ``dask-worker``, ``dask-scheduler``, or ``dask-mpi`` commands or
+the ``interface=`` keyword with the dask-jobqueue ``Cluster`` objects:
+
+.. code-block:: bash
+
+   mpirun --np 4 dask-mpi --scheduler-file /home/$USER/scheduler.json --interface ib0
+
+In the code example above, we have assumed that your cluster has an Infiniband
+network interface called ``ib0``. You can check this by asking your system
+administrator or by inspecting the output of ``ifconfig``
+
+.. code-block:: bash
+
+	$ ifconfig
+	lo          Link encap:Local Loopback                       # Localhost
+				inet addr:127.0.0.1  Mask:255.0.0.0
+				inet6 addr: ::1/128 Scope:Host
+	eth0        Link encap:Ethernet  HWaddr XX:XX:XX:XX:XX:XX   # Ethernet
+				inet addr:192.168.0.101
+				...
+	ib0         Link encap:Infiniband                           # Fast InfiniBand
+				inet addr:172.42.0.101
+
+https://stackoverflow.com/questions/43881157/how-do-i-use-an-infiniband-network-with-dask
+
+
+Local Storage
+-------------
+
+Users often exceed memory limits available to a specific Dask deployment.  In
+normal operation, Dask spills excess data to disk, often to the default
+temporary directory.
+
+However, in HPC systems this default temporary directory may point to an
+network file system (NFS) mount which can cause problems as Dask tries to read
+and write many small files.  *Beware, reading and writing many tiny files from
+many distributed processes is a good way to shut down a national
+supercomputer*.
+
+If available, it's good practice to point Dask workers to local storage, or
+hard drives that are physically on each node.  Your IT administrators will be
+able to point you to these locations.  You can do this with the
+``--local-directory`` or ``local_directory=`` keyword in the ``dask-worker``
+command::
+
+   dask-mpi ... --local-directory /path/to/local/storage
+
+or any of the other Dask Setup utilities, or by specifying the
+following :doc:`configuration value <../../configuration>`:
+
+.. code-block:: yaml
+
+   temporary-directory: /path/to/local/storage
+
+However, not all HPC systems have local storage.  If this is the case then you
+may want to turn off Dask's ability to spill to disk altogether.  See `this
+page <https://distributed.dask.org/en/latest/worker.html#memory-management>`_
+for more information on Dask's memory policies.  Consider changing the
+following values in your ``~/.config/dask/distributed.yaml`` file to disable
+spilling data to disk:
+
+.. code-block:: yaml
+
+   distributed:
+     worker:
+       memory:
+         target: false  # don't spill to disk
+         spill: false  # don't spill to disk
+         pause: 0.80  # pause execution at 80% memory use
+         terminate: 0.95  # restart the worker at 95% use
+
+This stops Dask workers from spilling to disk, and instead relies entirely on
+mechanisms to stop them from processing when they reach memory limits.
+
+As a reminder, you can set the memory limit for a worker using the
+``--memory-limit`` keyword::
+
+   dask-mpi ... --memory-limit 10GB
+
+
+Launch Many Small Jobs
+----------------------
+
+.. note:: This section is not necessary if you use a tool like dask-jobqueue.
+
+HPC job schedulers are optimized for large monolithic jobs with many nodes that
+all need to run as a group at the same time.  Dask jobs can be quite a bit more
+flexible: workers can come and go without strongly affecting the job.  If we
+split our job into many smaller jobs, we can often get through the job
+scheduling queue much more quickly than a typical job.  This is particularly
+valuable when we want to get started right away and interact with a Jupyter
+notebook session rather than waiting for hours for a suitable allocation block
+to become free.
+
+So, to get a large cluster quickly, we recommend allocating a dask-scheduler
+process on one node with a modest wall time (the intended time of your session)
+and then allocating many small single-node dask-worker jobs with shorter wall
+times (perhaps 30 minutes) that can easily squeeze into extra space in the job
+scheduler.  As you need more computation, you can add more of these single-node
+jobs or let them expire.
+
+
+Use Dask to co-launch a Jupyter server
+--------------------------------------
+
+Dask can help you by launching other services alongside it.  For example, you
+can run a Jupyter notebook server on the machine running the ``dask-scheduler``
+process with the following commands
+
+.. code-block:: python
+
+   from dask.distributed import Client
+   client = Client(scheduler_file='scheduler.json')
+
+   import socket
+   host = client.run_on_scheduler(socket.gethostname)
+
+   def start_jlab(dask_scheduler):
+       import subprocess
+       proc = subprocess.Popen(['/path/to/jupyter', 'lab', '--ip', host, '--no-browser'])
+       dask_scheduler.jlab_proc = proc
+
+   client.run_on_scheduler(start_jlab)
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-kubernetes-helm.rst 2022.02.0+dfsg-1/docs/source/deploying-kubernetes-helm.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-kubernetes-helm.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-kubernetes-helm.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,408 @@
+Kubernetes and Helm
+===================
+
+It is easy to launch a Dask cluster and a Jupyter_ notebook server on cloud
+resources using Kubernetes_ and Helm_.
+
+.. _Kubernetes: https://kubernetes.io/
+.. _Helm: https://helm.sh/
+.. _Jupyter: https://jupyter.org/
+
+This is particularly useful when you want to deploy a fresh Python environment
+on Cloud services like Amazon Web Services, Google Compute Engine, or
+Microsoft Azure.
+
+If you already have Python environments running in a pre-existing Kubernetes
+cluster, then you may prefer the :doc:`Kubernetes native<deploying-kubernetes-native>`
+documentation, which is a bit lighter weight.
+
+Launch Kubernetes Cluster
+-------------------------
+
+This document assumes that you have a Kubernetes cluster and Helm installed.
+
+If this is not the case, then you might consider setting up a Kubernetes cluster
+on one of the common cloud providers like Google, Amazon, or
+Microsoft.  We recommend the first part of the documentation in the guide
+`Zero to JupyterHub <https://zero-to-jupyterhub.readthedocs.io/en/latest/>`_
+that focuses on Kubernetes and Helm (you do not need to follow all of these
+instructions). In particular, you don't need to install JupyterHub.
+
+- `Creating a Kubernetes Cluster <https://zero-to-jupyterhub.readthedocs.io/en/latest/create-k8s-cluster.html>`_
+- `Setting up Helm <https://zero-to-jupyterhub.readthedocs.io/en/latest/setup-helm.html>`_
+
+Alternatively, you may want to experiment with Kubernetes locally using
+`Minikube <https://kubernetes.io/docs/getting-started-guides/minikube/>`_.
+
+Which Chart is Right for You?
+-----------------------------
+
+Dask maintains a Helm chart repository containing various charts for the Dask community
+https://helm.dask.org/ .
+You will need to add this to your known channels and update your local charts::
+
+   helm repo add dask https://helm.dask.org/
+   helm repo update
+
+We provides two Helm charts. The right one to choose depends on whether you're
+deploying Dask for a single user or for many users.
+
+
+================  =====================================================================
+Helm Chart        Use Case
+================  =====================================================================
+``dask/dask``     Single-user deployment with one notebook server and one Dask Cluster.
+``dask/daskhub``  Multi-user deployment with JupyterHub and Dask Gateway.
+================  =====================================================================
+
+See :ref:`kubernetes-helm.single` or :ref:`kubernetes-helm.multi` for detailed
+instructions on deploying either of these.
+As you might suspect, deploying ``dask/daskhub`` is a bit more complicated since
+there are more components. If you're just deploying for a single user we'd recommend
+using ``dask/dask``.
+
+.. _kubernetes-helm.single:
+
+Helm Install Dask for a Single User
+-----------------------------------
+
+Once your Kubernetes cluster is ready, you can deploy dask using the Dask Helm_ chart::
+
+   helm install my-dask dask/dask
+
+This deploys a ``dask-scheduler``, several ``dask-worker`` processes, and
+also an optional Jupyter server.
+
+
+Verify Deployment
+^^^^^^^^^^^^^^^^^
+
+This might take a minute to deploy.  You can check its status with
+``kubectl``::
+
+   kubectl get pods
+   kubectl get services
+
+   $ kubectl get pods
+   NAME                                  READY     STATUS              RESTARTS    AGE
+   bald-eel-jupyter-924045334-twtxd      0/1       ContainerCreating   0            1m
+   bald-eel-scheduler-3074430035-cn1dt   1/1       Running             0            1m
+   bald-eel-worker-3032746726-202jt      1/1       Running             0            1m
+   bald-eel-worker-3032746726-b8nqq      1/1       Running             0            1m
+   bald-eel-worker-3032746726-d0chx      0/1       ContainerCreating   0            1m
+
+   $ kubectl get services
+   NAME                 TYPE           CLUSTER-IP      EXTERNAL-IP      PORT(S)                      AGE
+   bald-eel-jupyter     LoadBalancer   10.11.247.201   35.226.183.149   80:30173/TCP                  2m
+   bald-eel-scheduler   LoadBalancer   10.11.245.241   35.202.201.129   8786:31166/TCP,80:31626/TCP   2m
+   kubernetes           ClusterIP      10.11.240.1     <none>           443/TCP
+   48m
+
+You can use the addresses under ``EXTERNAL-IP`` to connect to your now-running
+Jupyter and Dask systems.
+
+Notice the name ``bald-eel``.  This is the name that Helm has given to your
+particular deployment of Dask.  You could, for example, have multiple
+Dask-and-Jupyter clusters running at once, and each would be given a different
+name.  Note that you will need to use this name to refer to your deployment in the future.
+Additionally, you can list all active helm deployments with::
+
+   helm list
+
+   NAME            REVISION        UPDATED                         STATUS      CHART           NAMESPACE
+   bald-eel        1               Wed Dec  6 11:19:54 2017        DEPLOYED    dask-0.1.0      default
+
+
+Connect to Dask and Jupyter
+^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+When we ran ``kubectl get services``, we saw some externally visible IPs:
+
+.. code-block:: bash
+
+   mrocklin@pangeo-181919:~$ kubectl get services
+   NAME                 TYPE           CLUSTER-IP      EXTERNAL-IP      PORT(S)                       AGE
+   bald-eel-jupyter     LoadBalancer   10.11.247.201   35.226.183.149   80:30173/TCP                  2m
+   bald-eel-scheduler   LoadBalancer   10.11.245.241   35.202.201.129   8786:31166/TCP,80:31626/TCP   2m
+   kubernetes           ClusterIP      10.11.240.1     <none>           443/TCP                       48m
+
+We can navigate to these services from any web browser. Here, one is the Dask diagnostic
+dashboard, and the other is the Jupyter server.  You can log into the Jupyter
+notebook server with the password, ``dask``.
+
+You can create a notebook and create a Dask client from there.  The
+``DASK_SCHEDULER_ADDRESS`` environment variable has been populated with the
+address of the Dask scheduler.  This is available in Python from the ``dask.config`` object.
+
+.. code-block:: python
+
+   >>> import dask
+   >>> dask.config.get('scheduler_address')
+   'bald-eel-scheduler:8786'
+
+Although you don't need to use this address, the Dask client will find this
+variable automatically.
+
+.. code-block:: python
+
+   from dask.distributed import Client, config
+   client = Client()
+
+
+Configure Environment
+^^^^^^^^^^^^^^^^^^^^^
+
+By default, the Helm deployment launches three workers using one core each and
+a standard conda environment. We can customize this environment by creating a
+small yaml file that implements a subset of the values in the
+`dask helm chart values.yaml file <https://github.com/dask/helm-chart/blob/main/dask/values.yaml>`_.
+
+For example, we can increase the number of workers, and include extra conda and
+pip packages to install on the both the workers and Jupyter server (these two
+environments should be matched).
+
+.. code-block:: yaml
+
+   # config.yaml
+
+   worker:
+     replicas: 8
+     resources:
+       limits:
+         cpu: 2
+         memory: 7.5G
+       requests:
+         cpu: 2
+         memory: 7.5G
+     env:
+       - name: EXTRA_CONDA_PACKAGES
+         value: numba xarray -c conda-forge
+       - name: EXTRA_PIP_PACKAGES
+         value: s3fs dask-ml --upgrade
+
+   # We want to keep the same packages on the worker and jupyter environments
+   jupyter:
+     enabled: true
+     env:
+       - name: EXTRA_CONDA_PACKAGES
+         value: numba xarray matplotlib -c conda-forge
+       - name: EXTRA_PIP_PACKAGES
+         value: s3fs dask-ml --upgrade
+
+This config file overrides the configuration for the number and size of workers and the
+conda and pip packages installed on the worker and Jupyter containers.  In
+general, we will want to make sure that these two software environments match.
+
+Update your deployment to use this configuration file.  Note that *you will not
+use helm install* for this stage: that would create a *new* deployment on the
+same Kubernetes cluster.  Instead, you will upgrade your existing deployment by
+using the current name::
+
+    helm upgrade bald-eel dask/dask -f config.yaml
+
+This will update those containers that need to be updated.  It may take a minute or so.
+
+As a reminder, you can list the names of deployments you have using ``helm
+list``
+
+
+Check status and logs
+^^^^^^^^^^^^^^^^^^^^^
+
+For standard issues, you should be able to see the worker status and logs using the
+Dask dashboard (in particular, you can see the worker links from the ``info/`` page).
+However, if your workers aren't starting, you can check the status of pods and
+their logs with the following commands:
+
+.. code-block:: bash
+
+   kubectl get pods
+   kubectl logs <PODNAME>
+
+.. code-block:: bash
+
+   mrocklin@pangeo-181919:~$ kubectl get pods
+   NAME                                  READY     STATUS    RESTARTS   AGE
+   bald-eel-jupyter-3805078281-n1qk2     1/1       Running   0          18m
+   bald-eel-scheduler-3074430035-cn1dt   1/1       Running   0          58m
+   bald-eel-worker-1931881914-1q09p      1/1       Running   0          18m
+   bald-eel-worker-1931881914-856mm      1/1       Running   0          18m
+   bald-eel-worker-1931881914-9lgzb      1/1       Running   0          18m
+   bald-eel-worker-1931881914-bdn2c      1/1       Running   0          16m
+   bald-eel-worker-1931881914-jq70m      1/1       Running   0          17m
+   bald-eel-worker-1931881914-qsgj7      1/1       Running   0          18m
+   bald-eel-worker-1931881914-s2phd      1/1       Running   0          17m
+   bald-eel-worker-1931881914-srmmg      1/1       Running   0          17m
+
+   mrocklin@pangeo-181919:~$ kubectl logs bald-eel-worker-1931881914-856mm
+   EXTRA_CONDA_PACKAGES environment variable found.  Installing.
+   Fetching package metadata ...........
+   Solving package specifications: .
+   Package plan for installation in environment /opt/conda/envs/dask:
+   The following NEW packages will be INSTALLED:
+       fasteners: 0.14.1-py36_2 conda-forge
+       monotonic: 1.3-py36_0    conda-forge
+       zarr:      2.1.4-py36_0  conda-forge
+   Proceed ([y]/n)?
+   monotonic-1.3- 100% |###############################| Time: 0:00:00  11.16 MB/s
+   fasteners-0.14 100% |###############################| Time: 0:00:00 576.56 kB/s
+   ...
+
+
+Delete a Helm deployment
+^^^^^^^^^^^^^^^^^^^^^^^^
+
+You can always delete a helm deployment using its name::
+
+   helm delete bald-eel --purge
+
+Note that this does not destroy any clusters that you may have allocated on a
+Cloud service (you will need to delete those explicitly).
+
+
+Avoid the Jupyter Server
+^^^^^^^^^^^^^^^^^^^^^^^^
+
+Sometimes you do not need to run a Jupyter server alongside your Dask cluster.
+
+.. code-block:: yaml
+
+   jupyter:
+     enabled: false
+
+.. _kubernetes-helm.multi:
+
+Helm Install Dask for Mulitple Users
+------------------------------------
+
+The ``dask/daskhub`` Helm Chart deploys JupyterHub_, `Dask Gateway`_, and configures
+the two to work well together. In particular, Dask Gateway is registered as
+a JupyterHub service so that Dask Gateway can re-use JupyterHub's authentication,
+and the JupyterHub environment is configured to connect to the Dask Gateway
+without any arguments.
+
+.. note::
+
+   The ``dask/daskhub`` helm chart came out of the `Pangeo`_ project, a community
+   platform for big data geoscience.
+
+.. _Pangeo: http://pangeo.io/
+.. _Dask Gateway: https://gateway.dask.org/
+.. _JupyterHub: https://jupyterhub.readthedocs.io/en/stable/
+
+The ``dask/daskhub`` helm chart uses the JupyterHub and Dask-Gateway helm charts.
+You'll want to consult the `JupyterHub helm documentation <https://zero-to-jupyterhub.readthedocs.io/en/latest/setup-jupyterhub/setup-jupyterhub.html>`_ and
+and `Dask Gateway helm documentation <https://gateway.dask.org/install-kube.html>`_ for further customization. The default values
+are at https://github.com/dask/helm-chart/blob/main/daskhub/values.yaml.
+
+Verify that you've set up a Kubernetes cluster and added Dask's helm charts:
+
+.. code-block:: console
+
+   $ helm repo add dask https://helm.dask.org/
+   $ helm repo update
+
+JupyterHub and Dask Gateway require a few secret tokens. We'll generate them
+on the command line and insert the tokens in a ``secrets.yaml`` file that will
+be passed to Helm.
+
+Run the following command, and copy the output. This is our `token-1`.
+
+.. code-block:: console
+
+   $ openssl rand -hex 32  # generate token-1
+
+Run command again and copy the output again. This is our `token-2`.
+
+.. code-block:: console
+
+   $ openssl rand -hex 32  # generate token-2
+
+Now substitute those two values for ``<token-1>`` and ``<token-2>`` below.
+Note that ``<token-2>`` is used twice, once for ``jupyterhub.hub.services.dask-gateway.apiToken``, and a second time for ``dask-gateway.gateway.auth.jupyterhub.apiToken``.
+
+.. code-block:: yaml
+
+   # file: secrets.yaml
+   jupyterhub:
+     proxy:
+       secretToken: "<token-1>"
+     hub:
+       services:
+         dask-gateway:
+           apiToken: "<token-2>"
+
+   dask-gateway:
+     gateway:
+       auth:
+         jupyterhub:
+           apiToken: "<token-2>"
+
+Now we're ready to install DaskHub
+
+.. code-block:: console
+
+   $ helm upgrade --wait --install --render-subchart-notes \
+       dhub dask/daskhub \
+       --values=secrets.yaml
+
+
+The output explains how to find the IPs for your JupyterHub depoyment.
+
+.. code-block:: console
+
+   $ kubectl get service proxy-public
+   NAME           TYPE           CLUSTER-IP      EXTERNAL-IP      PORT(S)                      AGE
+   proxy-public   LoadBalancer   10.43.249.239   35.202.158.223   443:31587/TCP,80:30500/TCP   2m40s
+
+
+Creating a Dask Cluster
+^^^^^^^^^^^^^^^^^^^^^^^
+
+To create a Dask cluster on this deployment, users need to connect to the Dask Gateway
+
+.. code-block:: python
+
+   >>> from dask_gateway import GatewayCluster
+   >>> cluster = GatewayCluster()
+   >>> client = cluster.get_client()
+   >>> cluster
+
+Depending on the configuration, users may need to ``cluster.scale(n)`` to
+get workers. See https://gateway.dask.org/ for more on Dask Gateway.
+
+Matching the User Environment
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
+Dask Clients will be running the JupyterHub's singleuser environment. To ensure
+that the same environment is used for the scheduler and workers, you can provide
+it as a Gateway option and configure the ``singleuser`` environment to default
+to the value set by JupyterHub.
+
+.. code-block:: yaml
+
+   # config.yaml
+   jupyterhub:
+     singleuser:
+       extraEnv:
+         DASK_GATEWAY__CLUSTER__OPTIONS__IMAGE: '{JUPYTER_IMAGE_SPEC}'
+
+   dask-gateway:
+     gateway:
+       extraConfig:
+         optionHandler: |
+           from dask_gateway_server.options import Options, Integer, Float, String
+           def option_handler(options):
+               if ":" not in options.image:
+                   raise ValueError("When specifying an image you must also provide a tag")
+               return {
+                   "image": options.image,
+               }
+           c.Backend.cluster_options = Options(
+               String("image", default="pangeo/base-notebook:2020.07.28", label="Image"),
+               handler=option_handler,
+           )
+
+The user environment will need to include ``dask-gateway``. Any packages installed
+manually after the ``singleuser`` pod started will not be included in the worker
+environment.
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-kubernetes-native.rst 2022.02.0+dfsg-1/docs/source/deploying-kubernetes-native.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-kubernetes-native.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-kubernetes-native.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,6 @@
+Kubernetes Native
+=================
+
+See external documentation on Dask-Kubernetes_ for more information.
+
+.. _Dask-Kubernetes: https://kubernetes.dask.org
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-kubernetes.rst 2022.02.0+dfsg-1/docs/source/deploying-kubernetes.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-kubernetes.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-kubernetes.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,84 @@
+Kubernetes
+==========
+
+.. toctree::
+   :maxdepth: 1
+   :hidden:
+
+   Helm <deploying-kubernetes-helm.rst>
+   Native <deploying-kubernetes-native.rst>
+
+Kubernetes_ is a popular system for deploying distributed applications on clusters,
+particularly in the cloud.  You can use Kubernetes to launch Dask workers in the
+following two ways:
+
+1.  **Helm**:
+
+    You can deploy Dask and (optionally) Jupyter or JupyterHub on Kubernetes
+    easily using Helm_
+
+    .. code-block:: bash
+
+       helm repo add dask https://helm.dask.org/    # add the Dask Helm chart repository
+       helm repo update                             # get latest Helm charts
+       # For single-user deployments, use dask/dask
+       helm install my-dask dask/dask               # deploy standard Dask chart
+       # For multi-user deployments, use dask/daskhub
+       helm install my-dask dask/daskhub            # deploy JupyterHub & Dask
+
+    This is a good choice if you want to do the following:
+
+    1.  Run a managed Dask cluster for a long period of time
+    2.  Also deploy a Jupyter / JupyterHub server from which to run code
+    3.  Share the same Dask cluster between many automated services
+    4.  Try out Dask for the first time on a cloud-based system
+        like Amazon, Google, or Microsoft Azure where you already have
+        a Kubernetes cluster. If you don't already have Kubernetes deployed,
+        see our :doc:`Cloud documentation <deploying-cloud>`.
+
+    You can also use the ``HelmCluster`` cluster manager from dask-kubernetes to manage your
+    Helm Dask cluster from within your Python session.
+
+    .. code-block:: python
+
+       from dask_kubernetes import HelmCluster
+
+       cluster = HelmCluster(release_name="myrelease")
+       cluster.scale(10)
+
+    .. note::
+
+      For more information, see :doc:`Dask and Helm documentation <deploying-kubernetes-helm>`.
+
+2.  **Native**:
+    You can quickly deploy Dask workers on Kubernetes
+    from within a Python script or interactive session using Dask-Kubernetes_
+
+    .. code-block:: python
+
+       from dask_kubernetes import KubeCluster
+       cluster = KubeCluster.from_yaml('worker-template.yaml')
+       cluster.scale(20)  # add 20 workers
+       cluster.adapt()    # or create and destroy workers dynamically based on workload
+
+       from dask.distributed import Client
+       client = Client(cluster)
+
+    This is a good choice if you want to do the following:
+
+    1.  Dynamically create a personal and ephemeral deployment for interactive use
+    2.  Allow many individuals the ability to launch their own custom dask deployments,
+        rather than depend on a centralized system
+    3.  Quickly adapt Dask cluster size to the current workload
+
+    .. note::
+
+      For more information, see Dask-Kubernetes_ documentation.
+
+You may also want to see the documentation on using
+:doc:`Dask with Docker containers <deploying-docker>`
+to help you manage your software environments on Kubernetes.
+
+.. _Kubernetes: https://kubernetes.io/
+.. _Dask-Kubernetes: https://kubernetes.dask.org/
+.. _Helm: https://helm.sh/
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-python-advanced.rst 2022.02.0+dfsg-1/docs/source/deploying-python-advanced.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-python-advanced.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-python-advanced.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,210 @@
+Python API (advanced)
+=====================
+
+.. currentmodule:: distributed
+
+In some rare cases, experts may want to create ``Scheduler``, ``Worker``, and
+``Nanny``  objects explicitly in Python.  This is often necessary when making
+tools to automatically deploy Dask in custom settings.
+
+It is more common to create a :doc:`Local cluster with Client() on a single
+machine <deploying-python>` or use the :doc:`Command Line Interface (CLI) <deploying-cli>`.
+New readers are recommended to start there.
+
+If you do want to start Scheduler and Worker objects yourself you should be a
+little familiar with ``async``/``await`` style Python syntax.  These objects
+are awaitable and are commonly used within ``async with`` context managers.
+Here are a few examples to show a few ways to start and finish things.
+
+Full Example
+------------
+
+.. autosummary::
+   Scheduler
+   Worker
+   Client
+
+We first start with a comprehensive example of setting up a Scheduler, two Workers,
+and one Client in the same event loop, running a simple computation, and then
+cleaning everything up.
+
+.. code-block:: python
+
+   import asyncio
+   from dask.distributed import Scheduler, Worker, Client
+
+   async def f():
+       async with Scheduler() as s:
+           async with Worker(s.address) as w1, Worker(s.address) as w2:
+               async with Client(s.address, asynchronous=True) as client:
+                   future = client.submit(lambda x: x + 1, 10)
+                   result = await future
+                   print(result)
+
+   asyncio.get_event_loop().run_until_complete(f())
+
+Now we look at simpler examples that build up to this case.
+
+Scheduler
+---------
+
+.. autosummary::
+   Scheduler
+
+We create scheduler by creating a ``Scheduler()`` object, and then ``await``
+that object to wait for it to start up.  We can then wait on the ``.finished``
+method to wait until it closes.  In the meantime the scheduler will be active
+managing the cluster..
+
+.. code-block:: python
+
+   import asyncio
+   from dask.distributed import Scheduler, Worker
+
+   async def f():
+       s = Scheduler()        # scheduler created, but not yet running
+       s = await s            # the scheduler is running
+       await s.finished()     # wait until the scheduler closes
+
+   asyncio.get_event_loop().run_until_complete(f())
+
+This program will run forever, or until some external process connects to the
+scheduler and tells it to stop.  If you want to close things yourself you can
+close any ``Scheduler``, ``Worker``, ``Nanny``, or ``Client`` class by awaiting
+the ``.close`` method:
+
+.. code-block:: python
+
+   await s.close()
+
+
+Worker
+------
+
+.. autosummary::
+   Worker
+
+The worker follows the same API.
+The only difference is that the worker needs to know the address of the
+scheduler.
+
+.. code-block:: python
+
+   import asyncio
+   from dask.distributed import Scheduler, Worker
+
+   async def f(scheduler_address):
+       w = await Worker(scheduler_address)
+       await w.finished()
+
+   asyncio.get_event_loop().run_until_complete(f("tcp://127.0.0.1:8786"))
+
+
+Start many in one event loop
+----------------------------
+
+.. autosummary::
+   Scheduler
+   Worker
+
+We can run as many of these objects as we like in the same event loop.
+
+.. code-block:: python
+
+   import asyncio
+   from dask.distributed import Scheduler, Worker
+
+   async def f():
+       s = await Scheduler()
+       w = await Worker(s.address)
+       await w.finished()
+       await s.finished()
+
+   asyncio.get_event_loop().run_until_complete(f())
+
+
+Use Context Managers
+--------------------
+
+We can also use ``async with`` context managers to make sure that we clean up
+properly.  Here is the same example as from above:
+
+.. code-block:: python
+
+   import asyncio
+   from dask.distributed import Scheduler, Worker
+
+   async def f():
+       async with Scheduler() as s:
+           async with Worker(s.address) as w:
+               await w.finished()
+               await s.finished()
+
+   asyncio.get_event_loop().run_until_complete(f())
+
+Alternatively, in the example below we also include a ``Client``, run a small
+computation, and then allow things to clean up after that computation..
+
+.. code-block:: python
+
+   import asyncio
+   from dask.distributed import Scheduler, Worker, Client
+
+   async def f():
+       async with Scheduler() as s:
+           async with Worker(s.address) as w1, Worker(s.address) as w2:
+               async with Client(s.address, asynchronous=True) as client:
+                   future = client.submit(lambda x: x + 1, 10)
+                   result = await future
+                   print(result)
+
+   asyncio.get_event_loop().run_until_complete(f())
+
+This is equivalent to creating and ``awaiting`` each server, and then calling
+``.close`` on each as we leave the context.
+In this example we don't wait on ``s.finished()``, so this will terminate
+relatively quickly.  You could have called ``await s.finished()`` though if you
+wanted this to run forever.
+
+Nanny
+-----
+
+.. autosummary::
+   Nanny
+
+Alternatively, we can replace ``Worker`` with ``Nanny`` if we want your workers
+to be managed in a separate process.  The ``Nanny`` constructor follows the
+same API. This allows workers to restart themselves in case of failure. Also,
+it provides some additional monitoring, and is useful when coordinating many
+workers that should live in different processes in order to avoid the GIL_.
+
+.. code-block:: python
+
+   # w = await Worker(s.address)
+   w = await Nanny(s.address)
+
+.. _GIL: https://docs.python.org/3/glossary.html#term-gil
+
+
+API
+---
+
+These classes have a variety of keyword arguments that you can use to control
+their behavior.  See the API documentation below for more information.
+
+Scheduler
+~~~~~~~~~
+.. autoclass:: Scheduler
+   :members:
+
+Worker
+~~~~~~
+
+.. autoclass:: Worker
+   :members:
+
+Nanny
+~~~~~
+
+.. autoclass:: Nanny
+   :members:
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-python.rst 2022.02.0+dfsg-1/docs/source/deploying-python.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-python.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-python.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,98 @@
+Python API
+==========
+
+You can create a ``dask.distributed`` scheduler by importing and creating a
+``Client`` with no arguments.  This overrides whatever default was previously
+set.
+
+.. code-block:: python
+
+   from dask.distributed import Client
+   client = Client()
+
+You can navigate to ``http://localhost:8787/status`` to see the diagnostic
+dashboard if you have Bokeh installed.
+
+Client
+------
+
+You can trivially set up a local cluster on your machine by instantiating a Dask
+Client with no arguments
+
+.. code-block:: python
+
+   from dask.distributed import Client
+   client = Client()
+
+This sets up a scheduler in your local process along with a number of workers and
+threads per worker related to the number of cores in your machine.
+
+If you want to run workers in your same process, you can pass the
+``processes=False`` keyword argument.
+
+.. code-block:: python
+
+   client = Client(processes=False)
+
+This is sometimes preferable if you want to avoid inter-worker communication
+and your computations release the GIL.  This is common when primarily using
+NumPy or Dask Array.
+
+
+LocalCluster
+------------
+
+The ``Client()`` call described above is shorthand for creating a LocalCluster
+and then passing that to your client.
+
+.. code-block:: python
+
+   from dask.distributed import Client, LocalCluster
+   cluster = LocalCluster()
+   client = Client(cluster)
+
+This is equivalent, but somewhat more explicit.
+
+You may want to look at the
+keyword arguments available on ``LocalCluster`` to understand the options available
+to you on handling the mixture of threads and processes, like specifying explicit
+ports, and so on.
+
+Cluster manager features
+------------------------
+
+Instantiating a cluster manager class like ``LocalCluster`` and then passing it to the
+``Client`` is a common pattern. Cluster managers also provide useful utilities to help
+you understand what is going on.
+
+For example you can retreive the Dashboard URL.
+
+.. code-block:: python
+
+   >>> cluster.dashboard_link
+   'http://127.0.0.1:8787/status'
+
+You can retreive logs from cluster components.
+
+.. code-block:: python
+
+   >>> cluster.get_logs()
+   {'Cluster': '',
+   'Scheduler': "distributed.scheduler - INFO - Clear task state\ndistributed.scheduler - INFO -   S...
+
+If you are using a cluster manager that supports scaling you can modify the number of workers manually
+or automatically based on workload.
+
+.. code-block:: python
+
+   >>> cluster.scale(10)  # Sets the number of workers to 10
+
+   >>> cluster.adapt(minimum=1, maximum=10)  # Allows the cluster to auto scale to 10 when tasks are computed
+
+Reference
+---------
+
+.. currentmodule:: distributed.deploy.local
+
+.. autoclass:: LocalCluster
+   :members:
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying.rst 2022.02.0+dfsg-1/docs/source/deploying.rst
--- 2022.01.0+dfsg-1/docs/source/deploying.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,146 @@
+Deploying Clusters
+==================
+
+.. toctree::
+   :maxdepth: 1
+   :hidden:
+
+   deploying-python.rst
+   deploying-cli.rst
+   deploying-ssh.rst
+   deploying-docker.rst
+   deploying-hpc.rst
+   deploying-kubernetes.rst
+   deploying-cloud.rst
+   deploying-python-advanced.rst
+
+The ``dask.distributed`` scheduler works well on a single machine and scales to many machines
+in a cluster. We recommend using ``dask.distributed`` clusters at all scales for the following
+reasons:
+
+1.  It provides access to asynchronous API, notably :doc:`Futures <../../futures>`
+2.  It provides a diagnostic dashboard that can provide valuable insight on
+    performance and progress
+3.  It handles data locality with sophistication, and so can be more
+    efficient than the multiprocessing scheduler on workloads that require
+    multiple processes
+
+This page describes various ways to set up Dask clusters on different hardware, either
+locally on your own machine or on a distributed cluster.  If you are just
+getting started then you can save this page for later as Dask runs perfectly well on a single machine
+without a distributed scheduler. But once you start using Dask in anger you'll find a lot of benefit
+both in terms of scaling and debugging by using the distirbuted scheduler.
+
+You can continue reading or watch the screencast below:
+
+.. raw:: html
+
+   <iframe width="560"
+           height="315"
+           src="https://www.youtube.com/embed/TQM9zIBzNBo"
+           style="margin: 0 auto 20px auto; display: block;"
+           frameborder="0"
+           allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture"
+           allowfullscreen></iframe>
+
+If you import Dask, set up a computation, and call ``compute``, then you
+will use the single-machine scheduler by default.  To use the ``dask.distributed``
+scheduler you must set up a ``Client``.
+
+.. code-block:: python
+
+   import dask.dataframe as dd
+   df = dd.read_csv(...)
+   df.x.sum().compute()  # This uses the single-machine scheduler by default
+
+.. code-block:: python
+
+   from dask.distributed import Client
+   client = Client(...)  # Connect to distributed cluster and override default
+   df.x.sum().compute()  # This now runs on the distributed system
+
+There are many ways to start the distributed scheduler and worker components that your client
+needs to connect to. You can run them manually using :doc:`command line tools <deploying-cli>`
+but often the most straight forward way is to use a *cluster manager* utility class.
+
+.. code-block:: python
+
+   from dask.distributed import Client, LocalCluster
+   cluster = LocalCluster()  # Launches a scheduler and workers locally
+   client = Client(cluster)  # Connect to distributed cluster and override default
+   df.x.sum().compute()  # This now runs on the distributed system
+
+There are a number of different *cluster managers* available, so you can use
+Dask distributed with a range of platforms. These *cluster managers* deploy a scheduler
+and the necessary workers as determined by communicating with the *resource manager*.
+All *cluster managers* follow the same interface but have platform specific configuration
+options. This makes it convenient to switch from your local machine to a remote multi-node
+cluster without sacrificing the flexibility of the platform you are deploying on.
+
+`Dask Jobqueue <https://github.com/dask/dask-jobqueue>`_, for example, is a set of
+*cluster managers* for HPC users and works with job queueing systems
+(in this case, the *resource manager*) such as `PBS <https://en.wikipedia.org/wiki/Portable_Batch_System>`_,
+`Slurm <https://en.wikipedia.org/wiki/Slurm_Workload_Manager>`_,
+and `SGE <https://en.wikipedia.org/wiki/Oracle_Grid_Engine>`_.
+Those workers are then allocated physical hardware resources.
+
+.. code-block:: python
+
+   from dask.distributed import Client
+   from dask_jobqueue import PBSCluster
+   cluster = PBSCluster()  # Launches a scheduler and workers on HPC via PBS
+   client = Client(cluster)  # Connect to distributed cluster and override default
+   df.x.sum().compute()  # This now runs on the distributed system
+
+.. figure:: images/dask-cluster-manager.svg
+   :scale: 50%
+
+   An overview of cluster management with Dask distributed.
+
+To summarize, you can use the default, single-machine scheduler to use Dask
+on your local machine. If you'd like use a cluster *or* simply take advantage
+of the :doc:`extensive diagnostics <../diagnostics-distributed>`,
+you can use Dask distributed. The following resources explain
+in more detail how to set up Dask on a variety of local and distributed hardware:
+
+- Single Machine:
+    - :doc:`Default Scheduler <scheduling>`: The no-setup default.
+      Uses local threads or processes for larger-than-memory processing
+    - :doc:`dask.distributed <deploying-python>`: The sophistication of
+      the newer system on a single machine.  This provides more advanced
+      features while still requiring almost no setup.
+- Distributed computing:
+    - `Beginner's Guide to Configuring a Dask distributed Cluster <https://blog.dask.org/2020/07/30/beginners-config>`_
+    - `Overview of cluster management options <https://blog.dask.org/2020/07/23/current-state-of-distributed-dask-clusters>`_
+    - :doc:`Manual Setup <deploying-cli>`: The command line interface to set up
+      ``dask-scheduler`` and ``dask-worker`` processes.  Useful for IT or
+      anyone building a deployment solution.
+    - :doc:`SSH <deploying-ssh>`: Use SSH to set up Dask across an un-managed
+      cluster.
+    - :doc:`High Performance Computers <deploying-hpc>`: How to run Dask on
+      traditional HPC environments using tools like MPI, or job schedulers like
+      SLURM, SGE, TORQUE, LSF, and so on.
+    - :doc:`Kubernetes <deploying-kubernetes>`: Deploy Dask with the
+      popular Kubernetes resource manager using either Helm or a native deployment.
+    - `YARN / Hadoop <https://yarn.dask.org/en/latest/>`_: Deploy
+      Dask on YARN clusters, such as are found in traditional Hadoop
+      installations.
+    - `Dask Gateway <https://gateway.dask.org/>`_ provides a secure,
+      multi-tenant server for managing Dask clusters and allows users to launch
+      and use Dask clusters in a shared cluster environment.
+    - :doc:`Python API (advanced) <deploying-python-advanced>`: Create
+      ``Scheduler`` and ``Worker`` objects from Python as part of a distributed
+      Tornado TCP application.  This page is useful for those building custom
+      frameworks.
+    - :doc:`Docker <deploying-docker>` images are available and may be useful
+      in some of the solutions above.
+    - :doc:`Cloud <deploying-cloud>` for current recommendations on how to
+      deploy Dask and Jupyter on common cloud providers like Amazon, Google, or
+      Microsoft Azure.
+- Hosted / managed Dask clusters (listed in alphabetical order):
+    - `Coiled <https://coiled.io/>`_ handles the creation and management of
+      Dask clusters on cloud computing environments (AWS, Azure, and GCP).
+    - `Domino Data Lab <https://www.dominodatalab.com/>`_ lets users create
+      Dask clusters in a hosted platform.
+    - `Saturn Cloud <https://saturncloud.io/>`_ lets users create
+      Dask clusters in a hosted platform or within their own AWS accounts.
diff -pruN 2022.01.0+dfsg-1/docs/source/deploying-ssh.rst 2022.02.0+dfsg-1/docs/source/deploying-ssh.rst
--- 2022.01.0+dfsg-1/docs/source/deploying-ssh.rst	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/deploying-ssh.rst	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,54 @@
+SSH
+===
+
+It is easy to set up Dask on informally managed networks of machines using SSH.
+This can be done manually using SSH and the
+Dask :doc:`command line interface <deploying-cli>`,
+or automatically using either the :class:`dask.distributed.SSHCluster` Python *cluster manager* or the
+``dask-ssh`` command line tool. This document describes both of these options.
+
+.. note::
+   Before instaniating a ``SSHCluster`` it is recommended to configure keyless SSH
+   for your local machine and other machines. For example, on a Mac to SSH into
+   localhost (local machine) you need to ensure the Remote Login option is set in
+   System Preferences -> Sharing. In addition, ``id_rsa.pub`` should be in
+   ``authorized_keys`` for keyless login.
+
+Python Interface
+----------------
+
+.. currentmodule:: dask.distributed
+
+.. autofunction:: SSHCluster
+
+Command Line
+------------
+
+The convenience script ``dask-ssh`` opens several SSH connections to your
+target computers and initializes the network accordingly. You can
+give it a list of hostnames or IP addresses::
+
+   $ dask-ssh 192.168.0.1 192.168.0.2 192.168.0.3 192.168.0.4
+
+Or you can use normal UNIX grouping::
+
+   $ dask-ssh 192.168.0.{1,2,3,4}
+
+Or you can specify a hostfile that includes a list of hosts::
+
+   $ cat hostfile.txt
+   192.168.0.1
+   192.168.0.2
+   192.168.0.3
+   192.168.0.4
+
+   $ dask-ssh --hostfile hostfile.txt
+
+.. note::
+
+   The command line documentation here may differ depending on your installed
+   version. We recommend referring to the output of ``dask-ssh --help``.
+
+.. click:: distributed.cli.dask_ssh:main
+   :prog: dask-ssh
+   :show-nested:
diff -pruN 2022.01.0+dfsg-1/docs/source/ext/dask_config_sphinx_ext.py 2022.02.0+dfsg-1/docs/source/ext/dask_config_sphinx_ext.py
--- 2022.01.0+dfsg-1/docs/source/ext/dask_config_sphinx_ext.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/ext/dask_config_sphinx_ext.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,7 +1,8 @@
 import requests
 import yaml
 from docutils import nodes
-from docutils.parsers.rst import Directive, directives
+from docutils.parsers.rst import directives  # type: ignore  # Not in type stubs
+from docutils.parsers.rst import Directive
 
 
 def get_remote_yaml(url):
diff -pruN 2022.01.0+dfsg-1/docs/source/faq.rst 2022.02.0+dfsg-1/docs/source/faq.rst
--- 2022.01.0+dfsg-1/docs/source/faq.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/faq.rst	2022-02-11 16:21:12.000000000 +0000
@@ -169,7 +169,7 @@ those on Hadoop, HPC, Kubernetes, and Cl
     - `Dask MPI <https://mpi.dask.org>`_ which uses MPI for deployment in
       batch settings
 
-    For more information see :doc:`how-to/deploy-dask/hpc`
+    For more information see :doc:`deploying-hpc`
 
 3.  **Kubernetes/Cloud**: Newer clusters may employ Kubernetes for deployment.
     This is particularly commonly used today on major cloud providers,
@@ -182,7 +182,7 @@ those on Hadoop, HPC, Kubernetes, and Cl
     - **Dask-Kubernetes**: for native Kubernetes integration for fast moving
       or ephemeral deployments.
 
-    For more information see :doc:`how-to/deploy-dask/kubernetes`
+    For more information see :doc:`deploying-kubernetes`
 
 
 Is Dask secure?
diff -pruN 2022.01.0+dfsg-1/docs/source/graphviz.rst 2022.02.0+dfsg-1/docs/source/graphviz.rst
--- 2022.01.0+dfsg-1/docs/source/graphviz.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/graphviz.rst	2022-02-11 16:21:12.000000000 +0000
@@ -15,6 +15,9 @@ where parallelism may not be possible,
 or areas where many tasks depend on each other,
 which may cause a great deal of communication.
 
+Visualize the low level graph
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+
 The ``.visualize`` method and ``dask.visualize`` function work exactly like
 the ``.compute`` method and ``dask.compute`` function,
 except that rather than computing the result,
@@ -32,10 +35,12 @@ In the case that you prefer to visualize
    y = x + x.T
 
    # y.compute()
+
+   # visualize the low level Dask graph
    y.visualize(filename='transpose.svg')
 
 .. image:: images/transpose.svg
-   :alt: Dask task graph for adding an array to its transpose
+   :alt: Dask low level task graph for adding an array to its transpose
 
 Note that the ``visualize`` function is powered by the `GraphViz <https://www.graphviz.org/>`_
 system library.  This library has a few considerations:
@@ -48,3 +53,44 @@ system library.  This library has a few
     For large computations you might have to simplify your computation a bit
     for the visualize method to work well.
 
+Visualize the high level graph
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+The low level Dask task graph can be overwhelimg, especially for large computations.
+A more concise alternative is to look at the Dask high level graph instead.
+The high level graph can be visualized using ``.dask.visualize()``.
+
+.. code-block:: python
+
+   import dask.array as da
+   x = da.ones((15, 15), chunks=(5, 5))
+   y = x + x.T
+
+   # visualize the high level Dask graph
+   y.dask.visualize(filename='transpose-hlg.svg')
+
+.. image:: images/transpose-hlg-hovertooltip.png
+   :alt: Dask high level task graph for adding an array to its transpose
+
+Hovering your mouse above each high level graph label will bring up
+a tooltip with more detailed information about that layer.
+Note that if you save the graph to disk using the ``filename=`` keyword argument
+in ``visualize``, then the tooltips wil only be preserved by the SVG image format.
+
+High level graph HTML representation
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Dask high level graphs also have their own HTML representation,
+which is useful if you like to work with Jupyter notebooks.
+
+.. code-block:: python
+
+   import dask.array as da
+   x = da.ones((15, 15), chunks=(5, 5))
+   y = x + x.T
+
+   y.dask  # shows the HTML representation in a Jupyter notebook
+
+.. image:: images/transpose-hlg-html-repr.png
+   :alt: Dask high level graph HTML representation
+
+You can click on any of the layer names to expand or collapse more detailed
+information about each layer.
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/adaptive.rst 2022.02.0+dfsg-1/docs/source/how-to/adaptive.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/adaptive.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/adaptive.rst	2022-02-11 16:21:12.000000000 +0000
@@ -72,10 +72,10 @@ Dependence on a Resource Manager
 The Dask scheduler does not know how to launch workers on its own. Instead, it
 relies on an external resource scheduler like Kubernetes above, or
 Yarn, SGE, SLURM, Mesos, or some other in-house system (see :doc:`how to deploy Dask
-clusters <../how-to/deploy-dask-clusters>` for options).  In order to use adaptive deployments, you
+clusters <../deploying>` for options).  In order to use adaptive deployments, you
 must provide some mechanism for the scheduler to launch new workers.  Typically,
 this is done by using one of the solutions listed in the :doc:`how to deploy Dask
-clusters <../how-to/deploy-dask-clusters>`, or by subclassing from the Cluster superclass and
+clusters <../deploying>`, or by subclassing from the Cluster superclass and
 implementing that API.
 
 .. autosummary::
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/connect-to-remote-data.rst 2022.02.0+dfsg-1/docs/source/how-to/connect-to-remote-data.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/connect-to-remote-data.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/connect-to-remote-data.rst	2022-02-11 16:21:12.000000000 +0000
@@ -298,9 +298,9 @@ Internals
 ---------
 
 Dask contains internal tools for extensible data ingestion in the
-``dask.bytes`` package and using `fsspec`_.
+``dask.bytes`` package and uses external tools like ``open_files`` from `fsspec`_.
 .  *These functions are developer-focused rather than for
-direct consumption by users.  These functions power user facing functions like*
+direct consumption by users.  These functions power user-facing functions like*
 ``dd.read_csv`` *and* ``db.read_text`` *which are probably more useful for most
 users.*
 
@@ -309,20 +309,18 @@ users.*
 
 .. autosummary::
    read_bytes
-   open_files
 
-These functions are extensible in their output formats (bytes, file objects),
-their input locations (file system, S3, HDFS), line delimiters, and compression
+This function is extensible in its output format (bytes),
+its input locations (file system, S3, HDFS), line delimiters, and compression
 formats.
 
-Both functions are *lazy*, returning either
-pointers to blocks of bytes (``read_bytes``) or open file objects
-(``open_files``).  They can handle different storage backends by prepending
-protocols like ``s3://`` or ``hdfs://`` (see below). They handle compression formats
+This function is *lazy*, returning pointers to blocks of bytes (``read_bytes``).  
+It handles different storage backends by prepending
+protocols like ``s3://`` or ``hdfs://`` (see below). It handles compression formats
 listed in ``fsspec.compression``, some of which may require additional packages
 to be installed.
 
-These functions are not used for all data sources.  Some data sources like HDF5
+This function is not used for all data sources.  Some data sources like HDF5
 are quite particular and receive custom treatment.
 
 Delimiters
@@ -351,11 +349,10 @@ added by inserting functions into dictio
 added directly to the codebase.
 
 However, most compression technologies like ``gzip`` do not support efficient
-random access, and so are useful for streaming ``open_files`` but not useful for
+random access, and so are useful for streaming ``fsspec.open_files`` but not useful for
 ``read_bytes`` which splits files at various points.
 
 API
 ^^^
 
 .. autofunction:: read_bytes
-.. autofunction:: open_files
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/cli.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/cli.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/cli.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/cli.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,100 +0,0 @@
-Command Line
-============
-
-This is the most fundamental way to deploy Dask on multiple machines.  In
-production environments, this process is often automated by some other resource
-manager. Hence, it is rare that people need to follow these instructions
-explicitly.  Instead, these instructions are useful for IT professionals who
-may want to set up automated services to deploy Dask within their institution.
-
-A ``dask.distributed`` network consists of one ``dask-scheduler`` process and
-several ``dask-worker`` processes that connect to that scheduler.  These are
-normal Python processes that can be executed from the command line.  We launch
-the ``dask-scheduler`` executable in one process and the ``dask-worker``
-executable in several processes, possibly on different machines.
-
-To accomplish this, launch ``dask-scheduler`` on one node::
-
-   $ dask-scheduler
-   Scheduler at:   tcp://192.0.0.100:8786
-
-Then, launch ``dask-worker`` on the rest of the nodes, providing the address to
-the node that hosts ``dask-scheduler``::
-
-   $ dask-worker tcp://192.0.0.100:8786
-   Start worker at:  tcp://192.0.0.1:12345
-   Registered to:    tcp://192.0.0.100:8786
-
-   $ dask-worker tcp://192.0.0.100:8786
-   Start worker at:  tcp://192.0.0.2:40483
-   Registered to:    tcp://192.0.0.100:8786
-
-   $ dask-worker tcp://192.0.0.100:8786
-   Start worker at:  tcp://192.0.0.3:27372
-   Registered to:    tcp://192.0.0.100:8786
-
-The workers connect to the scheduler, which then sets up a long-running network
-connection back to the worker.  The workers will learn the location of other
-workers from the scheduler.
-
-
-Handling Ports
---------------
-
-The scheduler and workers both need to accept TCP connections on an open port.
-By default, the scheduler binds to port ``8786`` and the worker binds to a
-random open port.  If you are behind a firewall then you may have to open
-particular ports or tell Dask to listen on particular ports with the ``--port``
-and ``--worker-port`` keywords.::
-
-   dask-scheduler --port 8000
-   dask-worker --dashboard-address 8000 --nanny-port 8001
-
-
-Nanny Processes
----------------
-
-Dask workers are run within a nanny process that monitors the worker process
-and restarts it if necessary.
-
-
-Diagnostic Web Servers
-----------------------
-
-Additionally, Dask schedulers and workers host interactive diagnostic web
-servers using `Bokeh <https://docs.bokeh.org>`_.  These are optional, but
-generally useful to users.  The diagnostic server on the scheduler is
-particularly valuable, and is served on port ``8787`` by default (configurable
-with the ``--dashboard-address`` keyword).
-
-For more information about relevant ports, please take a look at the available
-:ref:`command line options <worker-scheduler-cli-options>`.
-
-Automated Tools
----------------
-
-There are various mechanisms to deploy these executables on a cluster, ranging
-from manually SSH-ing into all of the machines to more automated systems like
-SGE/SLURM/Torque or Yarn/Mesos.  Additionally, cluster SSH tools exist to send
-the same commands to many machines.  We recommend searching online for "cluster
-ssh" or "cssh".
-
-
-.. _worker-scheduler-cli-options:
-
-CLI Options
------------
-
-.. note::
-
-   The command line documentation here may differ depending on your installed
-   version. We recommend referring to the output of ``dask-scheduler --help``
-   and ``dask-worker --help``.
-
-.. click:: distributed.cli.dask_scheduler:main
-   :prog: dask-scheduler
-   :show-nested:
-
-.. click:: distributed.cli.dask_worker:main
-   :prog: dask-worker
-   :show-nested:
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/cloud.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/cloud.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/cloud.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/cloud.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,40 +0,0 @@
-Cloud Deployments
-=================
-
-There are a variety of ways to deploy Dask on cloud providers.
-Cloud providers provide managed services,
-like Kubernetes, Yarn, or custom APIs with which Dask can connect easily.
-You may want to consider the following options:
-
-1.  A managed Kubernetes service and Dask's
-    :doc:`Kubernetes and Helm integration <kubernetes-helm>`.
-2.  A managed Yarn service,
-    like `Amazon EMR <https://aws.amazon.com/emr/>`_
-    or `Google Cloud DataProc <https://cloud.google.com/dataproc/>`_
-    and `Dask-Yarn <https://yarn.dask.org>`_.
-
-    Specific documentation for the popular Amazon EMR service can be found
-    `here <https://yarn.dask.org/en/latest/aws-emr.html>`_
-3.  Vendor specific services, like Amazon ECS, and
-    `Dask Cloud Provider <https://cloudprovider.dask.org/en/latest/>`_
-
-Data Access
------------
-
-You may want to install additional libraries in your Jupyter and worker images
-to access the object stores of each cloud:
-
--  `s3fs <https://s3fs.readthedocs.io/>`_ for Amazon's S3
--  `gcsfs <https://gcsfs.readthedocs.io/>`_ for Google's GCS
--  `adlfs <https://github.com/dask/adlfs/>`_ for Microsoft's ADL
-
-Historical Libraries
---------------------
-
-Dask previously maintained libraries for deploying Dask on
-Amazon's EC2 and Google GKE.
-Due to sporadic interest,
-and churn both within the Dask library and EC2 itself,
-these were not well maintained.
-They have since been deprecated in favor of the
-:doc:`Kubernetes and Helm <kubernetes-helm>` solution.
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/docker.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/docker.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/docker.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/docker.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,69 +0,0 @@
-Docker Images
-=============
-
-Example docker images are maintained at https://github.com/dask/dask-docker
-and https://hub.docker.com/r/daskdev/ .
-
-Each image installs the full Dask conda package (including the distributed
-scheduler), Numpy, and Pandas on top of a Miniconda installation on top of
-a Debian image.
-
-These images are large, around 1GB.
-
--   ``daskdev/dask``: This a normal debian + miniconda image with the full Dask
-    conda package (including the distributed scheduler), Numpy, and Pandas.
-    This image is about 1GB in size.
-
--   ``daskdev/dask-notebook``: This is based on the
-    `Jupyter base-notebook image <https://hub.docker.com/r/jupyter/base-notebook/>`_
-    and so it is suitable for use both normally as a Jupyter server, and also as
-    part of a JupyterHub deployment.  It also includes a matching Dask software
-    environment described above.  This image is about 2GB in size.
-
-Example
--------
-
-Here is a simple example on the local host network
-
-.. code-block:: bash
-
-   docker run -it --network host daskdev/dask dask-scheduler  # start scheduler
-
-   docker run -it --network host daskdev/dask dask-worker localhost:8786 # start worker
-   docker run -it --network host daskdev/dask dask-worker localhost:8786 # start worker
-   docker run -it --network host daskdev/dask dask-worker localhost:8786 # start worker
-
-   docker run -it --network host daskdev/dask-notebook  # start Jupyter server
-
-
-Extensibility
--------------
-
-Users can mildly customize the software environment by populating the
-environment variables ``EXTRA_APT_PACKAGES``, ``EXTRA_CONDA_PACKAGES``, and
-``EXTRA_PIP_PACKAGES``.  If these environment variables are set in the container,
-they will trigger calls to the following respectively::
-
-   apt-get install $EXTRA_APT_PACKAGES
-   conda install $EXTRA_CONDA_PACKAGES
-   python -m pip install $EXTRA_PIP_PACKAGES
-
-For example, the following ``conda`` installs the ``joblib`` package into
-the Dask worker software environment:
-
-.. code-block:: bash
-
-   docker run -it -e EXTRA_CONDA_PACKAGES="joblib" daskdev/dask dask-worker localhost:8786
-
-Note that using these can significantly delay the container from starting,
-especially when using ``apt``, or ``conda`` (``pip`` is relatively fast).
-
-Remember that it is important for software versions to match between Dask
-workers and Dask clients.  As a result, it is often useful to include the same
-extra packages in both Jupyter and Worker images.
-
-Source
-------
-
-Docker files are maintained at https://github.com/dask/dask-docker.
-This repository also includes a docker-compose configuration.
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/hpc.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/hpc.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/hpc.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/hpc.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,269 +0,0 @@
-High Performance Computers
-==========================
-
-Relevant Machines
------------------
-
-This page includes instructions and guidelines when deploying Dask on high
-performance supercomputers commonly found in scientific and industry research
-labs.  These systems commonly have the following attributes:
-
-1.  Some mechanism to launch MPI applications or use job schedulers like
-    SLURM, SGE, TORQUE, LSF, DRMAA, PBS, or others
-2.  A shared network file system visible to all machines in the cluster
-3.  A high performance network interconnect, such as Infiniband
-4.  Little or no node-local storage
-
-
-Where to start
---------------
-
-Most of this page documents various ways and best practices to use Dask on an
-HPC cluster.  This is technical and aimed both at users with some experience
-deploying Dask and also system administrators.
-
-The preferred and simplest way to run Dask on HPC systems today both for new,
-experienced users or administrator is to use
-`dask-jobqueue <https://jobqueue.dask.org>`_.
-
-However, dask-jobqueue is slightly oriented toward interactive analysis usage,
-and it might be better to use tools like dask-mpi in some routine batch
-production workloads.
-
-
-Dask-jobqueue and Dask-drmaa
-----------------------------
-
-The following projects provide easy high-level access to Dask using resource
-managers that are commonly deployed on HPC systems:
-
-1.  `dask-jobqueue <https://jobqueue.dask.org>`_ for use with PBS,
-    SLURM, LSF, SGE and other resource managers
-2.  `dask-drmaa <https://github.com/dask/dask-drmaa>`_ for use with any DRMAA
-    compliant resource manager
-
-They provide interfaces that look like the following:
-
-.. code-block:: python
-
-   from dask_jobqueue import PBSCluster
-
-   cluster = PBSCluster(cores=36,
-                        memory="100GB",
-                        project='P48500028',
-                        queue='premium',
-                        interface='ib0',
-                        walltime='02:00:00')
-
-   cluster.scale(100)  # Start 100 workers in 100 jobs that match the description above
-
-   from dask.distributed import Client
-   client = Client(cluster)    # Connect to that cluster
-
-Dask-jobqueue provides a lot of possibilities like adaptive dynamic scaling
-of workers, we recommend reading the `dask-jobqueue documentation
-<https://jobqueue.dask.org>`_ first to get a basic system running and then
-returning to this documentation for fine-tuning if necessary.
-
-
-Using MPI
----------
-
-.. note:: This section may not be necessary if you use a tool like
-   dask-jobqueue.
-
-You can launch a Dask network using ``mpirun`` or ``mpiexec`` and the
-``dask-mpi`` command line executable.
-
-.. code-block:: bash
-
-   mpirun --np 4 dask-mpi --scheduler-file /home/$USER/scheduler.json
-
-.. code-block:: python
-
-   from dask.distributed import Client
-   client = Client(scheduler_file='/path/to/scheduler.json')
-
-This depends on the `mpi4py <https://mpi4py.readthedocs.io/>`_ library.  It only
-uses MPI to start the Dask cluster and not for inter-node communication. MPI
-implementations differ: the use of ``mpirun --np 4`` is specific to the
-``mpich`` or ``open-mpi`` MPI implementation installed through conda and linked
-to mpi4py.
-
-.. code-block:: bash
-
-   conda install mpi4py
-
-It is not necessary to use exactly this implementation, but you may want to
-verify that your ``mpi4py`` Python library is linked against the proper
-``mpirun/mpiexec`` executable and that the flags used (like ``--np 4``) are
-correct for your system.  The system administrator of your cluster should be
-very familiar with these concerns and able to help.
-
-In some setups, MPI processes are not allowed to fork other processes. In this
-case, we recommend using ``--no-nanny`` option in order to prevent dask from
-using an additional nanny process to manage workers.
-
-Run ``dask-mpi --help`` to see more options for the ``dask-mpi`` command.
-
-
-Using a Shared Network File System and a Job Scheduler
-------------------------------------------------------
-
-.. note:: This section is not necessary if you use a tool like dask-jobqueue.
-
-Some clusters benefit from a shared File System (NFS, GPFS, Lustre or alike),
-and can use this to communicate the scheduler location to the workers::
-
-   dask-scheduler --scheduler-file /path/to/scheduler.json  # writes address to file
-
-   dask-worker --scheduler-file /path/to/scheduler.json  # reads file for address
-   dask-worker --scheduler-file /path/to/scheduler.json  # reads file for address
-
-.. code-block:: python
-
-   >>> client = Client(scheduler_file='/path/to/scheduler.json')
-
-This can be particularly useful when deploying ``dask-scheduler`` and
-``dask-worker`` processes using a job scheduler like
-SGE/SLURM/Torque/etc.  Here is an example using SGE's ``qsub`` command::
-
-    # Start a dask-scheduler somewhere and write the connection information to a file
-    qsub -b y /path/to/dask-scheduler --scheduler-file /home/$USER/scheduler.json
-
-    # Start 100 dask-worker processes in an array job pointing to the same file
-    qsub -b y -t 1-100 /path/to/dask-worker --scheduler-file /home/$USER/scheduler.json
-
-Note, the ``--scheduler-file`` option is *only* valuable if your scheduler and
-workers share a network file system.
-
-
-High Performance Network
-------------------------
-
-Many HPC systems have both standard Ethernet networks as well as
-high-performance networks capable of increased bandwidth.  You can instruct
-Dask to use the high-performance network interface by using the ``--interface``
-keyword with the ``dask-worker``, ``dask-scheduler``, or ``dask-mpi`` commands or
-the ``interface=`` keyword with the dask-jobqueue ``Cluster`` objects:
-
-.. code-block:: bash
-
-   mpirun --np 4 dask-mpi --scheduler-file /home/$USER/scheduler.json --interface ib0
-
-In the code example above, we have assumed that your cluster has an Infiniband
-network interface called ``ib0``. You can check this by asking your system
-administrator or by inspecting the output of ``ifconfig``
-
-.. code-block:: bash
-
-	$ ifconfig
-	lo          Link encap:Local Loopback                       # Localhost
-				inet addr:127.0.0.1  Mask:255.0.0.0
-				inet6 addr: ::1/128 Scope:Host
-	eth0        Link encap:Ethernet  HWaddr XX:XX:XX:XX:XX:XX   # Ethernet
-				inet addr:192.168.0.101
-				...
-	ib0         Link encap:Infiniband                           # Fast InfiniBand
-				inet addr:172.42.0.101
-
-https://stackoverflow.com/questions/43881157/how-do-i-use-an-infiniband-network-with-dask
-
-
-Local Storage
--------------
-
-Users often exceed memory limits available to a specific Dask deployment.  In
-normal operation, Dask spills excess data to disk, often to the default
-temporary directory.
-
-However, in HPC systems this default temporary directory may point to an
-network file system (NFS) mount which can cause problems as Dask tries to read
-and write many small files.  *Beware, reading and writing many tiny files from
-many distributed processes is a good way to shut down a national
-supercomputer*.
-
-If available, it's good practice to point Dask workers to local storage, or
-hard drives that are physically on each node.  Your IT administrators will be
-able to point you to these locations.  You can do this with the
-``--local-directory`` or ``local_directory=`` keyword in the ``dask-worker``
-command::
-
-   dask-mpi ... --local-directory /path/to/local/storage
-
-or any of the other Dask Setup utilities, or by specifying the
-following :doc:`configuration value <../../configuration>`:
-
-.. code-block:: yaml
-
-   temporary-directory: /path/to/local/storage
-
-However, not all HPC systems have local storage.  If this is the case then you
-may want to turn off Dask's ability to spill to disk altogether.  See `this
-page <https://distributed.dask.org/en/latest/worker.html#memory-management>`_
-for more information on Dask's memory policies.  Consider changing the
-following values in your ``~/.config/dask/distributed.yaml`` file to disable
-spilling data to disk:
-
-.. code-block:: yaml
-
-   distributed:
-     worker:
-       memory:
-         target: false  # don't spill to disk
-         spill: false  # don't spill to disk
-         pause: 0.80  # pause execution at 80% memory use
-         terminate: 0.95  # restart the worker at 95% use
-
-This stops Dask workers from spilling to disk, and instead relies entirely on
-mechanisms to stop them from processing when they reach memory limits.
-
-As a reminder, you can set the memory limit for a worker using the
-``--memory-limit`` keyword::
-
-   dask-mpi ... --memory-limit 10GB
-
-
-Launch Many Small Jobs
-----------------------
-
-.. note:: This section is not necessary if you use a tool like dask-jobqueue.
-
-HPC job schedulers are optimized for large monolithic jobs with many nodes that
-all need to run as a group at the same time.  Dask jobs can be quite a bit more
-flexible: workers can come and go without strongly affecting the job.  If we
-split our job into many smaller jobs, we can often get through the job
-scheduling queue much more quickly than a typical job.  This is particularly
-valuable when we want to get started right away and interact with a Jupyter
-notebook session rather than waiting for hours for a suitable allocation block
-to become free.
-
-So, to get a large cluster quickly, we recommend allocating a dask-scheduler
-process on one node with a modest wall time (the intended time of your session)
-and then allocating many small single-node dask-worker jobs with shorter wall
-times (perhaps 30 minutes) that can easily squeeze into extra space in the job
-scheduler.  As you need more computation, you can add more of these single-node
-jobs or let them expire.
-
-
-Use Dask to co-launch a Jupyter server
---------------------------------------
-
-Dask can help you by launching other services alongside it.  For example, you
-can run a Jupyter notebook server on the machine running the ``dask-scheduler``
-process with the following commands
-
-.. code-block:: python
-
-   from dask.distributed import Client
-   client = Client(scheduler_file='scheduler.json')
-
-   import socket
-   host = client.run_on_scheduler(socket.gethostname)
-
-   def start_jlab(dask_scheduler):
-       import subprocess
-       proc = subprocess.Popen(['/path/to/jupyter', 'lab', '--ip', host, '--no-browser'])
-       dask_scheduler.jlab_proc = proc
-
-   client.run_on_scheduler(start_jlab)
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-helm.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-helm.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-helm.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-helm.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,408 +0,0 @@
-Kubernetes and Helm
-===================
-
-It is easy to launch a Dask cluster and a Jupyter_ notebook server on cloud
-resources using Kubernetes_ and Helm_.
-
-.. _Kubernetes: https://kubernetes.io/
-.. _Helm: https://helm.sh/
-.. _Jupyter: https://jupyter.org/
-
-This is particularly useful when you want to deploy a fresh Python environment
-on Cloud services like Amazon Web Services, Google Compute Engine, or
-Microsoft Azure.
-
-If you already have Python environments running in a pre-existing Kubernetes
-cluster, then you may prefer the :doc:`Kubernetes native<kubernetes-native>`
-documentation, which is a bit lighter weight.
-
-Launch Kubernetes Cluster
--------------------------
-
-This document assumes that you have a Kubernetes cluster and Helm installed.
-
-If this is not the case, then you might consider setting up a Kubernetes cluster
-on one of the common cloud providers like Google, Amazon, or
-Microsoft.  We recommend the first part of the documentation in the guide
-`Zero to JupyterHub <https://zero-to-jupyterhub.readthedocs.io/en/latest/>`_
-that focuses on Kubernetes and Helm (you do not need to follow all of these
-instructions). In particular, you don't need to install JupyterHub.
-
-- `Creating a Kubernetes Cluster <https://zero-to-jupyterhub.readthedocs.io/en/latest/create-k8s-cluster.html>`_
-- `Setting up Helm <https://zero-to-jupyterhub.readthedocs.io/en/latest/setup-helm.html>`_
-
-Alternatively, you may want to experiment with Kubernetes locally using
-`Minikube <https://kubernetes.io/docs/getting-started-guides/minikube/>`_.
-
-Which Chart is Right for You?
------------------------------
-
-Dask maintains a Helm chart repository containing various charts for the Dask community
-https://helm.dask.org/ .
-You will need to add this to your known channels and update your local charts::
-
-   helm repo add dask https://helm.dask.org/
-   helm repo update
-
-We provides two Helm charts. The right one to choose depends on whether you're
-deploying Dask for a single user or for many users.
-
-
-================  =====================================================================
-Helm Chart        Use Case
-================  =====================================================================
-``dask/dask``     Single-user deployment with one notebook server and one Dask Cluster.
-``dask/daskhub``  Multi-user deployment with JupyterHub and Dask Gateway.
-================  =====================================================================
-
-See :ref:`kubernetes-helm.single` or :ref:`kubernetes-helm.multi` for detailed
-instructions on deploying either of these.
-As you might suspect, deploying ``dask/daskhub`` is a bit more complicated since
-there are more components. If you're just deploying for a single user we'd recommend
-using ``dask/dask``.
-
-.. _kubernetes-helm.single:
-
-Helm Install Dask for a Single User
------------------------------------
-
-Once your Kubernetes cluster is ready, you can deploy dask using the Dask Helm_ chart::
-
-   helm install my-dask dask/dask
-
-This deploys a ``dask-scheduler``, several ``dask-worker`` processes, and
-also an optional Jupyter server.
-
-
-Verify Deployment
-^^^^^^^^^^^^^^^^^
-
-This might take a minute to deploy.  You can check its status with
-``kubectl``::
-
-   kubectl get pods
-   kubectl get services
-
-   $ kubectl get pods
-   NAME                                  READY     STATUS              RESTARTS    AGE
-   bald-eel-jupyter-924045334-twtxd      0/1       ContainerCreating   0            1m
-   bald-eel-scheduler-3074430035-cn1dt   1/1       Running             0            1m
-   bald-eel-worker-3032746726-202jt      1/1       Running             0            1m
-   bald-eel-worker-3032746726-b8nqq      1/1       Running             0            1m
-   bald-eel-worker-3032746726-d0chx      0/1       ContainerCreating   0            1m
-
-   $ kubectl get services
-   NAME                 TYPE           CLUSTER-IP      EXTERNAL-IP      PORT(S)                      AGE
-   bald-eel-jupyter     LoadBalancer   10.11.247.201   35.226.183.149   80:30173/TCP                  2m
-   bald-eel-scheduler   LoadBalancer   10.11.245.241   35.202.201.129   8786:31166/TCP,80:31626/TCP   2m
-   kubernetes           ClusterIP      10.11.240.1     <none>           443/TCP
-   48m
-
-You can use the addresses under ``EXTERNAL-IP`` to connect to your now-running
-Jupyter and Dask systems.
-
-Notice the name ``bald-eel``.  This is the name that Helm has given to your
-particular deployment of Dask.  You could, for example, have multiple
-Dask-and-Jupyter clusters running at once, and each would be given a different
-name.  Note that you will need to use this name to refer to your deployment in the future.
-Additionally, you can list all active helm deployments with::
-
-   helm list
-
-   NAME            REVISION        UPDATED                         STATUS      CHART           NAMESPACE
-   bald-eel        1               Wed Dec  6 11:19:54 2017        DEPLOYED    dask-0.1.0      default
-
-
-Connect to Dask and Jupyter
-^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-When we ran ``kubectl get services``, we saw some externally visible IPs:
-
-.. code-block:: bash
-
-   mrocklin@pangeo-181919:~$ kubectl get services
-   NAME                 TYPE           CLUSTER-IP      EXTERNAL-IP      PORT(S)                       AGE
-   bald-eel-jupyter     LoadBalancer   10.11.247.201   35.226.183.149   80:30173/TCP                  2m
-   bald-eel-scheduler   LoadBalancer   10.11.245.241   35.202.201.129   8786:31166/TCP,80:31626/TCP   2m
-   kubernetes           ClusterIP      10.11.240.1     <none>           443/TCP                       48m
-
-We can navigate to these services from any web browser. Here, one is the Dask diagnostic
-dashboard, and the other is the Jupyter server.  You can log into the Jupyter
-notebook server with the password, ``dask``.
-
-You can create a notebook and create a Dask client from there.  The
-``DASK_SCHEDULER_ADDRESS`` environment variable has been populated with the
-address of the Dask scheduler.  This is available in Python from the ``dask.config`` object.
-
-.. code-block:: python
-
-   >>> import dask
-   >>> dask.config.get('scheduler_address')
-   'bald-eel-scheduler:8786'
-
-Although you don't need to use this address, the Dask client will find this
-variable automatically.
-
-.. code-block:: python
-
-   from dask.distributed import Client, config
-   client = Client()
-
-
-Configure Environment
-^^^^^^^^^^^^^^^^^^^^^
-
-By default, the Helm deployment launches three workers using one core each and
-a standard conda environment. We can customize this environment by creating a
-small yaml file that implements a subset of the values in the
-`dask helm chart values.yaml file <https://github.com/dask/helm-chart/blob/main/dask/values.yaml>`_.
-
-For example, we can increase the number of workers, and include extra conda and
-pip packages to install on the both the workers and Jupyter server (these two
-environments should be matched).
-
-.. code-block:: yaml
-
-   # config.yaml
-
-   worker:
-     replicas: 8
-     resources:
-       limits:
-         cpu: 2
-         memory: 7.5G
-       requests:
-         cpu: 2
-         memory: 7.5G
-     env:
-       - name: EXTRA_CONDA_PACKAGES
-         value: numba xarray -c conda-forge
-       - name: EXTRA_PIP_PACKAGES
-         value: s3fs dask-ml --upgrade
-
-   # We want to keep the same packages on the worker and jupyter environments
-   jupyter:
-     enabled: true
-     env:
-       - name: EXTRA_CONDA_PACKAGES
-         value: numba xarray matplotlib -c conda-forge
-       - name: EXTRA_PIP_PACKAGES
-         value: s3fs dask-ml --upgrade
-
-This config file overrides the configuration for the number and size of workers and the
-conda and pip packages installed on the worker and Jupyter containers.  In
-general, we will want to make sure that these two software environments match.
-
-Update your deployment to use this configuration file.  Note that *you will not
-use helm install* for this stage: that would create a *new* deployment on the
-same Kubernetes cluster.  Instead, you will upgrade your existing deployment by
-using the current name::
-
-    helm upgrade bald-eel dask/dask -f config.yaml
-
-This will update those containers that need to be updated.  It may take a minute or so.
-
-As a reminder, you can list the names of deployments you have using ``helm
-list``
-
-
-Check status and logs
-^^^^^^^^^^^^^^^^^^^^^
-
-For standard issues, you should be able to see the worker status and logs using the
-Dask dashboard (in particular, you can see the worker links from the ``info/`` page).
-However, if your workers aren't starting, you can check the status of pods and
-their logs with the following commands:
-
-.. code-block:: bash
-
-   kubectl get pods
-   kubectl logs <PODNAME>
-
-.. code-block:: bash
-
-   mrocklin@pangeo-181919:~$ kubectl get pods
-   NAME                                  READY     STATUS    RESTARTS   AGE
-   bald-eel-jupyter-3805078281-n1qk2     1/1       Running   0          18m
-   bald-eel-scheduler-3074430035-cn1dt   1/1       Running   0          58m
-   bald-eel-worker-1931881914-1q09p      1/1       Running   0          18m
-   bald-eel-worker-1931881914-856mm      1/1       Running   0          18m
-   bald-eel-worker-1931881914-9lgzb      1/1       Running   0          18m
-   bald-eel-worker-1931881914-bdn2c      1/1       Running   0          16m
-   bald-eel-worker-1931881914-jq70m      1/1       Running   0          17m
-   bald-eel-worker-1931881914-qsgj7      1/1       Running   0          18m
-   bald-eel-worker-1931881914-s2phd      1/1       Running   0          17m
-   bald-eel-worker-1931881914-srmmg      1/1       Running   0          17m
-
-   mrocklin@pangeo-181919:~$ kubectl logs bald-eel-worker-1931881914-856mm
-   EXTRA_CONDA_PACKAGES environment variable found.  Installing.
-   Fetching package metadata ...........
-   Solving package specifications: .
-   Package plan for installation in environment /opt/conda/envs/dask:
-   The following NEW packages will be INSTALLED:
-       fasteners: 0.14.1-py36_2 conda-forge
-       monotonic: 1.3-py36_0    conda-forge
-       zarr:      2.1.4-py36_0  conda-forge
-   Proceed ([y]/n)?
-   monotonic-1.3- 100% |###############################| Time: 0:00:00  11.16 MB/s
-   fasteners-0.14 100% |###############################| Time: 0:00:00 576.56 kB/s
-   ...
-
-
-Delete a Helm deployment
-^^^^^^^^^^^^^^^^^^^^^^^^
-
-You can always delete a helm deployment using its name::
-
-   helm delete bald-eel --purge
-
-Note that this does not destroy any clusters that you may have allocated on a
-Cloud service (you will need to delete those explicitly).
-
-
-Avoid the Jupyter Server
-^^^^^^^^^^^^^^^^^^^^^^^^
-
-Sometimes you do not need to run a Jupyter server alongside your Dask cluster.
-
-.. code-block:: yaml
-
-   jupyter:
-     enabled: false
-
-.. _kubernetes-helm.multi:
-
-Helm Install Dask for Mulitple Users
-------------------------------------
-
-The ``dask/daskhub`` Helm Chart deploys JupyterHub_, `Dask Gateway`_, and configures
-the two to work well together. In particular, Dask Gateway is registered as
-a JupyterHub service so that Dask Gateway can re-use JupyterHub's authentication,
-and the JupyterHub environment is configured to connect to the Dask Gateway
-without any arguments.
-
-.. note::
-
-   The ``dask/daskhub`` helm chart came out of the `Pangeo`_ project, a community
-   platform for big data geoscience.
-
-.. _Pangeo: http://pangeo.io/
-.. _Dask Gateway: https://gateway.dask.org/
-.. _JupyterHub: https://jupyterhub.readthedocs.io/en/stable/
-
-The ``dask/daskhub`` helm chart uses the JupyterHub and Dask-Gateway helm charts.
-You'll want to consult the `JupyterHub helm documentation <https://zero-to-jupyterhub.readthedocs.io/en/latest/setup-jupyterhub/setup-jupyterhub.html>`_ and
-and `Dask Gateway helm documentation <https://gateway.dask.org/install-kube.html>`_ for further customization. The default values
-are at https://github.com/dask/helm-chart/blob/main/daskhub/values.yaml.
-
-Verify that you've set up a Kubernetes cluster and added Dask's helm charts:
-
-.. code-block:: console
-
-   $ helm repo add dask https://helm.dask.org/
-   $ helm repo update
-
-JupyterHub and Dask Gateway require a few secret tokens. We'll generate them
-on the command line and insert the tokens in a ``secrets.yaml`` file that will
-be passed to Helm.
-
-Run the following command, and copy the output. This is our `token-1`.
-
-.. code-block:: console
-
-   $ openssl rand -hex 32  # generate token-1
-
-Run command again and copy the output again. This is our `token-2`.
-
-.. code-block:: console
-
-   $ openssl rand -hex 32  # generate token-2
-
-Now substitute those two values for ``<token-1>`` and ``<token-2>`` below.
-Note that ``<token-2>`` is used twice, once for ``jupyterhub.hub.services.dask-gateway.apiToken``, and a second time for ``dask-gateway.gateway.auth.jupyterhub.apiToken``.
-
-.. code-block:: yaml
-
-   # file: secrets.yaml
-   jupyterhub:
-     proxy:
-       secretToken: "<token-1>"
-     hub:
-       services:
-         dask-gateway:
-           apiToken: "<token-2>"
-
-   dask-gateway:
-     gateway:
-       auth:
-         jupyterhub:
-           apiToken: "<token-2>"
-
-Now we're ready to install DaskHub
-
-.. code-block:: console
-
-   $ helm upgrade --wait --install --render-subchart-notes \
-       dhub dask/daskhub \
-       --values=secrets.yaml
-
-
-The output explains how to find the IPs for your JupyterHub depoyment.
-
-.. code-block:: console
-
-   $ kubectl get service proxy-public
-   NAME           TYPE           CLUSTER-IP      EXTERNAL-IP      PORT(S)                      AGE
-   proxy-public   LoadBalancer   10.43.249.239   35.202.158.223   443:31587/TCP,80:30500/TCP   2m40s
-
-
-Creating a Dask Cluster
-^^^^^^^^^^^^^^^^^^^^^^^
-
-To create a Dask cluster on this deployment, users need to connect to the Dask Gateway
-
-.. code-block:: python
-
-   >>> from dask_gateway import GatewayCluster
-   >>> cluster = GatewayCluster()
-   >>> client = cluster.get_client()
-   >>> cluster
-
-Depending on the configuration, users may need to ``cluster.scale(n)`` to
-get workers. See https://gateway.dask.org/ for more on Dask Gateway.
-
-Matching the User Environment
-^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-Dask Clients will be running the JupyterHub's singleuser environment. To ensure
-that the same environment is used for the scheduler and workers, you can provide
-it as a Gateway option and configure the ``singleuser`` environment to default
-to the value set by JupyterHub.
-
-.. code-block:: yaml
-
-   # config.yaml
-   jupyterhub:
-     singleuser:
-       extraEnv:
-         DASK_GATEWAY__CLUSTER__OPTIONS__IMAGE: '{JUPYTER_IMAGE_SPEC}'
-
-   dask-gateway:
-     gateway:
-       extraConfig:
-         optionHandler: |
-           from dask_gateway_server.options import Options, Integer, Float, String
-           def option_handler(options):
-               if ":" not in options.image:
-                   raise ValueError("When specifying an image you must also provide a tag")
-               return {
-                   "image": options.image,
-               }
-           c.Backend.cluster_options = Options(
-               String("image", default="pangeo/base-notebook:2020.07.28", label="Image"),
-               handler=option_handler,
-           )
-
-The user environment will need to include ``dask-gateway``. Any packages installed
-manually after the ``singleuser`` pod started will not be included in the worker
-environment.
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-native.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-native.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-native.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes-native.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,6 +0,0 @@
-Kubernetes Native
-=================
-
-See external documentation on Dask-Kubernetes_ for more information.
-
-.. _Dask-Kubernetes: https://kubernetes.dask.org
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/kubernetes.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,84 +0,0 @@
-Kubernetes
-==========
-
-.. toctree::
-   :maxdepth: 1
-   :hidden:
-
-   Helm <kubernetes-helm.rst>
-   Native <kubernetes-native.rst>
-
-Kubernetes_ is a popular system for deploying distributed applications on clusters,
-particularly in the cloud.  You can use Kubernetes to launch Dask workers in the
-following two ways:
-
-1.  **Helm**:
-
-    You can deploy Dask and (optionally) Jupyter or JupyterHub on Kubernetes
-    easily using Helm_
-
-    .. code-block:: bash
-
-       helm repo add dask https://helm.dask.org/    # add the Dask Helm chart repository
-       helm repo update                             # get latest Helm charts
-       # For single-user deployments, use dask/dask
-       helm install my-dask dask/dask               # deploy standard Dask chart
-       # For multi-user deployments, use dask/daskhub
-       helm install my-dask dask/daskhub            # deploy JupyterHub & Dask
-
-    This is a good choice if you want to do the following:
-
-    1.  Run a managed Dask cluster for a long period of time
-    2.  Also deploy a Jupyter / JupyterHub server from which to run code
-    3.  Share the same Dask cluster between many automated services
-    4.  Try out Dask for the first time on a cloud-based system
-        like Amazon, Google, or Microsoft Azure where you already have
-        a Kubernetes cluster. If you don't already have Kubernetes deployed,
-        see our :doc:`Cloud documentation <cloud>`.
-
-    You can also use the ``HelmCluster`` cluster manager from dask-kubernetes to manage your
-    Helm Dask cluster from within your Python session.
-
-    .. code-block:: python
-
-       from dask_kubernetes import HelmCluster
-
-       cluster = HelmCluster(release_name="myrelease")
-       cluster.scale(10)
-
-    .. note::
-
-      For more information, see :doc:`Dask and Helm documentation <kubernetes-helm>`.
-
-2.  **Native**:
-    You can quickly deploy Dask workers on Kubernetes
-    from within a Python script or interactive session using Dask-Kubernetes_
-
-    .. code-block:: python
-
-       from dask_kubernetes import KubeCluster
-       cluster = KubeCluster.from_yaml('worker-template.yaml')
-       cluster.scale(20)  # add 20 workers
-       cluster.adapt()    # or create and destroy workers dynamically based on workload
-
-       from dask.distributed import Client
-       client = Client(cluster)
-
-    This is a good choice if you want to do the following:
-
-    1.  Dynamically create a personal and ephemeral deployment for interactive use
-    2.  Allow many individuals the ability to launch their own custom dask deployments,
-        rather than depend on a centralized system
-    3.  Quickly adapt Dask cluster size to the current workload
-
-    .. note::
-
-      For more information, see Dask-Kubernetes_ documentation.
-
-You may also want to see the documentation on using
-:doc:`Dask with Docker containers <docker>`
-to help you manage your software environments on Kubernetes.
-
-.. _Kubernetes: https://kubernetes.io/
-.. _Dask-Kubernetes: https://kubernetes.dask.org/
-.. _Helm: https://helm.sh/
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/python-advanced.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/python-advanced.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/python-advanced.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/python-advanced.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,210 +0,0 @@
-Python API (advanced)
-=====================
-
-.. currentmodule:: distributed
-
-In some rare cases, experts may want to create ``Scheduler``, ``Worker``, and
-``Nanny``  objects explicitly in Python.  This is often necessary when making
-tools to automatically deploy Dask in custom settings.
-
-It is more common to create a :doc:`Local cluster with Client() on a single
-machine <single-distributed>` or use the :doc:`Command Line Interface (CLI) <cli>`.
-New readers are recommended to start there.
-
-If you do want to start Scheduler and Worker objects yourself you should be a
-little familiar with ``async``/``await`` style Python syntax.  These objects
-are awaitable and are commonly used within ``async with`` context managers.
-Here are a few examples to show a few ways to start and finish things.
-
-Full Example
-------------
-
-.. autosummary::
-   Scheduler
-   Worker
-   Client
-
-We first start with a comprehensive example of setting up a Scheduler, two Workers,
-and one Client in the same event loop, running a simple computation, and then
-cleaning everything up.
-
-.. code-block:: python
-
-   import asyncio
-   from dask.distributed import Scheduler, Worker, Client
-
-   async def f():
-       async with Scheduler() as s:
-           async with Worker(s.address) as w1, Worker(s.address) as w2:
-               async with Client(s.address, asynchronous=True) as client:
-                   future = client.submit(lambda x: x + 1, 10)
-                   result = await future
-                   print(result)
-
-   asyncio.get_event_loop().run_until_complete(f())
-
-Now we look at simpler examples that build up to this case.
-
-Scheduler
----------
-
-.. autosummary::
-   Scheduler
-
-We create scheduler by creating a ``Scheduler()`` object, and then ``await``
-that object to wait for it to start up.  We can then wait on the ``.finished``
-method to wait until it closes.  In the meantime the scheduler will be active
-managing the cluster..
-
-.. code-block:: python
-
-   import asyncio
-   from dask.distributed import Scheduler, Worker
-
-   async def f():
-       s = Scheduler()        # scheduler created, but not yet running
-       s = await s            # the scheduler is running
-       await s.finished()     # wait until the scheduler closes
-
-   asyncio.get_event_loop().run_until_complete(f())
-
-This program will run forever, or until some external process connects to the
-scheduler and tells it to stop.  If you want to close things yourself you can
-close any ``Scheduler``, ``Worker``, ``Nanny``, or ``Client`` class by awaiting
-the ``.close`` method:
-
-.. code-block:: python
-
-   await s.close()
-
-
-Worker
-------
-
-.. autosummary::
-   Worker
-
-The worker follows the same API.
-The only difference is that the worker needs to know the address of the
-scheduler.
-
-.. code-block:: python
-
-   import asyncio
-   from dask.distributed import Scheduler, Worker
-
-   async def f(scheduler_address):
-       w = await Worker(scheduler_address)
-       await w.finished()
-
-   asyncio.get_event_loop().run_until_complete(f("tcp://127.0.0.1:8786"))
-
-
-Start many in one event loop
-----------------------------
-
-.. autosummary::
-   Scheduler
-   Worker
-
-We can run as many of these objects as we like in the same event loop.
-
-.. code-block:: python
-
-   import asyncio
-   from dask.distributed import Scheduler, Worker
-
-   async def f():
-       s = await Scheduler()
-       w = await Worker(s.address)
-       await w.finished()
-       await s.finished()
-
-   asyncio.get_event_loop().run_until_complete(f())
-
-
-Use Context Managers
---------------------
-
-We can also use ``async with`` context managers to make sure that we clean up
-properly.  Here is the same example as from above:
-
-.. code-block:: python
-
-   import asyncio
-   from dask.distributed import Scheduler, Worker
-
-   async def f():
-       async with Scheduler() as s:
-           async with Worker(s.address) as w:
-               await w.finished()
-               await s.finished()
-
-   asyncio.get_event_loop().run_until_complete(f())
-
-Alternatively, in the example below we also include a ``Client``, run a small
-computation, and then allow things to clean up after that computation..
-
-.. code-block:: python
-
-   import asyncio
-   from dask.distributed import Scheduler, Worker, Client
-
-   async def f():
-       async with Scheduler() as s:
-           async with Worker(s.address) as w1, Worker(s.address) as w2:
-               async with Client(s.address, asynchronous=True) as client:
-                   future = client.submit(lambda x: x + 1, 10)
-                   result = await future
-                   print(result)
-
-   asyncio.get_event_loop().run_until_complete(f())
-
-This is equivalent to creating and ``awaiting`` each server, and then calling
-``.close`` on each as we leave the context.
-In this example we don't wait on ``s.finished()``, so this will terminate
-relatively quickly.  You could have called ``await s.finished()`` though if you
-wanted this to run forever.
-
-Nanny
------
-
-.. autosummary::
-   Nanny
-
-Alternatively, we can replace ``Worker`` with ``Nanny`` if we want your workers
-to be managed in a separate process.  The ``Nanny`` constructor follows the
-same API. This allows workers to restart themselves in case of failure. Also,
-it provides some additional monitoring, and is useful when coordinating many
-workers that should live in different processes in order to avoid the GIL_.
-
-.. code-block:: python
-
-   # w = await Worker(s.address)
-   w = await Nanny(s.address)
-
-.. _GIL: https://docs.python.org/3/glossary.html#term-gil
-
-
-API
----
-
-These classes have a variety of keyword arguments that you can use to control
-their behavior.  See the API documentation below for more information.
-
-Scheduler
-~~~~~~~~~
-.. autoclass:: Scheduler
-   :members:
-
-Worker
-~~~~~~
-
-.. autoclass:: Worker
-   :members:
-
-Nanny
-~~~~~
-
-.. autoclass:: Nanny
-   :members:
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/single-distributed.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/single-distributed.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/single-distributed.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/single-distributed.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,74 +0,0 @@
-:orphan:
-
-Single Machine: dask.distributed
-================================
-
-The ``dask.distributed`` scheduler works well on a single machine.  It is sometimes
-preferred over the default scheduler for the following reasons:
-
-1.  It provides access to asynchronous API, notably :doc:`Futures <../../futures>`
-2.  It provides a diagnostic dashboard that can provide valuable insight on
-    performance and progress
-3.  It handles data locality with more sophistication, and so can be more
-    efficient than the multiprocessing scheduler on workloads that require
-    multiple processes
-
-You can create a ``dask.distributed`` scheduler by importing and creating a
-``Client`` with no arguments.  This overrides whatever default was previously
-set.
-
-.. code-block:: python
-
-   from dask.distributed import Client
-   client = Client()
-
-You can navigate to ``http://localhost:8787/status`` to see the diagnostic
-dashboard if you have Bokeh installed.
-
-Client
-------
-
-You can trivially set up a local cluster on your machine by instantiating a Dask
-Client with no arguments
-
-.. code-block:: python
-
-   from dask.distributed import Client
-   client = Client()
-
-This sets up a scheduler in your local process along with a number of workers and
-threads per worker related to the number of cores in your machine.
-
-If you want to run workers in your same process, you can pass the
-``processes=False`` keyword argument.
-
-.. code-block:: python
-
-   client = Client(processes=False)
-
-This is sometimes preferable if you want to avoid inter-worker communication
-and your computations release the GIL.  This is common when primarily using
-NumPy or Dask Array.
-
-
-LocalCluster
-------------
-
-The ``Client()`` call described above is shorthand for creating a LocalCluster
-and then passing that to your client.
-
-.. code-block:: python
-
-   from dask.distributed import Client, LocalCluster
-   cluster = LocalCluster()
-   client = Client(cluster)
-
-This is equivalent, but somewhat more explicit.  You may want to look at the
-keyword arguments available on ``LocalCluster`` to understand the options available
-to you on handling the mixture of threads and processes, like specifying explicit
-ports, and so on.
-
-.. currentmodule:: distributed.deploy.local
-
-.. autoclass:: LocalCluster
-   :members:
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/single-machine.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/single-machine.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/single-machine.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/single-machine.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,75 +0,0 @@
-:orphan:
-
-Single-Machine Scheduler
-========================
-
-The default Dask scheduler provides parallelism on a single machine by using
-either threads or processes.  It is the default choice used by Dask because it
-requires no setup. You don't need to make any choices or set anything up to
-use this scheduler. However, you do have a choice between threads and processes:
-
-1.  **Threads**: Use multiple threads in the same process.  This option is good
-    for numeric code that releases the GIL_ (like NumPy, Pandas, Scikit-Learn,
-    Numba, ...) because data is free to share.  This is the default scheduler for
-    ``dask.array``, ``dask.dataframe``, and ``dask.delayed``
-
-2.  **Processes**: Send data to separate processes for processing.  This option
-    is good when operating on pure Python objects like strings or JSON-like
-    dictionary data that holds onto the GIL_, but not very good when operating
-    on numeric data like Pandas DataFrames or NumPy arrays.  Using processes
-    avoids GIL issues, but can also result in a lot of inter-process
-    communication, which can be slow.  This is the default scheduler for
-    ``dask.bag``, and it is sometimes useful with ``dask.dataframe``
-
-    Note that the ``dask.distributed`` scheduler is often a better choice when
-    working with GIL-bound code.  See :doc:`dask.distributed on a single
-    machine <single-distributed>`
-
-3.  **Single-threaded**: Execute computations in a single thread.  This option
-    provides no parallelism, but is useful when debugging or profiling.
-    Turning your parallel execution into a sequential one can be a convenient
-    option in many situations where you want to better understand what is going
-    on
-
-.. _GIL: https://docs.python.org/3/glossary.html#term-gil
-
-
-Selecting Threads, Processes, or Single Threaded
-------------------------------------------------
-
-You can select between these options by specifying one of the following three
-values to the ``scheduler=`` keyword:
-
--  ``"threads"``: Uses a ThreadPoolExecutor in the local process
--  ``"processes"``: Uses a ProcessPoolExecutor to spread work between processes
--  ``"single-threaded"``: Uses a for-loop in the current thread
-
-You can specify these options in any of the following ways:
-
--   When calling ``.compute()``
-
-    .. code-block:: python
-
-       x.compute(scheduler='threads')
-
--   With a context manager
-
-    .. code-block:: python
-
-       with dask.config.set(scheduler='threads'):
-           x.compute()
-           y.compute()
-
--   As a global setting
-
-    .. code-block:: python
-
-       dask.config.set(scheduler='threads')
-
-
-Use the Distributed Scheduler
------------------------------
-
-Dask's newer distributed scheduler also works well on a single machine and
-offers more features and diagnostics.  See :doc:`this page
-<single-distributed>` for more information.
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/ssh.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/ssh.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask/ssh.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask/ssh.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,58 +0,0 @@
-SSH
-===
-
-It is easy to set up Dask on informally managed networks of machines using SSH.
-This can be done manually using SSH and the
-Dask :doc:`command line interface <cli>`,
-or automatically using either the ``SSHCluster`` Python command or the
-``dask-ssh`` command line tool. This document describes both of these options.
-
-.. note::
-   Before instaniating a ``SSHCluster`` it is recommended to configure keyless SSH
-   for your local machine and other machines. For example, on a Mac to SSH into
-   localhost (local machine) you need to ensure the Remote Login option is set in
-   System Preferences -> Sharing. In addition, ``id_rsa.pub`` should be in
-   ``authorized_keys`` for keyless login.
-
-Python Interface
-----------------
-
-.. currentmodule:: distributed.deploy.ssh
-
-.. autofunction:: SSHCluster
-
-Command Line
-------------
-
-The convenience script ``dask-ssh`` opens several SSH connections to your
-target computers and initializes the network accordingly. You can
-give it a list of hostnames or IP addresses::
-
-   $ dask-ssh 192.168.0.1 192.168.0.2 192.168.0.3 192.168.0.4
-
-Or you can use normal UNIX grouping::
-
-   $ dask-ssh 192.168.0.{1,2,3,4}
-
-Or you can specify a hostfile that includes a list of hosts::
-
-   $ cat hostfile.txt
-   192.168.0.1
-   192.168.0.2
-   192.168.0.3
-   192.168.0.4
-
-   $ dask-ssh --hostfile hostfile.txt
-
-The ``dask-ssh`` utility depends on the ``paramiko``::
-
-    python -m pip install paramiko
-
-.. note::
-
-   The command line documentation here may differ depending on your installed
-   version. We recommend referring to the output of ``dask-ssh --help``.
-
-.. click:: distributed.cli.dask_ssh:main
-   :prog: dask-ssh
-   :show-nested:
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask-clusters.rst 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask-clusters.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/deploy-dask-clusters.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/deploy-dask-clusters.rst	1970-01-01 00:00:00.000000000 +0000
@@ -1,119 +0,0 @@
-Deploy Dask clusters
-====================
-
-This page describes various ways to set up Dask on different hardware, either
-locally on your own machine or on a distributed cluster.  If you are just
-getting started, then this page is unnecessary.  Dask does not require any setup
-if you only want to use it on a single computer.
-
-You can continue reading or watch the screencast below:
-
-.. raw:: html
-
-   <iframe width="560"
-           height="315"
-           src="https://www.youtube.com/embed/TQM9zIBzNBo"
-           style="margin: 0 auto 20px auto; display: block;"
-           frameborder="0"
-           allow="accelerometer; autoplay; encrypted-media; gyroscope; picture-in-picture"
-           allowfullscreen></iframe>
-
-Dask has two families of task schedulers:
-
-1.  **Single-machine scheduler**: This scheduler provides basic features on a
-    local process or thread pool.  This scheduler was made first and is the
-    default.  It is simple and cheap to use.  It can only be used on a single
-    machine and does not scale.
-2.  **Distributed scheduler**: This scheduler is more sophisticated. It offers
-    more features, but also requires a bit more effort to set up.  It can
-    run locally or distributed across a cluster.
-
-|
-
-.. figure:: ../images/dask-overview-distributed-callout.svg
-   :alt: Dask is composed of three parts. "Collections" create "Task Graphs" which are then sent to the "Scheduler" for execution. There are two types of schedulers that are described in more detail below.
-   
-   High level collections are used to generate task graphs which can be executed on a single machine or a cluster. Using the Distributed scheduler enables creation of a Dask cluster for multi-machine computation.
-
-|
-
-If you import Dask, set up a computation, and call ``compute``, then you
-will use the single-machine scheduler by default.  To use the ``dask.distributed``
-scheduler you must set up a ``Client``
-
-.. code-block:: python
-
-   import dask.dataframe as dd
-   df = dd.read_csv(...)
-   df.x.sum().compute()  # This uses the single-machine scheduler by default
-
-.. code-block:: python
-
-   from dask.distributed import Client
-   client = Client(...)  # Connect to distributed cluster and override default
-   df.x.sum().compute()  # This now runs on the distributed system
-
-Note that the newer ``dask.distributed`` scheduler is often preferable, even on
-single workstations.  It contains many diagnostics and features not found in
-the older single-machine scheduler.
-
-There are also a number of different *cluster managers* available, so you can use
-Dask distributed with a range of platforms. These *cluster managers* deploy a scheduler
-and the necessary workers as determined by communicating with the *resource manager*.
-`Dask Jobqueue <https://github.com/dask/dask-jobqueue>`_, for example, is a set of
-*cluster managers* for HPC users and works with job queueing systems
-(in this case, the *resource manager*) such as `PBS <https://en.wikipedia.org/wiki/Portable_Batch_System>`_,
-`Slurm <https://en.wikipedia.org/wiki/Slurm_Workload_Manager>`_,
-and `SGE <https://en.wikipedia.org/wiki/Oracle_Grid_Engine>`_.
-Those workers are then allocated physical hardware resources.
-
-.. figure:: ../images/dask-cluster-manager.svg
-   :scale: 50%
-
-   An overview of cluster management with Dask distributed.
-
-To summarize, you can use the default, single-machine scheduler to use Dask
-on your local machine. If you'd like use a cluster *or* simply take advantage
-of the :doc:`extensive diagnostics <../diagnostics-distributed>`,
-you can use Dask distributed. The following resources explain
-in more detail how to set up Dask on a variety of local and distributed hardware:
-
-- Single Machine:
-    - :doc:`Default Scheduler <deploy-dask/single-machine>`: The no-setup default.
-      Uses local threads or processes for larger-than-memory processing
-    - :doc:`dask.distributed <deploy-dask/single-distributed>`: The sophistication of
-      the newer system on a single machine.  This provides more advanced
-      features while still requiring almost no setup.
-- Distributed computing:
-    - `Beginner's Guide to Configuring a Dask distributed Cluster <https://blog.dask.org/2020/07/30/beginners-config>`_
-    - `Overview of cluster management options <https://blog.dask.org/2020/07/23/current-state-of-distributed-dask-clusters>`_
-    - :doc:`Manual Setup <deploy-dask/cli>`: The command line interface to set up
-      ``dask-scheduler`` and ``dask-worker`` processes.  Useful for IT or
-      anyone building a deployment solution.
-    - :doc:`SSH <deploy-dask/ssh>`: Use SSH to set up Dask across an un-managed
-      cluster.
-    - :doc:`High Performance Computers <deploy-dask/hpc>`: How to run Dask on
-      traditional HPC environments using tools like MPI, or job schedulers like
-      SLURM, SGE, TORQUE, LSF, and so on.
-    - :doc:`Kubernetes <deploy-dask/kubernetes>`: Deploy Dask with the
-      popular Kubernetes resource manager using either Helm or a native deployment.
-    - `YARN / Hadoop <https://yarn.dask.org/en/latest/>`_: Deploy
-      Dask on YARN clusters, such as are found in traditional Hadoop
-      installations.
-    - `Dask Gateway <https://gateway.dask.org/>`_ provides a secure,
-      multi-tenant server for managing Dask clusters and allows users to launch
-      and use Dask clusters in a shared cluster environment.
-    - :doc:`Python API (advanced) <deploy-dask/python-advanced>`: Create
-      ``Scheduler`` and ``Worker`` objects from Python as part of a distributed
-      Tornado TCP application.  This page is useful for those building custom
-      frameworks.
-    - :doc:`Docker <deploy-dask/docker>` images are available and may be useful
-      in some of the solutions above.
-    - :doc:`Cloud <deploy-dask/cloud>` for current recommendations on how to
-      deploy Dask and Jupyter on common cloud providers like Amazon, Google, or
-      Microsoft Azure.
-- Hosted / managed Dask clusters (listed in alphabetical order):
-    - `Coiled <https://coiled.io/>`_ handles the creation and management of
-      Dask clusters on cloud computing environments (AWS, Azure, and GCP).
-    - `Saturn Cloud <https://saturncloud.io/>`_ lets users create
-      Dask clusters in a hosted platform or within their own AWS accounts.
diff -pruN 2022.01.0+dfsg-1/docs/source/how-to/index.rst 2022.02.0+dfsg-1/docs/source/how-to/index.rst
--- 2022.01.0+dfsg-1/docs/source/how-to/index.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/how-to/index.rst	2022-02-11 16:21:12.000000000 +0000
@@ -14,11 +14,3 @@ using Dask. If you have an idea of a how
 
    *
    Use GPUs <../gpu.rst>
-   Deploy Dask via the CLI <deploy-dask/cli>
-   Deploy Dask via SSH <deploy-dask/ssh>
-   Deploy Dask on High Performance Computers <deploy-dask/hpc>
-   Deploy Dask on Kubernetes <deploy-dask/kubernetes>
-   Deploy Dask on YARN clusters  <https://yarn.dask.org/en/latest/>
-   Deploy Dask via the Python API (advanced) <deploy-dask/python-advanced>
-   Deploy Dask on Docker <deploy-dask/docker>
-   Deploy Dask on the Cloud (Amazon, Google, Microsoft Azure...) <deploy-dask/cloud>
Binary files 2022.01.0+dfsg-1/docs/source/images/transpose-hlg-hovertooltip.png and 2022.02.0+dfsg-1/docs/source/images/transpose-hlg-hovertooltip.png differ
Binary files 2022.01.0+dfsg-1/docs/source/images/transpose-hlg-html-repr.png and 2022.02.0+dfsg-1/docs/source/images/transpose-hlg-html-repr.png differ
diff -pruN 2022.01.0+dfsg-1/docs/source/index.rst 2022.02.0+dfsg-1/docs/source/index.rst
--- 2022.01.0+dfsg-1/docs/source/index.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/index.rst	2022-02-11 16:21:12.000000000 +0000
@@ -153,6 +153,7 @@ messy situations in everyday problems.
    futures.rst
    scheduling.rst
    graphs.rst
+   deploying.rst
    internals.rst
 
 .. toctree::
diff -pruN 2022.01.0+dfsg-1/docs/source/scheduling.rst 2022.02.0+dfsg-1/docs/source/scheduling.rst
--- 2022.01.0+dfsg-1/docs/source/scheduling.rst	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/scheduling.rst	2022-02-11 16:21:12.000000000 +0000
@@ -140,7 +140,7 @@ Dask Distributed (local)
    # or
    client = Client(processes=False)
 
-The Dask distributed scheduler can either be :doc:`setup on a cluster <how-to/deploy-dask-clusters>`
+The Dask distributed scheduler can either be :doc:`setup on a cluster <deploying>`
 or run locally on a personal machine.  Despite having the name "distributed",
 it is often pragmatic on local machines for a few reasons:
 
@@ -152,7 +152,7 @@ it is often pragmatic on local machines
     multiple processes
 
 You can read more about using the Dask distributed scheduler on a single machine in
-:doc:`these docs <how-to/deploy-dask/single-distributed>`.
+:doc:`these docs <deploying>`.
 
 
 Dask Distributed (Cluster)
@@ -160,7 +160,7 @@ Dask Distributed (Cluster)
 
 You can also run Dask on a distributed cluster.
 There are a variety of ways to set this up depending on your cluster.
-We recommend referring to :doc:`how to deploy Dask clusters <how-to/deploy-dask-clusters>` for more information.
+We recommend referring to :doc:`how to deploy Dask clusters <deploying>` for more information.
 
 .. _scheduling-configuration:
 
diff -pruN 2022.01.0+dfsg-1/docs/source/scripts/scheduling.py 2022.02.0+dfsg-1/docs/source/scripts/scheduling.py
--- 2022.01.0+dfsg-1/docs/source/scripts/scheduling.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/docs/source/scripts/scheduling.py	2022-02-11 16:21:12.000000000 +0000
@@ -55,12 +55,12 @@ import numpy as np
 
 x = np.logspace(0, 4, 10)
 trivial_results = dict()
-for get in [dask.get, threaded.get, local.get_sync, multiprocessing.get]:
+for get in (dask.get, threaded.get, local.get_sync, multiprocessing.get):
     y = list()
     for n in x:
         dsk, keys = trivial(int(n), 5)
         start = time()
-        get(dsk, keys)
+        get(dsk, keys)  # type: ignore
         end = time()
         y.append(end - start)
     trivial_results[get] = np.array(y)
@@ -99,7 +99,7 @@ for get in [threaded.get, local.get_sync
     for n in x:
         dsk, keys = crosstalk(1000, 5, int(n))
         start = time()
-        get(dsk, keys)
+        get(dsk, keys)  # type: ignore
         end = time()
         y.append(end - start)
     crosstalk_results[get] = np.array(y)
diff -pruN 2022.01.0+dfsg-1/.github/release.yml 2022.02.0+dfsg-1/.github/release.yml
--- 2022.01.0+dfsg-1/.github/release.yml	1970-01-01 00:00:00.000000000 +0000
+++ 2022.02.0+dfsg-1/.github/release.yml	2022-02-11 16:21:12.000000000 +0000
@@ -0,0 +1,26 @@
+# .github/release.yml
+
+changelog:
+  categories:
+    - title: New Features
+      labels:
+        - feature
+    - title: Enhancements
+      labels:
+        - enhancement
+    - title: Bug Fixes
+      labels:
+        - bug
+    - title: Deprecations
+      labels:
+        - deprecation
+    - title: Documentation
+      labels:
+        - documentation
+    - title: Maintenance
+      labels:
+        - tests
+        - hygiene
+    - title: Misc
+      labels:
+        - "*"
\ No newline at end of file
diff -pruN 2022.01.0+dfsg-1/.github/workflows/cancel.yml 2022.02.0+dfsg-1/.github/workflows/cancel.yml
--- 2022.01.0+dfsg-1/.github/workflows/cancel.yml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/.github/workflows/cancel.yml	1970-01-01 00:00:00.000000000 +0000
@@ -1,15 +0,0 @@
-name: Cancel
-
-on:
-  workflow_run:
-    workflows: ["Tests", "Additional", "Upstream"]
-    types:
-      - requested
-
-jobs:
-  cancel:
-    runs-on: ubuntu-latest
-    steps:
-    - uses: styfle/cancel-workflow-action@0.8.0
-      with:
-        workflow_id: ${{ github.event.workflow.id }}
\ No newline at end of file
diff -pruN 2022.01.0+dfsg-1/.github/workflows/tests.yml 2022.02.0+dfsg-1/.github/workflows/tests.yml
--- 2022.01.0+dfsg-1/.github/workflows/tests.yml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/.github/workflows/tests.yml	2022-02-11 16:21:12.000000000 +0000
@@ -2,6 +2,12 @@ name: Tests
 
 on: [push, pull_request]
 
+# When this workflow is queued, automatically cancel any previous running
+# or pending jobs from the same branch
+concurrency:
+  group: ${{ github.ref }}
+  cancel-in-progress: true
+
 jobs:
   test:
     runs-on: ${{ matrix.os }}
diff -pruN 2022.01.0+dfsg-1/LICENSE.txt 2022.02.0+dfsg-1/LICENSE.txt
--- 2022.01.0+dfsg-1/LICENSE.txt	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/LICENSE.txt	2022-02-11 16:21:12.000000000 +0000
@@ -1,6 +1,6 @@
 BSD 3-Clause License
 
-Copyright (c) 2014-2018, Anaconda, Inc. and contributors
+Copyright (c) 2014, Anaconda, Inc. and contributors
 All rights reserved.
 
 Redistribution and use in source and binary forms, with or without
diff -pruN 2022.01.0+dfsg-1/.pre-commit-config.yaml 2022.02.0+dfsg-1/.pre-commit-config.yaml
--- 2022.01.0+dfsg-1/.pre-commit-config.yaml	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/.pre-commit-config.yaml	2022-02-11 16:21:12.000000000 +0000
@@ -1,17 +1,17 @@
 repos:
   -   repo: https://github.com/pycqa/isort
-      rev: 5.9.3
+      rev: 5.10.1
       hooks:
       - id: isort
         language_version: python3
   - repo: https://github.com/asottile/pyupgrade
-    rev: v2.29.0
+    rev: v2.31.0
     hooks:
       - id: pyupgrade
         args:
           - --py37-plus
   -   repo: https://github.com/psf/black
-      rev: 21.9b0
+      rev: 22.1.0
       hooks:
       - id: black
         language_version: python3
diff -pruN 2022.01.0+dfsg-1/setup.py 2022.02.0+dfsg-1/setup.py
--- 2022.01.0+dfsg-1/setup.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/setup.py	2022-02-11 16:21:12.000000000 +0000
@@ -1,5 +1,7 @@
 #!/usr/bin/env python
 
+from __future__ import annotations
+
 import sys
 from os.path import exists
 
@@ -9,11 +11,11 @@ import versioneer
 
 # NOTE: These are tested in `continuous_integration/test_imports.sh` If
 # you modify these, make sure to change the corresponding line there.
-extras_require = {
+extras_require: dict[str, list[str]] = {
     "array": ["numpy >= 1.18"],
     "bag": [],  # keeping for backwards compatibility
     "dataframe": ["numpy >= 1.18", "pandas >= 1.0"],
-    "distributed": ["distributed == 2022.01.0"],
+    "distributed": ["distributed == 2022.02.0"],
     "diagnostics": [
         "bokeh >= 2.1.1",
         "jinja2",
diff -pruN 2022.01.0+dfsg-1/versioneer.py 2022.02.0+dfsg-1/versioneer.py
--- 2022.01.0+dfsg-1/versioneer.py	2022-01-14 20:37:34.000000000 +0000
+++ 2022.02.0+dfsg-1/versioneer.py	2022-02-11 16:21:12.000000000 +0000
@@ -348,11 +348,7 @@ https://creativecommons.org/publicdomain
 """
 
 
-try:
-    import configparser
-except ImportError:
-    import ConfigParser as configparser
-
+import configparser
 import errno
 import json
 import os
