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

[Dataset] Pin _StatsActor to the driver node #27765

Merged
merged 1 commit into from
Aug 11, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 11 additions & 1 deletion python/ray/data/_internal/stats.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
from ray.data._internal.block_list import BlockList
from ray.data.block import BlockMetadata
from ray.data.context import DatasetContext
from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy


def fmt(seconds: float) -> str:
Expand Down Expand Up @@ -123,10 +124,19 @@ def _get_or_create_stats_actor():
or _stats_actor[1] != ray.get_runtime_context().job_id.hex()
):
ctx = DatasetContext.get_current()
scheduling_strategy = ctx.scheduling_strategy
if not ray.util.client.ray.is_connected():
# Pin the stats actor to the local node
# so it fate-shares with the driver.
scheduling_strategy = NodeAffinitySchedulingStrategy(
ray.get_runtime_context().get_node_id(),
soft=False,
)

_stats_actor[0] = _StatsActor.options(
name="datasets_stats_actor",
get_if_exists=True,
scheduling_strategy=ctx.scheduling_strategy,
scheduling_strategy=scheduling_strategy,
).remote()
_stats_actor[1] = ray.get_runtime_context().job_id.hex()

Expand Down
14 changes: 8 additions & 6 deletions python/ray/data/dataset_pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
from ray.data.row import TableRow
from ray.types import ObjectRef
from ray.util.annotations import DeveloperAPI, PublicAPI
from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy

if TYPE_CHECKING:
import pandas
Expand Down Expand Up @@ -298,18 +299,19 @@ def split_at_indices(self, indices: List[int]) -> List["DatasetPipeline[T]"]:
def _split(
self, n: int, splitter: Callable[[Dataset], List["Dataset[T]"]]
) -> List["DatasetPipeline[T]"]:
resources = {}
ctx = DatasetContext.get_current()
scheduling_strategy = ctx.scheduling_strategy
if not ray.util.client.ray.is_connected():
# Pin the coordinator (and any child actors) to the local node to avoid
# errors during node failures. If the local node dies, then the driver
# will fate-share with the coordinator anyway.
resources["node:{}".format(ray.util.get_node_ip_address())] = 0.0001

ctx = DatasetContext.get_current()
scheduling_strategy = NodeAffinitySchedulingStrategy(
ray.get_runtime_context().get_node_id(),
soft=False,
)

coordinator = PipelineSplitExecutorCoordinator.options(
resources=resources,
scheduling_strategy=ctx.scheduling_strategy,
scheduling_strategy=scheduling_strategy,
).remote(self, n, splitter, DatasetContext.get_current())
if self._executed[0]:
raise RuntimeError("Pipeline cannot be read multiple times.")
Expand Down