Skip to content
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

Collect worker-worker and type bandwidth information #3094

Merged
merged 12 commits into from
Oct 1, 2019

Conversation

mrocklin
Copy link
Member

@mrocklin mrocklin commented Sep 26, 2019

This collects the bandwidth that we observe both by type,
and by worker-worker pair.

This is currently used in dashboards plots (see below) and may be used in scheduling decisions in the future:

cc @quasiben @pentschev

Screen Shot 2019-09-26 at 4 39 07 PM

@quasiben
Copy link
Member

Very cool!

@mrocklin
Copy link
Member Author

This information is also available if people want to dive into the scheduler internals directly:

In [1]: from dask.distributed import Client
In [2]: client = Client()
In [3]: import dask.array as da
In [4]: x = da.random.random((30000, 30000), chunks="128 MiB")

In [5]: y = (x + x.T) - x.mean(axis=0); y.sum().compute()
Out[5]: 449994918.14960176

In [6]: client.cluster.scheduler.bandwidth
Out[6]: 101225272.64331588

In [7]: client.cluster.scheduler.bandwidth_types
Out[7]: defaultdict(float, {'numpy.ndarray': 53980385.799921274})

In [8]: client.cluster.scheduler.bandwidth_workers
Out[8]:
defaultdict(float,
            {('tcp://127.0.0.1:52998',
              'tcp://127.0.0.1:52997'): 30879879.40114621,
             ('tcp://127.0.0.1:52998',
              'tcp://127.0.0.1:53000'): 32331470.115519114,
             ('tcp://127.0.0.1:53000',
              'tcp://127.0.0.1:52997'): 37212961.59090071,
             ('tcp://127.0.0.1:52997',
              'tcp://127.0.0.1:53000'): 16128730.449103117,
             ('tcp://127.0.0.1:52999',
              'tcp://127.0.0.1:52997'): 14742638.825626642,
             ('tcp://127.0.0.1:52997',
              'tcp://127.0.0.1:52999'): 172327687.3918158,
             ('tcp://127.0.0.1:52999',
              'tcp://127.0.0.1:53000'): 317673916.5222039,
             ('tcp://127.0.0.1:52998',
              'tcp://127.0.0.1:52999'): 5480834.635507911,
             ('tcp://127.0.0.1:52999',
              'tcp://127.0.0.1:52998'): 8030296.722465654,
             ('tcp://127.0.0.1:52997',
              'tcp://127.0.0.1:52998'): 137043235.69919452,
             ('tcp://127.0.0.1:53000',
              'tcp://127.0.0.1:52999'): 5711893.887075517,
             ('tcp://127.0.0.1:53000',
              'tcp://127.0.0.1:52998'): 59961461.29405087})

@mrocklin
Copy link
Member Author

And now the worker-worker data is also available visually:

image

@mrocklin
Copy link
Member Author

Now also available in JupyterLab

image

@mrocklin
Copy link
Member Author

mrocklin commented Oct 1, 2019

There are problems with this in the many-worker case:

  1. If there are, say, 1000 workers, then this will maintain a million element list, updated fairly frequently.
  2. Workers never learn when othere workers die, and so they don't forget bandwidths to peers. They continuously send this information to the scheduler, which can be a problem if there are many workers, or if our workers are highly adaptive, and come and go rapidly

@mrocklin
Copy link
Member Author

mrocklin commented Oct 1, 2019

OK, resolved. We no longer keep long-term bandwidths on the workers, but clear their information and only send up diffs to the scheduler. This should improve scalability as well.

@TomAugspurger
Copy link
Member

@mrocklin I'm digging into the worker code now, but perhaps you know off the top of your head: why would there ever be communication from a worker to itself?

Screen Shot 2020-05-13 at 10 26 50 AM

@mrocklin
Copy link
Member Author

mrocklin commented May 13, 2020 via email

@TomAugspurger
Copy link
Member

Yep, just noticed that when I restarted the cluster and the boxes were still there. Makes sense, thanks.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants