Skip to content

Poor adaptive target for empty clusters #6962

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
gjoseph92 opened this issue Aug 26, 2022 · 2 comments
Open

Poor adaptive target for empty clusters #6962

gjoseph92 opened this issue Aug 26, 2022 · 2 comments
Labels
adaptive All things relating to adaptive scaling enhancement Improve existing functionality or make things work better scheduling

Comments

@gjoseph92
Copy link
Collaborator

gjoseph92 commented Aug 26, 2022

If a cluster hasn't run any work yet, it will only recommend 1 worker initially, regardless of how many tasks are queued on the scheduler:

@gen_cluster(
    client=True,
    nthreads=[],
    config={"distributed.scheduler.default-task-durations": {"inc": 1}},
)
async def test_adaptive_target_empty_cluster(c, s):
    assert s.adaptive_target() == 0

    f = c.submit(inc, -1)
    await async_wait_for(lambda: s.tasks, timeout=5)
    assert s.adaptive_target() == 1

    fs = c.map(inc, range(1000))
    await async_wait_for(lambda: len(s.tasks) == len(fs) + 1, timeout=5)
    print(s.total_occupancy)
  > assert s.adaptive_target() > 1
E   AssertionError: assert 1 > 1

The scheduler's adaptive target is based on looking at its total_occupancy. But occupancy is only updated once tasks are scheduled (into processing). So if there are no workers, no tasks can be scheduled, and occupancy remains 0 even with tons of tasks in unrunnable.

I would expect the total_occupancy to also include the expected runtime of all unrunnable/queued tasks. That would result in faster scale-up from zero usually. Some deployment systems might be quite slow to scale. You might have to wait a few minutes to get 1 worker, to realize you then need more, and then wait a few minutes again. It would be better to ask for more up front.

This is what ensures we at least get one worker, otherwise we'd never scale up at all:

if self.unrunnable and not self.workers:
cpu = max(1, cpu)

@gjoseph92 gjoseph92 added enhancement Improve existing functionality or make things work better scheduling labels Aug 26, 2022
@fjetter fjetter added the adaptive All things relating to adaptive scaling label Aug 26, 2022
@jacobtomlinson
Copy link
Member

Yeah I've thought about this before. Typically we recommend folks set a minimum of 1 worker anyway because scale up latency is probably significant (especially for dask-cloudprovider or dask-jobqueue) so it's best to have 1 worker idling that can pick up tasks immediately and then the scheduler can make a more realistic recommendation.

Do you have a specific use case where scaling to 0 is desirable?

@gjoseph92
Copy link
Collaborator Author

I pointed it out more as an edge case. I definitely think scaling from 0 is desirable if it works. It's just odd to allow it but then not handle that case properly.

Task rebalancing during scale-up is usually pretty bad anyway, so I wonder how much of a difference a 0 vs 1 minimum would even make (if we had a good scale-up metric from 0).

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
adaptive All things relating to adaptive scaling enhancement Improve existing functionality or make things work better scheduling
Projects
None yet
Development

No branches or pull requests

3 participants