Skip to content

Poor work scheduling when cluster adapts size #4471

Open
@chrisroat

Description

@chrisroat

What happened:

When a cluster is autoscaled in increments while running, as can happen in a GKE cluster, the work concentrates on few workers and uses the cluster inefficiently. This seems to be worse when there are long running tasks. The example below simulates this by adjusting a local cluster's size as it is processing a graph with 10-second tasks.

The image below shows the final look of the task graph, and the animated gif shows the status screen as the cluster processes the graph. Many workers do zero long tasks, and only a few workers seem to be fully utilised.

If the cluster is initially set to 20 workers with no changes, work is distributed evenly and all workers are efficiently used.

@fjetter In this example, the workers under load are green a lot.

What you expected to happen:

After the cluster scales up, the work should be evenly divided among all the workers.

Minimal Complete Verifiable Example:

import time
import distributed
import dask.array as da

client = distributed.Client(n_workers=1, threads_per_worker=1, memory_limit='1G')
print(client.dashboard_link)

# Slow task.
def func1(chunk):
    if sum(chunk.shape) != 0: # Make initialization fast
        time.sleep(10)
    return chunk

def func2(chunk):
    return chunk

data = da.zeros((30, 30, 30), chunks=5)
result = data.map_overlap(func1, depth=1, dtype=data.dtype)
result = result.map_overlap(func2, depth=1, dtype=data.dtype)
future = client.compute(result)

print('started computation')

time.sleep(11)
print('scaling to 4 workers')
client.cluster.scale(4)

time.sleep(5)
print('scaling to 20 workers')
client.cluster.scale(20)

_ = future.result()

Anything else we need to know?:

In a real cluster with my real load (not in the simulation above), I also will see the scheduler CPU pegged near 100% (possibly due to #3898), even when all workers are working on the long tasks. This seems odd, since nothing is being actively changed in the scheduling.

Environment:

  • Dask (and distributed) version: 2021.01.1
  • Python version: 3.8.5
  • Operating System: Ubuntu 20.04
  • Install method (conda, pip, source): pip

Final task screenshot:
Screen Shot 2021-01-30 at 7 23 00 AM

Movie of workload (at 10x speed):
Kapture 2021-01-30 at 07 28 34

Metadata

Metadata

Assignees

No one assigned

    Labels

    adaptiveAll things relating to adaptive scalingstealing

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions