Skip to content

Conversation

@pimdh
Copy link
Contributor

@pimdh pimdh commented Apr 3, 2025

Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow extension arrays. This is needed to use Ray in code bases that use extension arrays.

The serialization already seemed sufficiently general, but as far as I can tell, the deserialization can not be done in generality. Hence, this setup allows registration of custom deserializers for extension types.

For serialization, the selector has been changed from ExtensionType to BaseExtensionType to accommodate for non-Python ExtensionArrays, like pyarrow.FixedShapeTensorArray.

This is at the moment a proof-of-concept. If you like the idea, I suppose the registration function may need to move to a better place, and docs need adding.

The implementation now works without registration on any extension type.

Related issue number

Closes #51959

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Note

Generalizes Arrow array (de)serialization to any pyarrow.BaseExtensionType, removing tensor-specific handling and adding tests for fixed/variable-shape tensors.

  • Arrow (De)serialization:
    • Switch from tensor-specific checks to generic pyarrow.BaseExtensionType handling.
    • Reconstruct extension arrays via type.wrap_array(storage); serialize via storage payload wrapped with extension metadata.
    • Remove ray.air.util.tensor_extensions.arrow dependencies and special-casing.
  • Tests:
    • Add roundtrip tests for pa.FixedShapeTensorArray and a custom variable-shape ExtensionType.
    • Import PicklableArrayPayload in tests for constructing payloads.

Written by Cursor Bugbot for commit 4bbcdbe. This will update automatically on new commits. Configure here.

@pimdh pimdh requested a review from a team as a code owner April 3, 2025 20:53
@pimdh pimdh changed the title [Core] Deserialization of PyArrow Extension Arrays [Core] Deserialization of PyArrow Extension Arrays by registration of deserializers Apr 3, 2025
@cszhu cszhu added data Ray Data-related issues arrow labels Apr 3, 2025
@hainesmichaelc hainesmichaelc added the community-contribution Contributed by the community label Apr 4, 2025
@cszhu cszhu removed the arrow label Apr 11, 2025
Copy link
Contributor

Choose a reason for hiding this comment

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

hmm, would we need to expose this as a user API?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think so, if we'd want to allow a user to use their own ExtensionTypes. If the public api should live elsewhere/look different, please let me know and I'll change it.

@gvspraveen gvspraveen added core Issues that should be addressed in Ray Core and removed data Ray Data-related issues labels Apr 22, 2025
@israbbani
Copy link
Contributor

@pimdh thank you for your contribution. We have an API for registering custom serializers and deserializers (I think it's what is used internally to register the arrow serializer/deserializer too).

Can that be used to add serializers and deserializers for PyArrow Extension Arrays? See Example 2 in https://docs.ray.io/en/latest/ray-core/objects/serialization.html#customized-serialization for more details.

@pimdh
Copy link
Contributor Author

pimdh commented Apr 24, 2025

Hi,
As I understand it, no that doesn't work, because this needs a modification of the built-in Arrow Table serializer. Perhaps one could replace that entire serializer with a custom one, but that isn't particularly user-friendly.
Thanks!

@israbbani
Copy link
Contributor

@pimdh I see your point. I'm adding @alexeykudinkin from the Ray Data team to shepherd this.

Copy link
Contributor

@alexeykudinkin alexeykudinkin left a comment

Choose a reason for hiding this comment

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

Thank you for your contribution @pimdh!

Can you please help me understand what kind of extensions you folks are using?

Copy link
Contributor

Choose a reason for hiding this comment

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

Let's actually lift this support to happen outside of the extension mechanism -- Ray should just support FixedShapeTensorArray not requiring any extensions for it

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm using the built-in FixedShapeTensorArray, and a custom variable shape tensor array, which seems equivalent to the Ray one (and the one in the PR in the arrow repo).
I'm reading and writing with Lance, so the array type affects the storage format. Hence I'm not so keen to migrate all my code to Ray's internal extension type.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Another representative example is this one for encoding bfloat16 arrays: https://github.com/lancedb/lance/blob/main/python/python/lance/_arrow/bf16.py

Copy link
Contributor

Choose a reason for hiding this comment

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

Is there a particular reason you're adding support for FixedShapeTensorType?

It should just work out of the box.

Copy link
Contributor

Choose a reason for hiding this comment

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

I've just tested it and it works out of the box:

import ray
import numpy as np
import pyarrow as pa
from pyarrow import FixedShapeTensorType


@ray.remote
def _gen_tensor():
    shape = (16, 16, 8)  # Shape per tensor
    num_tensors = 128  # Number of tensors
    tensors = [np.ones(shape, dtype=np.int8).flatten() for _ in range(num_tensors)]

    # Create array of fixed shape tensors
    return pa.array(tensors, type=pa.fixed_shape_tensor(pa.int8(), shape=shape))


tensors = ray.get(_gen_tensor.remote())

print(tensors)
print(tensors.type)

@israbbani israbbani added the data Ray Data-related issues label Apr 26, 2025
@pimdh
Copy link
Contributor Author

pimdh commented Jun 5, 2025

Anyone able to help me land this? Thanks!

@github-actions
Copy link

This pull request has been automatically marked as stale because it has not had
any activity for 14 days. It will be closed in another 14 days if no further activity occurs.
Thank you for your contributions.

You can always ask for help on our discussion forum or Ray's public slack channel.

If you'd like to keep this open, just leave any comment, and the stale label will be removed.

@github-actions github-actions bot added the stale The issue is stale. It will be closed within 7 days unless there are further conversation label Jun 20, 2025
Copy link
Contributor

@iamjustinhsu iamjustinhsu left a comment

Choose a reason for hiding this comment

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

Hi @pimdh, looks fine to me. I'll await @alexeykudinkin response since he's the expert on this

elif pa.types.is_map(a.type):
return _map_array_to_array_payload(a)
elif isinstance(a.type, tensor_extension_types):
return _tensor_array_to_array_payload(a)
Copy link
Contributor

Choose a reason for hiding this comment

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

_tensor_array_to_array_payload is this still being used?

Copy link
Contributor Author

@pimdh pimdh Sep 25, 2025

Choose a reason for hiding this comment

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

No it isn't, thanks. Removed

@alexeykudinkin
Copy link
Contributor

Thanks for making the changes @pimdh!

And special thanks to @mdekstrand for chiming in and getting everyone aligned!

@alexeykudinkin alexeykudinkin enabled auto-merge (squash) September 26, 2025 00:25
@pimdh
Copy link
Contributor Author

pimdh commented Sep 26, 2025

It looks like the CI failed due to an unrelated reason. @alexeykudinkin , would you mind restarting CI? Thanks

@edoakes edoakes removed the core Issues that should be addressed in Ray Core label Sep 26, 2025
@github-actions github-actions bot disabled auto-merge September 26, 2025 20:25
@alexeykudinkin alexeykudinkin enabled auto-merge (squash) September 26, 2025 20:25
Copy link
Contributor

@Sparks0219 Sparks0219 left a comment

Choose a reason for hiding this comment

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

LGTM

…tests

Signed-off-by: Pim de Haan <pim@cusp.ai>
Signed-off-by: Pim de Haan <pim@cusp.ai>
Signed-off-by: Pim de Haan <pim@cusp.ai>
@github-actions github-actions bot disabled auto-merge September 26, 2025 23:43
@alexeykudinkin alexeykudinkin merged commit ab045a4 into ray-project:master Sep 27, 2025
6 checks passed
dstrodtman pushed a commit to dstrodtman/ray that referenced this pull request Oct 6, 2025
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow
extension arrays. This is needed to use Ray in code bases that use
extension arrays.

~The serialization already seemed sufficiently general, but as far as I
can tell, the deserialization can not be done in generality. Hence, this
setup allows registration of custom deserializers for extension types.~

~For serialization, the selector has been changed from `ExtensionType`
to `BaseExtensionType` to accommodate for non-Python ExtensionArrays,
like `pyarrow.FixedShapeTensorArray`.~

~This is at the moment a proof-of-concept. If you like the idea, I
suppose the registration function may need to move to a better place,
and docs need adding.~

The implementation now works without registration on any extension type.

## Related issue number

Closes ray-project#51959

## Checks

- [X] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [X] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [X] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [X] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

<!-- CURSOR_SUMMARY -->
---

> [!NOTE]
> Generalizes Arrow array (de)serialization to any
`pyarrow.BaseExtensionType`, removing tensor-specific handling and
adding tests for fixed/variable-shape tensors.
>
> - **Arrow (De)serialization**:
> - Switch from tensor-specific checks to generic
`pyarrow.BaseExtensionType` handling.
> - Reconstruct extension arrays via `type.wrap_array(storage)`;
serialize via storage payload wrapped with extension metadata.
> - Remove `ray.air.util.tensor_extensions.arrow` dependencies and
special-casing.
> - **Tests**:
> - Add roundtrip tests for `pa.FixedShapeTensorArray` and a custom
variable-shape `ExtensionType`.
>   - Import `PicklableArrayPayload` in tests for constructing payloads.
>
> <sup>Written by [Cursor
Bugbot](https://cursor.com/dashboard?tab=bugbot) for commit
4bbcdbe. This will update automatically
on new commits. Configure
[here](https://cursor.com/dashboard?tab=bugbot).</sup>
<!-- /CURSOR_SUMMARY -->

---------

Signed-off-by: Pim de Haan <pim@cusp.ai>
Signed-off-by: Douglas Strodtman <douglas@anyscale.com>
joshkodi pushed a commit to joshkodi/ray that referenced this pull request Oct 13, 2025
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow
extension arrays. This is needed to use Ray in code bases that use
extension arrays.

~The serialization already seemed sufficiently general, but as far as I
can tell, the deserialization can not be done in generality. Hence, this
setup allows registration of custom deserializers for extension types.~

~For serialization, the selector has been changed from `ExtensionType`
to `BaseExtensionType` to accommodate for non-Python ExtensionArrays,
like `pyarrow.FixedShapeTensorArray`.~

~This is at the moment a proof-of-concept. If you like the idea, I
suppose the registration function may need to move to a better place,
and docs need adding.~

The implementation now works without registration on any extension type.

## Related issue number

Closes ray-project#51959

## Checks

- [X] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [X] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [X] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [X] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

<!-- CURSOR_SUMMARY -->
---

> [!NOTE]
> Generalizes Arrow array (de)serialization to any
`pyarrow.BaseExtensionType`, removing tensor-specific handling and
adding tests for fixed/variable-shape tensors.
>
> - **Arrow (De)serialization**:
> - Switch from tensor-specific checks to generic
`pyarrow.BaseExtensionType` handling.
> - Reconstruct extension arrays via `type.wrap_array(storage)`;
serialize via storage payload wrapped with extension metadata.
> - Remove `ray.air.util.tensor_extensions.arrow` dependencies and
special-casing.
> - **Tests**:
> - Add roundtrip tests for `pa.FixedShapeTensorArray` and a custom
variable-shape `ExtensionType`.
>   - Import `PicklableArrayPayload` in tests for constructing payloads.
>
> <sup>Written by [Cursor
Bugbot](https://cursor.com/dashboard?tab=bugbot) for commit
4bbcdbe. This will update automatically
on new commits. Configure
[here](https://cursor.com/dashboard?tab=bugbot).</sup>
<!-- /CURSOR_SUMMARY -->

---------

Signed-off-by: Pim de Haan <pim@cusp.ai>
Signed-off-by: Josh Kodi <joshkodi@gmail.com>
justinyeh1995 pushed a commit to justinyeh1995/ray that referenced this pull request Oct 20, 2025
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow
extension arrays. This is needed to use Ray in code bases that use
extension arrays.

~The serialization already seemed sufficiently general, but as far as I
can tell, the deserialization can not be done in generality. Hence, this
setup allows registration of custom deserializers for extension types.~

~For serialization, the selector has been changed from `ExtensionType`
to `BaseExtensionType` to accommodate for non-Python ExtensionArrays,
like `pyarrow.FixedShapeTensorArray`.~

~This is at the moment a proof-of-concept. If you like the idea, I
suppose the registration function may need to move to a better place,
and docs need adding.~

The implementation now works without registration on any extension type.

## Related issue number

Closes ray-project#51959

## Checks

- [X] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [X] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [X] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [X] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

<!-- CURSOR_SUMMARY -->
---

> [!NOTE]
> Generalizes Arrow array (de)serialization to any
`pyarrow.BaseExtensionType`, removing tensor-specific handling and
adding tests for fixed/variable-shape tensors.
> 
> - **Arrow (De)serialization**:
> - Switch from tensor-specific checks to generic
`pyarrow.BaseExtensionType` handling.
> - Reconstruct extension arrays via `type.wrap_array(storage)`;
serialize via storage payload wrapped with extension metadata.
> - Remove `ray.air.util.tensor_extensions.arrow` dependencies and
special-casing.
> - **Tests**:
> - Add roundtrip tests for `pa.FixedShapeTensorArray` and a custom
variable-shape `ExtensionType`.
>   - Import `PicklableArrayPayload` in tests for constructing payloads.
> 
> <sup>Written by [Cursor
Bugbot](https://cursor.com/dashboard?tab=bugbot) for commit
4bbcdbe. This will update automatically
on new commits. Configure
[here](https://cursor.com/dashboard?tab=bugbot).</sup>
<!-- /CURSOR_SUMMARY -->

---------

Signed-off-by: Pim de Haan <pim@cusp.ai>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow
extension arrays. This is needed to use Ray in code bases that use
extension arrays.

~The serialization already seemed sufficiently general, but as far as I
can tell, the deserialization can not be done in generality. Hence, this
setup allows registration of custom deserializers for extension types.~

~For serialization, the selector has been changed from `ExtensionType`
to `BaseExtensionType` to accommodate for non-Python ExtensionArrays,
like `pyarrow.FixedShapeTensorArray`.~

~This is at the moment a proof-of-concept. If you like the idea, I
suppose the registration function may need to move to a better place,
and docs need adding.~

The implementation now works without registration on any extension type.

## Related issue number

Closes ray-project#51959

## Checks

- [X] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [X] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [X] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [X] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

<!-- CURSOR_SUMMARY -->
---

> [!NOTE]
> Generalizes Arrow array (de)serialization to any
`pyarrow.BaseExtensionType`, removing tensor-specific handling and
adding tests for fixed/variable-shape tensors.
> 
> - **Arrow (De)serialization**:
> - Switch from tensor-specific checks to generic
`pyarrow.BaseExtensionType` handling.
> - Reconstruct extension arrays via `type.wrap_array(storage)`;
serialize via storage payload wrapped with extension metadata.
> - Remove `ray.air.util.tensor_extensions.arrow` dependencies and
special-casing.
> - **Tests**:
> - Add roundtrip tests for `pa.FixedShapeTensorArray` and a custom
variable-shape `ExtensionType`.
>   - Import `PicklableArrayPayload` in tests for constructing payloads.
> 
> <sup>Written by [Cursor
Bugbot](https://cursor.com/dashboard?tab=bugbot) for commit
4bbcdbe. This will update automatically
on new commits. Configure
[here](https://cursor.com/dashboard?tab=bugbot).</sup>
<!-- /CURSOR_SUMMARY -->

---------

Signed-off-by: Pim de Haan <pim@cusp.ai>
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow
extension arrays. This is needed to use Ray in code bases that use
extension arrays.

~The serialization already seemed sufficiently general, but as far as I
can tell, the deserialization can not be done in generality. Hence, this
setup allows registration of custom deserializers for extension types.~

~For serialization, the selector has been changed from `ExtensionType`
to `BaseExtensionType` to accommodate for non-Python ExtensionArrays,
like `pyarrow.FixedShapeTensorArray`.~

~This is at the moment a proof-of-concept. If you like the idea, I
suppose the registration function may need to move to a better place,
and docs need adding.~

The implementation now works without registration on any extension type.

## Related issue number

Closes ray-project#51959

## Checks

- [X] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [X] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [X] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [X] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

<!-- CURSOR_SUMMARY -->
---

> [!NOTE]
> Generalizes Arrow array (de)serialization to any
`pyarrow.BaseExtensionType`, removing tensor-specific handling and
adding tests for fixed/variable-shape tensors.
>
> - **Arrow (De)serialization**:
> - Switch from tensor-specific checks to generic
`pyarrow.BaseExtensionType` handling.
> - Reconstruct extension arrays via `type.wrap_array(storage)`;
serialize via storage payload wrapped with extension metadata.
> - Remove `ray.air.util.tensor_extensions.arrow` dependencies and
special-casing.
> - **Tests**:
> - Add roundtrip tests for `pa.FixedShapeTensorArray` and a custom
variable-shape `ExtensionType`.
>   - Import `PicklableArrayPayload` in tests for constructing payloads.
>
> <sup>Written by [Cursor
Bugbot](https://cursor.com/dashboard?tab=bugbot) for commit
4bbcdbe. This will update automatically
on new commits. Configure
[here](https://cursor.com/dashboard?tab=bugbot).</sup>
<!-- /CURSOR_SUMMARY -->

---------

Signed-off-by: Pim de Haan <pim@cusp.ai>
Signed-off-by: Aydin Abiar <aydin@anyscale.com>
Future-Outlier pushed a commit to Future-Outlier/ray that referenced this pull request Dec 7, 2025
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This feature adds the ability to (de)serialize arbitrary PyArrow
extension arrays. This is needed to use Ray in code bases that use
extension arrays.

~The serialization already seemed sufficiently general, but as far as I
can tell, the deserialization can not be done in generality. Hence, this
setup allows registration of custom deserializers for extension types.~

~For serialization, the selector has been changed from `ExtensionType`
to `BaseExtensionType` to accommodate for non-Python ExtensionArrays,
like `pyarrow.FixedShapeTensorArray`.~

~This is at the moment a proof-of-concept. If you like the idea, I
suppose the registration function may need to move to a better place,
and docs need adding.~

The implementation now works without registration on any extension type.

## Related issue number

Closes ray-project#51959

## Checks

- [X] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [X] I've run `scripts/format.sh` to lint the changes in this PR.
- [ ] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [ ] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [X] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [X] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

<!-- CURSOR_SUMMARY -->
---

> [!NOTE]
> Generalizes Arrow array (de)serialization to any
`pyarrow.BaseExtensionType`, removing tensor-specific handling and
adding tests for fixed/variable-shape tensors.
>
> - **Arrow (De)serialization**:
> - Switch from tensor-specific checks to generic
`pyarrow.BaseExtensionType` handling.
> - Reconstruct extension arrays via `type.wrap_array(storage)`;
serialize via storage payload wrapped with extension metadata.
> - Remove `ray.air.util.tensor_extensions.arrow` dependencies and
special-casing.
> - **Tests**:
> - Add roundtrip tests for `pa.FixedShapeTensorArray` and a custom
variable-shape `ExtensionType`.
>   - Import `PicklableArrayPayload` in tests for constructing payloads.
>
> <sup>Written by [Cursor
Bugbot](https://cursor.com/dashboard?tab=bugbot) for commit
4bbcdbe. This will update automatically
on new commits. Configure
[here](https://cursor.com/dashboard?tab=bugbot).</sup>
<!-- /CURSOR_SUMMARY -->

---------

Signed-off-by: Pim de Haan <pim@cusp.ai>
Signed-off-by: Future-Outlier <eric901201@gmail.com>
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 @external-author-action-required Alternate tag for PRs where the author doesn't have labeling permission. go add ONLY when ready to merge, run all tests unstale A PR that has been marked unstale. It will not get marked stale again if this label is on it.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Data] Support general Arrow ExtensionTypes