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

[BUG] Fix actor pool initialization in ray client mode #3028

Merged
merged 14 commits into from
Oct 21, 2024

Conversation

kevinzwang
Copy link
Member

@kevinzwang kevinzwang commented Oct 11, 2024

This PR moves the actor_pool_context method from the ray runner to the scheduler, and routes the relevant ActorPoolManager implementation to actor_pool_project. That way, it does not accidentally get the wrong actor pool context when the scheduler is running on a ray actor, which we do when in ray client mode.

With this change, in addition to separating the actor_pool_context method out of Runner into ActorPoolManager, I also move some other things around to clean things up, especially so that ray_runner.py no longer depends on the pyrunner.py.

Note:

  • One of the tests was cut down because it needed 3 CPUs to run, which combined with the 1 CPU for the scheduler actor, meant no more tasks could even be scheduled. I plan on adding some more informative errors/warnings when this happens in a future PR

@github-actions github-actions bot added the bug Something isn't working label Oct 11, 2024
Copy link

codspeed-hq bot commented Oct 11, 2024

CodSpeed Performance Report

Merging #3028 will not alter performance

Comparing kevin/ray-client-actors (d2b50b8) with main (6173006)

Summary

✅ 17 untouched benchmarks

Copy link

codecov bot commented Oct 11, 2024

Codecov Report

Attention: Patch coverage is 94.73684% with 8 lines in your changes missing coverage. Please review.

Project coverage is 78.26%. Comparing base (6173006) to head (d2b50b8).
Report is 1 commits behind head on main.

Files with missing lines Patch % Lines
daft/runners/partitioning.py 90.47% 6 Missing ⚠️
daft/execution/physical_plan.py 87.50% 1 Missing ⚠️
src/daft-scheduler/src/scheduler.rs 98.30% 1 Missing ⚠️
Additional details and impacted files

Impacted file tree graph

@@            Coverage Diff             @@
##             main    #3028      +/-   ##
==========================================
- Coverage   78.45%   78.26%   -0.20%     
==========================================
  Files         611      611              
  Lines       72619    72804     +185     
==========================================
+ Hits        56975    56979       +4     
- Misses      15644    15825     +181     
Files with missing lines Coverage Δ
daft/context.py 80.00% <100.00%> (+0.51%) ⬆️
daft/dataframe/dataframe.py 86.57% <100.00%> (-0.02%) ⬇️
daft/execution/native_executor.py 89.47% <100.00%> (+4.47%) ⬆️
daft/execution/rust_physical_plan_shim.py 94.56% <ø> (ø)
daft/io/file_path.py 100.00% <100.00%> (ø)
daft/plan_scheduler/physical_plan_scheduler.py 54.90% <100.00%> (ø)
daft/runners/pyrunner.py 85.00% <100.00%> (-0.88%) ⬇️
daft/runners/ray_runner.py 81.20% <100.00%> (+0.35%) ⬆️
daft/runners/runner.py 76.00% <ø> (+4.12%) ⬆️
daft/execution/physical_plan.py 94.18% <87.50%> (-0.10%) ⬇️
... and 2 more

... and 11 files with indirect coverage changes

@kevinzwang kevinzwang marked this pull request as draft October 14, 2024 19:01
@kevinzwang kevinzwang marked this pull request as ready for review October 16, 2024 23:21
Copy link
Contributor

@jaychia jaychia left a comment

Choose a reason for hiding this comment

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

Sweet! Seems like making the Scheduler the ActorPoolManager solved most of the problems. Pretty good.

I'm not 100% sure we want to add another test to our matrix though. Let me check with @samster25

self, psets: dict[str, list[PartitionT]], results_buffer_size: int | None
self,
psets: dict[str, list[PartitionT]],
actor_pool_manager: physical_plan.ActorPoolManager,
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice

@@ -271,6 +271,92 @@ def wait(self) -> None:
raise NotImplementedError()


class LocalPartitionSet(PartitionSet[MicroPartition]):
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this a pure movement of the class from pyrunner.py? Why did we decide to move it?

Copy link
Member Author

Choose a reason for hiding this comment

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

It was being imported by dataframe.py and ray_runner.py which was causing some circular import issues. I figured since it was being used outside of the pyrunner I would move it out to simplify the dependency tree.

@@ -259,6 +260,7 @@ fn physical_plan_to_partition_tasks(
physical_plan: &PhysicalPlan,
py: Python,
psets: &HashMap<String, Vec<PyObject>>,
actor_pool_manager: &PyObject,
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe a good idea to actually wrap this into a ExecutionContext? We could start using that to thread other things through as well (e.g. the execution config).

Not blocking for this PR, but could be great to formalize in the future.

Copy link
Member Author

Choose a reason for hiding this comment

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

That might make sense. I'll try that

Copy link
Member Author

Choose a reason for hiding this comment

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

After looking at it for a bit, I don't think it would be the best idea, since the execution config is actually put into the ray object store and sent to all of the tasks, whereas the actor pool manager does not need to leave the scheduler actor.

df = df.with_column(
"more_cpu_request",
assert_resources_2(col("id"), num_cpus=1),
)
Copy link
Contributor

Choose a reason for hiding this comment

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

Why rm?

Copy link
Member Author

Choose a reason for hiding this comment

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

See my note above

Note:

  • One of the tests was cut down because it needed 3 CPUs to run, which combined with the 1 CPU for the scheduler actor, meant no more tasks could even be scheduled. I plan on adding some more informative errors/warnings when this happens in a future PR

@kevinzwang kevinzwang enabled auto-merge (squash) October 21, 2024 18:56
@kevinzwang kevinzwang merged commit 23d4a1f into main Oct 21, 2024
40 checks passed
@kevinzwang kevinzwang deleted the kevin/ray-client-actors branch October 21, 2024 19:13
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants