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

[Datasets] Add support for callable classes to new execution backend. #31706

Merged
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
6 changes: 6 additions & 0 deletions python/ray/data/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,12 @@
read_tfrecords,
)


# Module-level cached global functions for callable classes. It needs to be defined here
# since it has to be process-global across cloudpickled funcs.
_cached_fn = None
_cached_cls = None

__all__ = [
"ActorPoolStrategy",
"Dataset",
Expand Down
51 changes: 44 additions & 7 deletions python/ray/data/_internal/execution/legacy_compat.py
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
"""

import ray.cloudpickle as cloudpickle
from typing import Iterator, Tuple
from typing import Iterator, Tuple, Any

import ray
from ray.data.block import Block, BlockMetadata, List
Expand All @@ -13,7 +13,12 @@
from ray.data._internal.stage_impl import RandomizeBlocksStage
from ray.data._internal.block_list import BlockList
from ray.data._internal.lazy_block_list import LazyBlockList
from ray.data._internal.compute import get_compute
from ray.data._internal.compute import (
get_compute,
CallableClass,
TaskPoolStrategy,
ActorPoolStrategy,
)
from ray.data._internal.memory_tracing import trace_allocation
from ray.data._internal.plan import ExecutionPlan, OneToOneStage, AllToAllStage, Stage
from ray.data._internal.execution.operators.map_operator import MapOperator
Expand Down Expand Up @@ -136,12 +141,44 @@ def _stage_to_operator(stage: Stage, input_op: PhysicalOperator) -> PhysicalOper
"""

if isinstance(stage, OneToOneStage):
if stage.fn_constructor_args or stage.fn_constructor_kwargs:
raise NotImplementedError
compute = get_compute(stage.compute)

block_fn = stage.block_fn
# TODO: implement arg packing and passing for test_map_batches_extra_args
fn_args = (stage.fn,) if stage.fn else ()
if stage.fn:
if isinstance(stage.fn, CallableClass):
if isinstance(compute, TaskPoolStrategy):
raise ValueError(
"``compute`` must be specified when using a callable class, "
"and must specify the actor compute strategy. "
'For example, use ``compute="actors"`` or '
"``compute=ActorPoolStrategy(min, max)``."
)
assert isinstance(compute, ActorPoolStrategy)

fn_constructor_args = stage.fn_constructor_args or ()
fn_constructor_kwargs = stage.fn_constructor_kwargs or {}
fn_ = stage.fn

def fn(item: Any) -> Any:
# Wrapper providing cached instantiation of stateful callable class
# UDFs.
if ray.data._cached_fn is None:
ray.data._cached_cls = fn_
ray.data._cached_fn = fn_(
*fn_constructor_args, **fn_constructor_kwargs
clarkzinzow marked this conversation as resolved.
Show resolved Hide resolved
)
else:
# A worker is destroyed when its actor is killed, so we
# shouldn't have any worker reuse across different UDF
# applications (i.e. different map operators).
assert ray.data._cached_cls == fn_
return ray.data._cached_fn(item)

else:
fn = stage.fn
fn_args = (fn,)
else:
fn_args = ()
if stage.fn_args:
fn_args += stage.fn_args
fn_kwargs = stage.fn_kwargs or {}
Expand All @@ -153,7 +190,7 @@ def do_map(blocks: Iterator[Block]) -> Iterator[Block]:
do_map,
input_op,
name=stage.name,
compute_strategy=get_compute(stage.compute),
compute_strategy=compute,
min_rows_per_bundle=stage.target_block_size,
ray_remote_args=stage.ray_remote_args,
)
Expand Down
35 changes: 21 additions & 14 deletions python/ray/data/tests/test_dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -2432,6 +2432,13 @@ def test_map_batches_basic(ray_start_regular_shared, tmp_path):


def test_map_batches_extra_args(ray_start_regular_shared, tmp_path):
def put(x):
# We only support automatic deref in the legacy backend.
if DatasetContext.get_current().new_execution_backend:
return x
else:
return ray.put(x)

# Test input validation
ds = ray.data.range(5)

Expand Down Expand Up @@ -2483,7 +2490,7 @@ def udf(batch, a):
udf,
batch_size=1,
batch_format="pandas",
fn_args=(ray.put(1),),
fn_args=(put(1),),
)
assert ds2.dataset_format() == "pandas"
ds_list = ds2.take()
Expand All @@ -2502,7 +2509,7 @@ def udf(batch, b=None):
udf,
batch_size=1,
batch_format="pandas",
fn_kwargs={"b": ray.put(2)},
fn_kwargs={"b": put(2)},
)
assert ds2.dataset_format() == "pandas"
ds_list = ds2.take()
Expand All @@ -2522,8 +2529,8 @@ def udf(batch, a, b=None):
udf,
batch_size=1,
batch_format="pandas",
fn_args=(ray.put(1),),
fn_kwargs={"b": ray.put(2)},
fn_args=(put(1),),
fn_kwargs={"b": put(2)},
)
assert ds2.dataset_format() == "pandas"
ds_list = ds2.take()
Expand All @@ -2548,7 +2555,7 @@ def __call__(self, x):
batch_size=1,
batch_format="pandas",
compute="actors",
fn_constructor_args=(ray.put(1),),
fn_constructor_args=(put(1),),
)
assert ds2.dataset_format() == "pandas"
ds_list = ds2.take()
Expand All @@ -2572,7 +2579,7 @@ def __call__(self, x):
batch_size=1,
batch_format="pandas",
compute="actors",
fn_constructor_kwargs={"b": ray.put(2)},
fn_constructor_kwargs={"b": put(2)},
)
assert ds2.dataset_format() == "pandas"
ds_list = ds2.take()
Expand All @@ -2598,8 +2605,8 @@ def __call__(self, x):
batch_size=1,
batch_format="pandas",
compute="actors",
fn_constructor_args=(ray.put(1),),
fn_constructor_kwargs={"b": ray.put(2)},
fn_constructor_args=(put(1),),
fn_constructor_kwargs={"b": put(2)},
)
assert ds2.dataset_format() == "pandas"
ds_list = ds2.take()
Expand All @@ -2610,8 +2617,8 @@ def __call__(self, x):

# Test callable chain.
ds = ray.data.read_parquet(str(tmp_path))
fn_constructor_args = (ray.put(1),)
fn_constructor_kwargs = {"b": ray.put(2)}
fn_constructor_args = (put(1),)
fn_constructor_kwargs = {"b": put(2)}
ds2 = (
ds.lazy()
.map_batches(
Expand Down Expand Up @@ -2640,17 +2647,17 @@ def __call__(self, x):

# Test function + callable chain.
ds = ray.data.read_parquet(str(tmp_path))
fn_constructor_args = (ray.put(1),)
fn_constructor_kwargs = {"b": ray.put(2)}
fn_constructor_args = (put(1),)
fn_constructor_kwargs = {"b": put(2)}
ds2 = (
ds.lazy()
.map_batches(
lambda df, a, b=None: b * df + a,
batch_size=1,
batch_format="pandas",
compute="actors",
fn_args=(ray.put(1),),
fn_kwargs={"b": ray.put(2)},
fn_args=(put(1),),
fn_kwargs={"b": put(2)},
)
.map_batches(
CallableFn,
Expand Down