Skip to content

Conversation

@chriso
Copy link
Contributor

@chriso chriso commented Oct 17, 2025

Description

Ray data can't serialize zero (byte) length numpy arrays:

import numpy as np
import ray.data

array = np.empty((2, 0), dtype=np.int8)

ds = ray.data.from_items([{"array": array}])

for batch in ds.iter_batches(batch_size=1):
     print(batch)

What I expect to see:

{'array': array([], shape=(1, 2, 0), dtype=int8)}

What I see:

/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py:736: RuntimeWarning: invalid value encountered in scalar divide
  offsets = np.arange(
2025-10-17 17:18:09,499 WARNING arrow.py:189 -- Failed to convert column 'array' into pyarrow array due to: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []; falling back to serialize as pickled python objects
Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 672, in from_numpy
    return cls._from_numpy(arr)
           ^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 736, in _from_numpy
    offsets = np.arange(
              ^^^^^^^^^^
ValueError: arange: cannot compute length

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 141, in convert_to_pyarrow_array
    return ArrowTensorArray.from_numpy(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 678, in from_numpy
    raise ArrowConversionError(data_str) from e
ray.air.util.tensor_extensions.arrow.ArrowConversionError: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []
2025-10-17 17:18:09,789 INFO logging.py:293 -- Registered dataset logger for dataset dataset_0_0
2025-10-17 17:18:09,815 WARNING resource_manager.py:134 -- ⚠️  Ray's object store is configured to use only 33.5% of available memory (2.0GiB out of 6.0GiB total). For optimal Ray Data performance, we recommend setting the object store to at least 50% of available memory. You can do this by setting the 'object_store_memory' parameter when calling ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable.
{'array': array([array([], shape=(2, 0), dtype=int8)], dtype=object)}

This PR fixes the issue so that zero-length arrays are serialized correctly, and the shape and dtype is preserved.

Additional information

This is ray==2.50.0.

@chriso chriso requested review from a team as code owners October 17, 2025 21:23
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request correctly addresses the serialization issue with zero-length numpy arrays by adding a special case for when num_items_per_element is zero. The fix is clean and directly solves the ValueError from np.arange. The addition of a new test case ensures this scenario is covered going forward. I've suggested a small improvement to the new test to make it more robust by parameterizing it with various zero-length array shapes.

chriso and others added 2 commits October 17, 2025 17:26
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Signed-off-by: Chris O'Hara <[email protected]>
Signed-off-by: Chris O'Hara <[email protected]>
@chriso chriso changed the title Support serializing zero-length numpy arrays [Data] Support serializing zero-length numpy arrays Oct 17, 2025
@ray-gardener ray-gardener bot added data Ray Data-related issues community-contribution Contributed by the community labels Oct 18, 2025
@alexeykudinkin alexeykudinkin added the go add ONLY when ready to merge, run all tests label Oct 20, 2025
@alexeykudinkin alexeykudinkin enabled auto-merge (squash) October 20, 2025 20:39
@alexeykudinkin alexeykudinkin merged commit cf4e04c into ray-project:master Oct 20, 2025
8 checks passed
kamil-kaczmarek pushed a commit that referenced this pull request Oct 20, 2025
## Description

Ray data can't serialize zero (byte) length numpy arrays:

```python3
import numpy as np
import ray.data

array = np.empty((2, 0), dtype=np.int8)

ds = ray.data.from_items([{"array": array}])

for batch in ds.iter_batches(batch_size=1):
     print(batch)
```

What I expect to see:

```
{'array': array([], shape=(1, 2, 0), dtype=int8)}
```

What I see:

```
/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py:736: RuntimeWarning: invalid value encountered in scalar divide
  offsets = np.arange(
2025-10-17 17:18:09,499 WARNING arrow.py:189 -- Failed to convert column 'array' into pyarrow array due to: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []; falling back to serialize as pickled python objects
Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 672, in from_numpy
    return cls._from_numpy(arr)
           ^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 736, in _from_numpy
    offsets = np.arange(
              ^^^^^^^^^^
ValueError: arange: cannot compute length

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 141, in convert_to_pyarrow_array
    return ArrowTensorArray.from_numpy(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 678, in from_numpy
    raise ArrowConversionError(data_str) from e
ray.air.util.tensor_extensions.arrow.ArrowConversionError: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []
2025-10-17 17:18:09,789 INFO logging.py:293 -- Registered dataset logger for dataset dataset_0_0
2025-10-17 17:18:09,815 WARNING resource_manager.py:134 -- ⚠️  Ray's object store is configured to use only 33.5% of available memory (2.0GiB out of 6.0GiB total). For optimal Ray Data performance, we recommend setting the object store to at least 50% of available memory. You can do this by setting the 'object_store_memory' parameter when calling ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable.
{'array': array([array([], shape=(2, 0), dtype=int8)], dtype=object)}
```

This PR fixes the issue so that zero-length arrays are serialized
correctly, and the shape and dtype is preserved.

## Additional information

This is `ray==2.50.0`.

---------

Signed-off-by: Chris O'Hara <[email protected]>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Signed-off-by: Kamil Kaczmarek <[email protected]>
xinyuangui2 pushed a commit to xinyuangui2/ray that referenced this pull request Oct 22, 2025
## Description

Ray data can't serialize zero (byte) length numpy arrays:

```python3
import numpy as np
import ray.data

array = np.empty((2, 0), dtype=np.int8)

ds = ray.data.from_items([{"array": array}])

for batch in ds.iter_batches(batch_size=1):
     print(batch)
```

What I expect to see:

```
{'array': array([], shape=(1, 2, 0), dtype=int8)}
```

What I see:

```
/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py:736: RuntimeWarning: invalid value encountered in scalar divide
  offsets = np.arange(
2025-10-17 17:18:09,499 WARNING arrow.py:189 -- Failed to convert column 'array' into pyarrow array due to: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []; falling back to serialize as pickled python objects
Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 672, in from_numpy
    return cls._from_numpy(arr)
           ^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 736, in _from_numpy
    offsets = np.arange(
              ^^^^^^^^^^
ValueError: arange: cannot compute length

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 141, in convert_to_pyarrow_array
    return ArrowTensorArray.from_numpy(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 678, in from_numpy
    raise ArrowConversionError(data_str) from e
ray.air.util.tensor_extensions.arrow.ArrowConversionError: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []
2025-10-17 17:18:09,789 INFO logging.py:293 -- Registered dataset logger for dataset dataset_0_0
2025-10-17 17:18:09,815 WARNING resource_manager.py:134 -- ⚠️  Ray's object store is configured to use only 33.5% of available memory (2.0GiB out of 6.0GiB total). For optimal Ray Data performance, we recommend setting the object store to at least 50% of available memory. You can do this by setting the 'object_store_memory' parameter when calling ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable.
{'array': array([array([], shape=(2, 0), dtype=int8)], dtype=object)}
```

This PR fixes the issue so that zero-length arrays are serialized
correctly, and the shape and dtype is preserved.

## Additional information

This is `ray==2.50.0`.

---------

Signed-off-by: Chris O'Hara <[email protected]>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Signed-off-by: xgui <[email protected]>
elliot-barn pushed a commit that referenced this pull request Oct 23, 2025
## Description

Ray data can't serialize zero (byte) length numpy arrays:

```python3
import numpy as np
import ray.data

array = np.empty((2, 0), dtype=np.int8)

ds = ray.data.from_items([{"array": array}])

for batch in ds.iter_batches(batch_size=1):
     print(batch)
```

What I expect to see:

```
{'array': array([], shape=(1, 2, 0), dtype=int8)}
```

What I see:

```
/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py:736: RuntimeWarning: invalid value encountered in scalar divide
  offsets = np.arange(
2025-10-17 17:18:09,499 WARNING arrow.py:189 -- Failed to convert column 'array' into pyarrow array due to: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []; falling back to serialize as pickled python objects
Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 672, in from_numpy
    return cls._from_numpy(arr)
           ^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 736, in _from_numpy
    offsets = np.arange(
              ^^^^^^^^^^
ValueError: arange: cannot compute length

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 141, in convert_to_pyarrow_array
    return ArrowTensorArray.from_numpy(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 678, in from_numpy
    raise ArrowConversionError(data_str) from e
ray.air.util.tensor_extensions.arrow.ArrowConversionError: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []
2025-10-17 17:18:09,789 INFO logging.py:293 -- Registered dataset logger for dataset dataset_0_0
2025-10-17 17:18:09,815 WARNING resource_manager.py:134 -- ⚠️  Ray's object store is configured to use only 33.5% of available memory (2.0GiB out of 6.0GiB total). For optimal Ray Data performance, we recommend setting the object store to at least 50% of available memory. You can do this by setting the 'object_store_memory' parameter when calling ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable.
{'array': array([array([], shape=(2, 0), dtype=int8)], dtype=object)}
```

This PR fixes the issue so that zero-length arrays are serialized
correctly, and the shape and dtype is preserved.

## Additional information

This is `ray==2.50.0`.

---------

Signed-off-by: Chris O'Hara <[email protected]>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Signed-off-by: elliot-barn <[email protected]>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
## Description

Ray data can't serialize zero (byte) length numpy arrays:

```python3
import numpy as np
import ray.data

array = np.empty((2, 0), dtype=np.int8)

ds = ray.data.from_items([{"array": array}])

for batch in ds.iter_batches(batch_size=1):
     print(batch)
```

What I expect to see:

```
{'array': array([], shape=(1, 2, 0), dtype=int8)}
```

What I see:

```
/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py:736: RuntimeWarning: invalid value encountered in scalar divide
  offsets = np.arange(
2025-10-17 17:18:09,499 WARNING arrow.py:189 -- Failed to convert column 'array' into pyarrow array due to: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []; falling back to serialize as pickled python objects
Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 672, in from_numpy
    return cls._from_numpy(arr)
           ^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 736, in _from_numpy
    offsets = np.arange(
              ^^^^^^^^^^
ValueError: arange: cannot compute length

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 141, in convert_to_pyarrow_array
    return ArrowTensorArray.from_numpy(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 678, in from_numpy
    raise ArrowConversionError(data_str) from e
ray.air.util.tensor_extensions.arrow.ArrowConversionError: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []
2025-10-17 17:18:09,789 INFO logging.py:293 -- Registered dataset logger for dataset dataset_0_0
2025-10-17 17:18:09,815 WARNING resource_manager.py:134 -- ⚠️  Ray's object store is configured to use only 33.5% of available memory (2.0GiB out of 6.0GiB total). For optimal Ray Data performance, we recommend setting the object store to at least 50% of available memory. You can do this by setting the 'object_store_memory' parameter when calling ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable.
{'array': array([array([], shape=(2, 0), dtype=int8)], dtype=object)}
```

This PR fixes the issue so that zero-length arrays are serialized
correctly, and the shape and dtype is preserved.

## Additional information

This is `ray==2.50.0`.

---------

Signed-off-by: Chris O'Hara <[email protected]>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
## Description

Ray data can't serialize zero (byte) length numpy arrays:

```python3
import numpy as np
import ray.data

array = np.empty((2, 0), dtype=np.int8)

ds = ray.data.from_items([{"array": array}])

for batch in ds.iter_batches(batch_size=1):
     print(batch)
```

What I expect to see:

```
{'array': array([], shape=(1, 2, 0), dtype=int8)}
```

What I see:

```
/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py:736: RuntimeWarning: invalid value encountered in scalar divide
  offsets = np.arange(
2025-10-17 17:18:09,499 WARNING arrow.py:189 -- Failed to convert column 'array' into pyarrow array due to: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []; falling back to serialize as pickled python objects
Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 672, in from_numpy
    return cls._from_numpy(arr)
           ^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 736, in _from_numpy
    offsets = np.arange(
              ^^^^^^^^^^
ValueError: arange: cannot compute length

The above exception was the direct cause of the following exception:

Traceback (most recent call last):
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 141, in convert_to_pyarrow_array
    return ArrowTensorArray.from_numpy(
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/chris.ohara/Downloads/.venv/lib/python3.12/site-packages/ray/air/util/tensor_extensions/arrow.py", line 678, in from_numpy
    raise ArrowConversionError(data_str) from e
ray.air.util.tensor_extensions.arrow.ArrowConversionError: Error converting data to Arrow: column: 'array', shape: (1, 2, 0), dtype: int8, data: []
2025-10-17 17:18:09,789 INFO logging.py:293 -- Registered dataset logger for dataset dataset_0_0
2025-10-17 17:18:09,815 WARNING resource_manager.py:134 -- ⚠️  Ray's object store is configured to use only 33.5% of available memory (2.0GiB out of 6.0GiB total). For optimal Ray Data performance, we recommend setting the object store to at least 50% of available memory. You can do this by setting the 'object_store_memory' parameter when calling ray.init() or by setting the RAY_DEFAULT_OBJECT_STORE_MEMORY_PROPORTION environment variable.
{'array': array([array([], shape=(2, 0), dtype=int8)], dtype=object)}
```

This PR fixes the issue so that zero-length arrays are serialized
correctly, and the shape and dtype is preserved.

## Additional information

This is `ray==2.50.0`.

---------

Signed-off-by: Chris O'Hara <[email protected]>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Signed-off-by: Aydin Abiar <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-contribution Contributed by the community data Ray Data-related issues go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants