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

[data] Implement Dataset.distinct #36655

Merged
merged 10 commits into from
Jun 23, 2023
Merged
Show file tree
Hide file tree
Changes from 4 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
27 changes: 27 additions & 0 deletions python/ray/data/dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -1719,6 +1719,33 @@ def groupby(self, key: Optional[str]) -> "GroupedData":

return GroupedData(self, key)

def distinct(self) -> "Dataset":
"""Remove duplicate rows from the :class:`~ray.data.Dataset`.

Examples:
>>> import ray
>>> ds = ray.data.from_items([1, 2, 3, 2, 3])
>>> ds.distinct().take_all()
[{'item': 1}, {'item': 2}, {'item': 3}]

Time complexity: O(dataset size * log(dataset size / parallelism))

.. note:: Currently distinct only supports :class:`~ray.data.Dataset`s with one single column.

Returns:
A new :class:`~ray.data.Dataset` with distinct rows.
"""
columns = self.columns(fetch_if_missing=True)
assert columns is not None
if len(columns) > 1:
# TODO(hchen): Remove this limitation once groupby supports
# multiple columns.
raise NotImplementedError(
"`distinct` currently only suports Datasets with one single column, "
"please apply `select_columns` before `distinct`."
)
return self.groupby(columns[0]).count().drop_columns(["count()"])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we add a TODO to implement an aggregate function for distinct, so we don't need to calculate count?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

instead of drop_columns(["count()"]), can we call select_columns(columns[0]), so we don't rely on the implicit naming of count()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we add a TODO to implement an aggregate function for distinct, so we don't need to calculate count?

I considered this initially. but considering count is already very cheap, this is probably no big benefit to implement a standalone distinct function.


@ConsumptionAPI
def aggregate(self, *aggs: AggregateFn) -> Union[Any, Dict[str, Any]]:
"""Aggregate the entire dataset as one group.
Expand Down
18 changes: 18 additions & 0 deletions python/ray/data/tests/test_all_to_all.py
Original file line number Diff line number Diff line change
Expand Up @@ -217,6 +217,24 @@ def test_repartition_shuffle_arrow(ray_start_regular_shared):
assert large._block_num_rows() == [500] * 20


def test_distinct(ray_start_regular_shared):
ds = ray.data.from_items([3, 2, 3, 1, 2, 3])
assert ds.distinct().sort("item").take_all() == [
{"item": 1},
{"item": 2},
{"item": 3},
]
ds = ray.data.from_items([
{"a": 1, "b": 1},
{"a": 1, "b": 2},
])
# Currently, we don't support distinct on multiple columns.
with pytest.raises(NotImplementedError):
ds.distinct().take_all()
# After selecting a single column, distinct should work.
assert ds.select_columns(["a"]).distinct().take_all() == [{"a": 1}]


def test_grouped_dataset_repr(ray_start_regular_shared):
ds = ray.data.from_items([{"key": "spam"}, {"key": "ham"}, {"key": "spam"}])
assert repr(ds.groupby("key")) == f"GroupedData(dataset={ds!r}, key='key')"
Expand Down