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 5 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
41 changes: 41 additions & 0 deletions python/ray/data/dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -602,6 +602,47 @@ def drop_columns(
lambda batch: batch.drop(columns=cols), compute=compute, **ray_remote_args
)

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

Columns passed in will be de-duped since ArrowBlock and PandasBlock
`select` does not explicitly handle duplicated columns.
heyitsmui marked this conversation as resolved.
Show resolved Hide resolved

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(["col1", "col2"])
>>> ds
Dataset(num_blocks=10, num_rows=10, schema={col1: int64, col2: int64})


Time complexity: O(dataset size / parallelism)

Args:
columns: Names of the columns to select. Columns not included in this
will be filtered out.
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).
"""
# dedup the input columns used for selection
unique_columns = list(set(columns))
heyitsmui marked this conversation as resolved.
Show resolved Hide resolved
return self.map_batches(
lambda batch: BlockAccessor.for_block(batch).select(columns=unique_columns),
compute=compute,
**ray_remote_args,
)

def flat_map(
self,
fn: RowUDF[T, U],
Expand Down
22 changes: 22 additions & 0 deletions python/ray/data/tests/test_dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -2224,6 +2224,28 @@ 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
df = pd.DataFrame({"col1": [1, 2, 3], "col2": [2, 3, 4], "col3": [3, 4, 5]})
# Test pandas and base cases
ds = ray.data.from_pandas(df)
assert ds._dataset_format() == "pandas"
assert ds.select_columns(columns=["col1", "col2", "col3"]).take(1) == [
{"col1": 1, "col2": 2, "col3": 3}
]
assert ds.select_columns(columns=["col1", "col2"]).take(1) == [
{"col1": 1, "col2": 2}
]
assert ds.select_columns(columns=[]).take(1) == [{}]
assert ds.select_columns(columns=["col1", "col2", "col2"]).take(1) == [
{"col1": 1, "col2": 2}
]

# Test arrow
ds = ds.select_columns(columns=["col1", "col2"], batch_format="pyarrow")
heyitsmui marked this conversation as resolved.
Show resolved Hide resolved
assert ds._dataset_format() == "arrow"
assert ds.take(1) == [{"col1": 1, "col2": 2}]

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