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

Support custom resource with remote args in random_shuffle_each_window() #29482

Merged
merged 4 commits into from
Oct 20, 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
5 changes: 4 additions & 1 deletion python/ray/data/dataset_pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -824,11 +824,14 @@ def random_shuffle_each_window(
*,
seed: Optional[int] = None,
num_blocks: Optional[int] = None,
**ray_remote_args,
) -> "DatasetPipeline[U]":
"""Apply :py:meth:`Dataset.random_shuffle <ray.data.Dataset.random_shuffle>` to
each dataset/window in this pipeline."""
return self.foreach_window(
lambda ds: ds.random_shuffle(seed=seed, num_blocks=num_blocks)
lambda ds: ds.random_shuffle(
seed=seed, num_blocks=num_blocks, **ray_remote_args
)
)

def sort_each_window(
Expand Down
27 changes: 27 additions & 0 deletions python/ray/data/tests/test_dataset_pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -634,6 +634,33 @@ def test_drop_columns(ray_start_regular_shared):
assert pipe.drop_columns(["col2"]).take(1) == [{"col1": 1, "col3": 3}]


def test_random_shuffle_each_window_with_custom_resource(ray_start_cluster):
ray.shutdown()
cluster = ray_start_cluster
# Create two nodes which have different custom resources.
cluster.add_node(
resources={"foo": 100},
num_cpus=1,
)
cluster.add_node(resources={"bar": 100}, num_cpus=1)

ray.init(cluster.address)

# Run pipeline in "bar" nodes.
pipe = ray.data.read_datasource(
ray.data.datasource.RangeDatasource(),
parallelism=10,
n=1000,
block_format="list",
ray_remote_args={"resources": {"bar": 1}},
).repeat(3)
pipe = pipe.random_shuffle_each_window(resources={"bar": 1})
for batch in pipe.iter_batches():
pass
assert "1 nodes used" in pipe.stats()
assert "2 nodes used" not in pipe.stats()


def test_in_place_transformation_doesnt_clear_objects(ray_start_regular_shared):
ds = ray.data.from_items([1, 2, 3, 4, 5, 6])

Expand Down