-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Consider candidates that don't hold any dependencies in decide_worker
#4925
Open
gjoseph92
wants to merge
25
commits into
dask:main
Choose a base branch
from
gjoseph92:decide_worker/add-random-candidates
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+55
−12
Open
Changes from 1 commit
Commits
Show all changes
25 commits
Select commit
Hold shift + click to select a range
5b17f55
Test workers without deps are considered
gjoseph92 4a81ca8
Consider random subset of workers in decide_worker
gjoseph92 c57fd72
no-sleep test
gjoseph92 d810d2d
Comment fastpath. Maybe this is still unnecessary?
gjoseph92 768d660
Pick from idle workers first
gjoseph92 346ab17
Update `many_independent_leaves` test
gjoseph92 420c99e
Uppercase Mb
gjoseph92 0a004b2
move N_RANDOM_WORKERS within conditional
gjoseph92 b050d14
Pass in sortedcontainers values, not pydict values
gjoseph92 9e99b7f
Use sleep test again
gjoseph92 f6acdc4
Simpler logic
gjoseph92 524da73
20 -> 10
gjoseph92 a5d37ae
Over-optimized
gjoseph92 5842ca8
Revert "Over-optimized"
gjoseph92 a159245
`random_choices_iter`. over-optimized for now.
gjoseph92 bb991d1
use `random.choices`
gjoseph92 58b4bf8
REBASEME Actor: don't hold key references on workers
gjoseph92 13975cb
Remove flaky data-length check
gjoseph92 fcb165e
No randomness if < 10 workers to choose from
gjoseph92 cd382c6
Ensure `decide_worker` args are plain dict_values
gjoseph92 cc57a8b
1 worker for `test_statistical_profiling`
gjoseph92 13911bc
no conditional on compiled
gjoseph92 f2445fe
rerun tests
gjoseph92 38e6b57
Merge remote-tracking branch 'upstream/main' into decide_worker/add-r…
gjoseph92 5794540
fix errant actor test
gjoseph92 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
random_choices_iter
. over-optimized for now.
- Loading branch information
commit a1592459984f3379582748b338765f9969b394e2
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,7 +18,7 @@ | |
from datetime import timedelta | ||
from functools import partial | ||
from numbers import Number | ||
from typing import Optional | ||
from typing import Optional, ValuesView | ||
|
||
import psutil | ||
import sortedcontainers | ||
|
@@ -2324,8 +2324,8 @@ def decide_worker(self, ts: TaskState) -> WorkerState: | |
if ts._dependencies or valid_workers is not None: | ||
ws = decide_worker( | ||
ts, | ||
self._workers.values(), | ||
self._idle.values(), | ||
self._workers_dv.values(), | ||
self._idle_dv.values(), | ||
valid_workers, | ||
partial(self.worker_objective, ts), | ||
) | ||
|
@@ -7461,8 +7461,8 @@ def _reevaluate_occupancy_worker(state: SchedulerState, ws: WorkerState): | |
@exceptval(check=False) | ||
def decide_worker( | ||
ts: TaskState, | ||
all_workers: sortedcontainers.SortedValuesView, | ||
idle_workers: sortedcontainers.SortedValuesView, | ||
all_workers: ValuesView, | ||
idle_workers: ValuesView, | ||
valid_workers: set, | ||
objective, | ||
) -> WorkerState: | ||
|
@@ -7482,6 +7482,8 @@ def decide_worker( | |
of bytes sent between workers. This is determined by calling the | ||
*objective* function. | ||
""" | ||
# NOTE: `all_workers` and `idle_workers` must be plain `dict_values` objects, | ||
# not a `SortedValuesView`, which is much slower to iterate over. | ||
ws: WorkerState = None | ||
wws: WorkerState | ||
dts: TaskState | ||
|
@@ -7494,13 +7496,14 @@ def decide_worker( | |
# Select all workers holding deps of this task | ||
candidates = {wws for dts in deps for wws in dts._who_has} | ||
# Add up to 10 random workers into `candidates`, preferring idle ones. | ||
sample_from = ( | ||
list(valid_workers) | ||
if valid_workers is not None | ||
else idle_workers or all_workers | ||
) | ||
candidates.update(random.sample(sample_from, min(10, len(sample_from)))) | ||
# ^ NOTE: `min` because `random.sample` errors if `len(sample) < k` | ||
worker_pool = valid_workers if valid_workers is not None else all_workers | ||
if len(candidates) < len(worker_pool): | ||
sample_from = idle_workers or worker_pool | ||
candidates.update( | ||
random_choices_iter(sample_from, 10) | ||
if len(sample_from) > 10 | ||
else sample_from | ||
) | ||
if valid_workers is None: | ||
if not candidates: | ||
candidates = set(all_workers) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Now that we have idle_workers in this function should this be There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Interesting; yes it probably should be. |
||
|
@@ -7524,6 +7527,11 @@ def decide_worker( | |
return ws | ||
|
||
|
||
def random_choices_iter(xs: Iterable, k: int) -> Iterable: | ||
"Randomly choose between 0 and *k* items from *xs*" | ||
return itertools.islice(itertools.takewhile(lambda _: random.random() < 0.5, xs), k) | ||
|
||
|
||
def validate_task_state(ts: TaskState): | ||
""" | ||
Validate the given TaskState. | ||
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What if
valid_workers and idle_workers
?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems like maybe we want to use
idle_workers or all_workers
above in line 7501There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We'd then intersect
candidates
withvalid_workers
, so at least we wouldn't ever pick an invalid worker:distributed/distributed/scheduler.py
Lines 7512 to 7519 in f2445fe
But it's true that if
idle_workers
andvalid_workers
are disjoint, then we haven't gained anything here.How about
(valid_workers.intersection(idle_workers) or valid_workers) if valid_workers is not None else (idle_workers or valid_workers)
?Or for simplicity we could ignore
idle_workers
when there arevalid_workers
given, and just usevalid_workers
.