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] Fix boundary sampling concatenation. #20784

Merged
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
26 changes: 12 additions & 14 deletions python/ray/data/impl/arrow_block.py
Original file line number Diff line number Diff line change
Expand Up @@ -224,20 +224,18 @@ def to_pandas(self) -> "pandas.DataFrame":
return self._table.to_pandas()

def to_numpy(self, column: str = None) -> np.ndarray:
if not column:
raise ValueError(
"`column` must be specified when calling .to_numpy() "
"on Arrow blocks.")
if column not in self._table.column_names:
raise ValueError(
"Cannot find column {}, available columns: {}".format(
column, self._table.column_names))
array = self._table[column]
if array.num_chunks > 1:
# TODO(ekl) combine fails since we can't concat ArrowTensorType?
array = array.combine_chunks()
assert array.num_chunks == 1, array
return self._table[column].chunk(0).to_numpy()
if column is None:
columns = self._table.column_names
clarkzinzow marked this conversation as resolved.
Show resolved Hide resolved
else:
if column not in self._table.column_names:
raise ValueError(
"Cannot find column {}, available columns: {}".format(
column, self._table.column_names))
columns = [column]
arrays = [self._table[col].to_numpy() for col in columns]
clarkzinzow marked this conversation as resolved.
Show resolved Hide resolved
if len(arrays) == 1:
return arrays[0]
return np.column_stack(arrays)

def to_arrow(self) -> "pyarrow.Table":
return self._table
Expand Down
11 changes: 8 additions & 3 deletions python/ray/data/impl/sort.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import ray
from ray.types import ObjectRef
from ray.data.block import Block, BlockAccessor
from ray.data.impl.arrow_block import DelegatingArrowBlockBuilder
from ray.data.impl.block_list import BlockList
from ray.data.impl.progress_bar import ProgressBar
from ray.data.impl.remote_fn import cached_remote_fn
Expand Down Expand Up @@ -56,11 +57,15 @@ def sample_boundaries(blocks: List[ObjectRef[Block]], key: SortKeyT,
# The dataset is empty
if len(samples) == 0:
return [None] * (num_reducers - 1)
sample_items = np.concatenate(samples)
sample_items.sort()
builder = DelegatingArrowBlockBuilder()
for sample in samples:
builder.add_block(sample)
samples = builder.build()
sample_items = BlockAccessor.for_block(samples).to_numpy()
sample_items = np.sort(sample_items)
ret = [
np.quantile(sample_items, q, interpolation="nearest")
for q in np.arange(0, 1, 1 / num_reducers)
for q in np.linspace(0, 1, num_reducers)
]
return ret[1:]

Expand Down
34 changes: 17 additions & 17 deletions python/ray/data/tests/test_dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -2931,7 +2931,7 @@ def test_groupby_arrow(ray_start_regular_shared):
assert agg_ds.count() == 0


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_agg_name_conflict(ray_start_regular_shared, num_parts):
# Test aggregation name conflict.
xs = list(range(100))
Expand Down Expand Up @@ -2959,7 +2959,7 @@ def test_groupby_agg_name_conflict(ray_start_regular_shared, num_parts):
{"A": 2, "foo": 50.0, "foo_2": 50.0}]


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_count(ray_start_regular_shared, num_parts):
# Test built-in count aggregation
seed = int(time.time())
Expand All @@ -2977,7 +2977,7 @@ def test_groupby_arrow_count(ray_start_regular_shared, num_parts):
{"A": 2, "count()": 33}]


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_sum(ray_start_regular_shared, num_parts):
# Test built-in sum aggregation
seed = int(time.time())
Expand All @@ -3001,7 +3001,7 @@ def test_groupby_arrow_sum(ray_start_regular_shared, num_parts):
"value") == 0


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_min(ray_start_regular_shared, num_parts):
# Test built-in min aggregation
seed = int(time.time())
Expand All @@ -3025,7 +3025,7 @@ def test_groupby_arrow_min(ray_start_regular_shared, num_parts):
ray.data.range_arrow(10).filter(lambda r: r["value"] > 10).min("value")


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_max(ray_start_regular_shared, num_parts):
# Test built-in max aggregation
seed = int(time.time())
Expand All @@ -3049,7 +3049,7 @@ def test_groupby_arrow_max(ray_start_regular_shared, num_parts):
ray.data.range_arrow(10).filter(lambda r: r["value"] > 10).max("value")


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_mean(ray_start_regular_shared, num_parts):
# Test built-in mean aggregation
seed = int(time.time())
Expand All @@ -3074,7 +3074,7 @@ def test_groupby_arrow_mean(ray_start_regular_shared, num_parts):
"value")


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_std(ray_start_regular_shared, num_parts):
# Test built-in std aggregation
seed = int(time.time())
Expand Down Expand Up @@ -3111,7 +3111,7 @@ def test_groupby_arrow_std(ray_start_regular_shared, num_parts):
assert ray.data.from_pandas(pd.DataFrame({"A": [3]})).std("A") == 0


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_multicolumn(ray_start_regular_shared, num_parts):
# Test built-in mean aggregation on multiple columns
seed = int(time.time())
Expand Down Expand Up @@ -3192,7 +3192,7 @@ def test_groupby_agg_bad_on(ray_start_regular_shared):
ray.data.from_items(xs).mean("A")


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_arrow_multi_agg(ray_start_regular_shared, num_parts):
seed = int(time.time())
print(f"Seeding RNG for test_groupby_arrow_multi_agg with: {seed}")
Expand Down Expand Up @@ -3288,7 +3288,7 @@ def test_groupby_simple(ray_start_regular_shared):
assert agg_ds.count() == 0


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_count(ray_start_regular_shared, num_parts):
# Test built-in count aggregation
seed = int(time.time())
Expand All @@ -3303,7 +3303,7 @@ def test_groupby_simple_count(ray_start_regular_shared, num_parts):
33)]


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_sum(ray_start_regular_shared, num_parts):
# Test built-in sum aggregation
seed = int(time.time())
Expand All @@ -3321,7 +3321,7 @@ def test_groupby_simple_sum(ray_start_regular_shared, num_parts):
assert ray.data.range(10).filter(lambda r: r > 10).sum() == 0


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_min(ray_start_regular_shared, num_parts):
# Test built-in min aggregation
seed = int(time.time())
Expand All @@ -3339,7 +3339,7 @@ def test_groupby_simple_min(ray_start_regular_shared, num_parts):
ray.data.range(10).filter(lambda r: r > 10).min()


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_max(ray_start_regular_shared, num_parts):
# Test built-in max aggregation
seed = int(time.time())
Expand All @@ -3358,7 +3358,7 @@ def test_groupby_simple_max(ray_start_regular_shared, num_parts):
ray.data.range(10).filter(lambda r: r > 10).max()


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_mean(ray_start_regular_shared, num_parts):
# Test built-in mean aggregation
seed = int(time.time())
Expand All @@ -3377,7 +3377,7 @@ def test_groupby_simple_mean(ray_start_regular_shared, num_parts):
ray.data.range(10).filter(lambda r: r > 10).mean()


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_std(ray_start_regular_shared, num_parts):
# Test built-in std aggregation
seed = int(time.time())
Expand Down Expand Up @@ -3420,7 +3420,7 @@ def test_groupby_simple_std(ray_start_regular_shared, num_parts):
assert ray.data.from_items([3]).std() == 0


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_multilambda(ray_start_regular_shared, num_parts):
# Test built-in mean aggregation
seed = int(time.time())
Expand All @@ -3446,7 +3446,7 @@ def test_groupby_simple_multilambda(ray_start_regular_shared, num_parts):
.mean([lambda x: x[0], lambda x: x[1]])


@pytest.mark.parametrize("num_parts", [1, 10, 100])
@pytest.mark.parametrize("num_parts", [1, 15, 100])
def test_groupby_simple_multi_agg(ray_start_regular_shared, num_parts):
seed = int(time.time())
print(f"Seeding RNG for test_groupby_simple_multi_agg with: {seed}")
Expand Down