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 select_columns API to allow users to select a subset of columns #29081

Merged
merged 17 commits into from
Oct 26, 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
3 changes: 3 additions & 0 deletions doc/source/data/api/dataset.rst
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ Dataset API
ray.data.Dataset.filter
ray.data.Dataset.add_column
ray.data.Dataset.drop_columns
ray.data.Dataset.select_columns
ray.data.Dataset.random_sample
ray.data.Dataset.limit

Expand Down Expand Up @@ -144,6 +145,8 @@ Basic Transformations

.. automethod:: ray.data.Dataset.drop_columns

.. automethod:: ray.data.Dataset.select_columns

.. automethod:: ray.data.Dataset.random_sample

.. automethod:: ray.data.Dataset.limit
Expand Down
3 changes: 3 additions & 0 deletions doc/source/data/api/dataset_pipeline.rst
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ DatasetPipeline API
ray.data.DatasetPipeline.filter
ray.data.DatasetPipeline.add_column
ray.data.DatasetPipeline.drop_columns
ray.data.DatasetPipeline.select_columns

**Sorting, Shuffling, Repartitioning**

Expand Down Expand Up @@ -98,6 +99,8 @@ Basic transformations

.. automethod:: ray.data.DatasetPipeline.drop_columns

.. automethod:: ray.data.DatasetPipeline.select_columns

Sorting, Shuffling, Repartitioning
----------------------------------

Expand Down
38 changes: 38 additions & 0 deletions python/ray/data/dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,44 @@ def drop_columns(
lambda batch: batch.drop(columns=cols), compute=compute, **ray_remote_args
)

def select_columns(
self,
cols: List[str],
*,
compute: Union[str, ComputeStrategy] = None,
**ray_remote_args,
) -> "Dataset[T]":
"""Select one or more columns from the dataset.

All input columns used to select need to be in the schema of the dataset.

Examples:
>>> import ray
>>> # Create a dataset with 3 columns
>>> ds = ray.data.from_items([{"col1": i, "col2": i+1, "col3": i+2}
... for i in range(10)])
>>> # Select only "col1" and "col2" columns.
>>> ds = ds.select_columns(cols=["col1", "col2"])
>>> ds
Dataset(num_blocks=10, num_rows=10, schema={col1: int64, col2: int64})


Time complexity: O(dataset size / parallelism)

Args:
cols: Names of the columns to select. If any name is not included in the
dataset schema, an exception will be raised.
compute: The compute strategy, either "tasks" (default) to use Ray
tasks, or ActorPoolStrategy(min, max) to use an autoscaling actor pool.
ray_remote_args: Additional resource requirements to request from
ray (e.g., num_gpus=1 to request GPUs for the map tasks).
"""
return self.map_batches(
lambda batch: BlockAccessor.for_block(batch).select(columns=cols),
compute=compute,
**ray_remote_args,
)

def flat_map(
self,
fn: RowUDF[T, U],
Expand Down
13 changes: 13 additions & 0 deletions python/ray/data/dataset_pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -810,6 +810,19 @@ def drop_columns(
lambda ds: ds.drop_columns(cols, compute=compute, **ray_remote_args)
)

def select_columns(
self,
cols: List[str],
*,
compute: Optional[str] = None,
**ray_remote_args,
) -> "DatasetPipeline[U]":
"""Apply :py:meth:`Dataset.select_columns <ray.data.Dataset.select_columns>` to
each dataset/window in this pipeline."""
return self.foreach_window(
lambda ds: ds.select_columns(cols, compute=compute, **ray_remote_args)
)

def repartition_each_window(
self, num_blocks: int, *, shuffle: bool = False
) -> "DatasetPipeline[U]":
Expand Down
37 changes: 37 additions & 0 deletions python/ray/data/tests/test_dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -2223,6 +2223,43 @@ def test_drop_columns(ray_start_regular_shared, tmp_path):
ds.drop_columns(["dummy_col", "col1", "col2"])


def test_select_columns(ray_start_regular_shared):
heyitsmui marked this conversation as resolved.
Show resolved Hide resolved
# Test pandas and arrow
df = pd.DataFrame({"col1": [1, 2, 3], "col2": [2, 3, 4], "col3": [3, 4, 5]})
ds1 = ray.data.from_pandas(df)
assert ds1._dataset_format() == "pandas"

ds2 = ds1.map_batches(lambda pa: pa, batch_size=1, batch_format="pyarrow")
assert ds2._dataset_format() == "arrow"

for each_ds in [ds1, ds2]:
assert each_ds.select_columns(cols=[]).take(1) == [{}]
assert each_ds.select_columns(cols=["col1", "col2", "col3"]).take(1) == [
{"col1": 1, "col2": 2, "col3": 3}
]
assert each_ds.select_columns(cols=["col1", "col2"]).take(1) == [
{"col1": 1, "col2": 2}
]
assert each_ds.select_columns(cols=["col2", "col1"]).take(1) == [
{"col1": 1, "col2": 2}
]
# Test selecting columns with duplicates
assert each_ds.select_columns(cols=["col1", "col2", "col2"]).schema().names == [
"col1",
"col2",
"col2",
]
# Test selecting a column that is not in the dataset schema
with pytest.raises(KeyError):
each_ds.select_columns(cols=["col1", "col2", "dummy_col"])
heyitsmui marked this conversation as resolved.
Show resolved Hide resolved

# Test simple
ds3 = ray.data.range(10)
assert ds3._dataset_format() == "simple"
with pytest.raises(ValueError):
ds3.select_columns(cols=[])

heyitsmui marked this conversation as resolved.
Show resolved Hide resolved

def test_map_batches_basic(ray_start_regular_shared, tmp_path):
# Test input validation
ds = ray.data.range(5)
Expand Down
16 changes: 16 additions & 0 deletions python/ray/data/tests/test_dataset_pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -627,6 +627,22 @@ def test_randomize_block_order_each_window(ray_start_regular_shared):
assert pipe.take() == [0, 1, 4, 5, 2, 3, 6, 7, 10, 11, 8, 9]


def test_add_column(ray_start_regular_shared):
heyitsmui marked this conversation as resolved.
Show resolved Hide resolved
df = pd.DataFrame({"col1": [1, 2, 3]})
ds = ray.data.from_pandas(df)
pipe = ds.repeat()
assert pipe.add_column("col2", lambda x: x["col1"] + 1).take(1) == [
{"col1": 1, "col2": 2}
]


def test_select_columns(ray_start_regular_shared):
df = pd.DataFrame({"col1": [1, 2, 3], "col2": [2, 3, 4], "col3": [3, 4, 5]})
ds = ray.data.from_pandas(df)
pipe = ds.repeat()
assert pipe.select_columns(["col2", "col3"]).take(1) == [{"col2": 2, "col3": 3}]


def test_drop_columns(ray_start_regular_shared):
df = pd.DataFrame({"col1": [1, 2, 3], "col2": [2, 3, 4], "col3": [3, 4, 5]})
ds = ray.data.from_pandas(df)
Expand Down