Skip to content

Distributed scheduler does not obey dask.order.order for num_workers=1, num_threads=1 #5555

@davidhao3300

Description

@davidhao3300

What happened:

Distributed scheduler appears to be ignoring the priority set by dask.order.order and instead does a bread-first execution order. This problem is particularly prominent when doing simple per-partition operation chains, where we are able to read one input partition, do some work, then write the results to a partition.

The end result for this particular usecase is that we load ALL parquet partitions first, apply the function to each one, then write each one to file.

While the issue is about num_workers=1, num_threads=1, I do think the suboptimal scheduling occurs in situations involving concurrency as well, and hopefully addressing the serial case will help with the concurrent case.

dask.order.order execution order:

Screen Shot 2021-12-01 at 6 47 14 PM

See how we process the left chain as much as possible before moving to the right chain, as expected

Actual execution:

Screen Shot 2021-12-01 at 6 46 36 PM

I made the function pause in the middle of execution on the top green node. Note how both chains are in-memory. I would've expected to see the bottom chain either completely grayed out (not started), or that the bottom-right node be the only thing in memory.

What you expected to happen:

We instead should be doing depth-first execution order like dask.order.order suggests.

Minimal Complete Verifiable Example:

import pandas as pd
import numpy as np
import dask.dataframe as dd
import dask
from dask.distributed import Client

# Write data to parquet as a setup, to demonstrate unnecessary memory usage
df = pd.DataFrame(np.random.randn(100,100), columns=[str(x) for x in range(100)])

# Can change npartitions to 10 to show that this problem becomes severe at higher partition count
ddf = dd.from_pandas(df, npartitions=2)
ddf.to_parquet("input")

# You may need to set host and dashboard_address to expose dashboard
# We set n_workers=1 and threads_per_worker=1 to get rid of concurrency for simplicity
client=Client(n_workers=1, threads_per_worker=1)

ddf = dd.read_parquet("input")

def f(df, partition_info=None):
    # Pause execution on second partition
    if partition_info["number"] == 1:
        import time
        while True:
            time.sleep(5)
    return df

ddf = ddf.map_partitions(f, meta=ddf)
ddf = ddf.to_parquet('test', compute=False)

# Run this to show that the dask.order.order output is reasonable
# ddf.visualize(color="order")

# This will hang (on purpose). Go to dashboard graph view to see the graph screenshotted in this issue.
dask.compute(ddf, scheduler=client)

Anything else we need to know?:

Environment:

  • Dask version: 2021.11.2
  • Python version: 3.8.x
  • Operating System:
  • Install method (conda, pip, source):

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions