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 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
1 change: 0 additions & 1 deletion python/ray/data/dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -2191,7 +2191,6 @@ def to_numpy_refs(self, *, column: Optional[str] = None
Returns:
A list of remote NumPy ndarrays created from this dataset.
"""

block_to_ndarray = cached_remote_fn(_block_to_ndarray)
return [
block_to_ndarray.remote(block, column=column)
Expand Down
21 changes: 18 additions & 3 deletions python/ray/data/extensions/tensor_extension.py
Original file line number Diff line number Diff line change
Expand Up @@ -1254,7 +1254,9 @@ def from_numpy(cls, arr):
else:
raise ValueError("Must give ndarray or iterable of ndarrays.")

def _to_numpy(self, index: Optional[int] = None):
def _to_numpy(self,
index: Optional[int] = None,
zero_copy_only: bool = False):
"""
Helper for getting either an element of the array of tensors as an
ndarray, or the entire array of tensors as a single ndarray.
Expand All @@ -1263,6 +1265,11 @@ def _to_numpy(self, index: Optional[int] = None):
index: The index of the tensor element that we wish to return as
an ndarray. If not given, the entire array of tensors is
returned as an ndarray.
zero_copy_only: If True, an exception will be raised if the
conversion to a NumPy array would require copying the
underlying data (e.g. in presence of nulls, or for
non-primitive types). This argument is currently ignored, so
zero-copy isn't enforced even if this argument is true.

Returns:
The corresponding tensor element as an ndarray if an index was
Expand Down Expand Up @@ -1295,15 +1302,23 @@ def _to_numpy(self, index: Optional[int] = None):
else:
# Getting the entire array of tensors.
shape = (len(self), ) + shape
# TODO(Clark): Enforce zero_copy_only.
# TODO(Clark): Support strides?
return np.ndarray(
shape, dtype=ext_dtype, buffer=data_buffer, offset=offset)

def to_numpy(self):
def to_numpy(self, zero_copy_only: bool = True):
"""
Convert the entire array of tensors into a single ndarray.

Args:
zero_copy_only: If True, an exception will be raised if the
conversion to a NumPy array would require copying the
underlying data (e.g. in presence of nulls, or for
non-primitive types). This argument is currently ignored, so
zero-copy isn't enforced even if this argument is true.

Returns:
A single ndarray representing the entire array of tensors.
"""
return self._to_numpy()
return self._to_numpy(zero_copy_only=zero_copy_only)
15 changes: 9 additions & 6 deletions python/ray/data/impl/arrow_block.py
Original file line number Diff line number Diff line change
Expand Up @@ -224,20 +224,23 @@ def to_pandas(self) -> "pandas.DataFrame":
return self._table.to_pandas()

def to_numpy(self, column: str = None) -> np.ndarray:
if not column:
if column is None:
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))
f"Cannot find column {column}, available columns: "
f"{self._table.column_names}")
array = self._table[column]
if array.num_chunks > 1:
# TODO(ekl) combine fails since we can't concat ArrowTensorType?
# 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()
else:
assert array.num_chunks == 1, array
array = array.chunk(0)
return array.to_numpy(zero_copy_only=False)

def to_arrow(self) -> "pyarrow.Table":
return self._table
Expand Down
19 changes: 14 additions & 5 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 All @@ -40,6 +41,10 @@ def sample_boundaries(blocks: List[ObjectRef[Block]], key: SortKeyT,
Return (num_reducers - 1) items in ascending order from the blocks that
partition the domain into ranges with approximately equally many elements.
"""
# TODO(Clark): Support multiple boundary sampling keys.
if isinstance(key, list) and len(key) > 1:
raise ValueError("Multiple boundary sampling keys not supported.")

n_samples = int(num_reducers * 10 / len(blocks))

sample_block = cached_remote_fn(_sample_block)
Expand All @@ -56,11 +61,16 @@ 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()
column = key[0][0] if isinstance(key, list) else None
sample_items = BlockAccessor.for_block(samples).to_numpy(column)
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 Expand Up @@ -109,8 +119,7 @@ def sort_impl(blocks: BlockList, key: SortKeyT,
return BlockList(blocks, metadata)


def _sample_block(block: Block[T], n_samples: int,
key: SortKeyT) -> np.ndarray:
def _sample_block(block: Block[T], n_samples: int, key: SortKeyT) -> Block[T]:
return BlockAccessor.for_block(block).sample(n_samples, key)


Expand Down
48 changes: 26 additions & 22 deletions python/ray/data/tests/test_dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -1234,7 +1234,12 @@ def test_to_pandas_refs(ray_start_regular_shared):
assert df.equals(dfds)


def test_to_numpy(ray_start_regular_shared):
def test_to_numpy_refs(ray_start_regular_shared):
# Simple Dataset
ds = ray.data.range(10)
arr = np.concatenate(ray.get(ds.to_numpy_refs()))
np.testing.assert_equal(arr, np.arange(0, 10))

# Tensor Dataset
ds = ray.data.range_tensor(10, parallelism=2)
arr = np.concatenate(ray.get(ds.to_numpy_refs(column="value")))
Expand All @@ -1245,10 +1250,9 @@ def test_to_numpy(ray_start_regular_shared):
arr = np.concatenate(ray.get(ds.to_numpy_refs(column="value")))
np.testing.assert_equal(arr, np.arange(0, 10))

# Simple Dataset
ds = ray.data.range(10)
arr = np.concatenate(ray.get(ds.to_numpy_refs()))
np.testing.assert_equal(arr, np.arange(0, 10))
# Table Dataset requires column
with pytest.raises(ValueError):
ray.get(ds.to_numpy_refs())


def test_to_arrow_refs(ray_start_regular_shared):
Expand Down Expand Up @@ -2931,7 +2935,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 +2963,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 +2981,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 +3005,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 +3029,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 +3053,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 +3078,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 +3115,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 +3196,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 +3292,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 +3307,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 +3325,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 +3343,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 +3362,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 +3381,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 +3424,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 +3450,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