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

More comprehensive WorkerState task counters #7167

Merged
merged 2 commits into from
Oct 24, 2022

Conversation

crusaderky
Copy link
Collaborator

@crusaderky crusaderky commented Oct 20, 2022

  • Expose a more comprehensive and accurate view of the number of tasks in each state on the worker to the scheduler (through the heartbeat) and to prometheus. Crucially, this PR gives us a measure of network saturation and co-assignment by showing how many tasks are queued up in fetch state.
  • O(1) count number of tasks in fetch state
  • Closes Inconsistent Worker.waiting_for_data_count #6319
  • Disallow directly writing to Worker.data. I am quite frankly baffled this was possible before. I don't think we should have a deprecation cycle for this?
  • Prometheus tasks metric stored has been renamed to memory CC @ntabris

tasks.add_metric(["ready"], len(self.server.state.ready))
tasks.add_metric(["waiting"], self.server.state.waiting_for_data_count)
for k, n in ws.task_counts.items():
tasks.add_metric([k], n)
Copy link
Collaborator Author

@crusaderky crusaderky Oct 20, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Renamed stored -> memory and added many more measures

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what's the difference between things in data and actors in "memory" state? (I noticed that actors weren't included in previous metric and are included in new metric)

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The "real" instance of an actor lives in WorkerState.actors instead of data. Its proxies on remote workers live in data. Actors are an edge case I would not spend time fine-tuning anything towards anyway.

# Actors can be in any state other than {fetch, flight, missing}
n_actors_in_memory = sum(
self.tasks[key].state == "memory" for key in self.actors
)
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is O(n) but actors are such a niche feature that we should not care in 99% of the cases

@github-actions
Copy link
Contributor

github-actions bot commented Oct 20, 2022

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

       15 files  ±  0         15 suites  ±0   6h 5m 41s ⏱️ - 12m 50s
  3 153 tests +  3    3 062 ✔️ +  4    85 💤 ±0    6  - 1 
23 328 runs  +21  22 394 ✔️ +22  913 💤 ±0  21  - 1 

For more details on these failures, see this check.

Results for commit 900e8b2. ± Comparison against base commit 6afce9c.

♻️ This comment has been updated with latest results.

@crusaderky crusaderky self-assigned this Oct 20, 2022
@crusaderky crusaderky marked this pull request as ready for review October 20, 2022 21:48
@crusaderky
Copy link
Collaborator Author

Ready for review and merge. All test failures seem unrelated.

Copy link
Member

@hendrikmakait hendrikmakait left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks @crusaderky!

"flight": len(self.in_flight_tasks),
}
# released | error
out["other"] = other = len(self.tasks) - sum(out.values())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Out of scope: I am wondering whether having the count for error tasks available would be worth the additional effort of maintaining yet another set. This feels like useful information in Prometheus, but might also be less relevant since we track erred tasks (after retries) on the scheduler as well. What do you think?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It felt a bit useless, from the point of view of performance metrics. An erred task is just sitting there until it's either released or its exception is retrieved.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fair point, I guess a counter of tasks transitioning into this state would be the interesting bit here rather than any point-in-time count.

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.

Inconsistent Worker.waiting_for_data_count
3 participants