-
-
Notifications
You must be signed in to change notification settings - Fork 730
adaptive_target: no more workers then runnable tasks #4155
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
Closed
Closed
Changes from all commits
Commits
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
There are no files selected for viewing
This file contains hidden or 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
This file contains hidden or 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 |
---|---|---|
|
@@ -6201,31 +6201,27 @@ def adaptive_target(self, comm=None, target_duration=None): | |
self.total_occupancy / target_duration | ||
) # TODO: threads per worker | ||
|
||
# Avoid a few long tasks from asking for many cores | ||
ws: WorkerState | ||
tasks_processing = 0 | ||
for ws in self.workers.values(): | ||
tasks_processing += len(ws._processing) | ||
|
||
if tasks_processing > cpu: | ||
break | ||
else: | ||
cpu = min(tasks_processing, cpu) | ||
|
||
if self.unrunnable and not self.workers: | ||
cpu = max(1, cpu) | ||
|
||
# Memory | ||
limit_bytes = {addr: ws._memory_limit for addr, ws in self.workers.items()} | ||
worker_bytes = [ws._nbytes for ws in self.workers.values()] | ||
limit = sum(limit_bytes.values()) | ||
total = sum(worker_bytes) | ||
if total > 0.6 * limit: | ||
# add more workers if more than 60% of memory is used | ||
limit = sum(ws.memory_limit for ws in self.workers.values()) | ||
used = sum(ws.nbytes for ws in self.workers.values()) | ||
memory = 0 | ||
if used > 0.6 * limit: | ||
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. it might be good to change this to |
||
memory = 2 * len(self.workers) | ||
else: | ||
memory = 0 | ||
|
||
target = max(memory, cpu) | ||
twoertwein marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
# avoid having more workers than runnable tasks | ||
runnable_states = ("processing", "no-worker") | ||
tasks = 0 | ||
for task_group in self.task_groups.values(): | ||
tasks += sum(task_group.states[state] for state in runnable_states) | ||
if tasks >= target: | ||
break | ||
target = min(target, tasks) | ||
|
||
if target >= len(self.workers): | ||
return target | ||
else: # Scale down? | ||
|
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.
this code prevented more worker to spawn for me.It is not needed anymore as the code now limits the number of targeted workers by the number of runnable tasks.