|
27 | 27 | from ray.util.annotations import DeveloperAPI, PublicAPI
|
28 | 28 |
|
29 | 29 | PYARROW_VERSION = get_pyarrow_version()
|
30 |
| -# Minimum version of Arrow that supports ExtensionScalars. |
31 |
| -# TODO(Clark): Remove conditional definition once we only support Arrow 8.0.0+. |
32 |
| -MIN_PYARROW_VERSION_SCALAR = parse_version("8.0.0") |
33 | 30 | # Minimum version of Arrow that supports subclassable ExtensionScalars.
|
34 | 31 | # TODO(Clark): Remove conditional definition once we only support Arrow 9.0.0+.
|
35 | 32 | MIN_PYARROW_VERSION_SCALAR_SUBCLASS = parse_version("9.0.0")
|
@@ -59,17 +56,6 @@ def __init__(self, data_str: str):
|
59 | 56 | super().__init__(message)
|
60 | 57 |
|
61 | 58 |
|
62 |
| -def _arrow_supports_extension_scalars(): |
63 |
| - """ |
64 |
| - Whether Arrow ExtensionScalars are supported in the current pyarrow version. |
65 |
| -
|
66 |
| - This returns True if the pyarrow version is 8.0.0+, or if the pyarrow version is |
67 |
| - unknown. |
68 |
| - """ |
69 |
| - # TODO(Clark): Remove utility once we only support Arrow 8.0.0+. |
70 |
| - return PYARROW_VERSION is None or PYARROW_VERSION >= MIN_PYARROW_VERSION_SCALAR |
71 |
| - |
72 |
| - |
73 | 59 | def _arrow_extension_scalars_are_subclassable():
|
74 | 60 | """
|
75 | 61 | Whether Arrow ExtensionScalars support subclassing in the current pyarrow version.
|
@@ -489,20 +475,16 @@ def __arrow_ext_scalar_class__(self):
|
489 | 475 | """
|
490 | 476 | return ArrowTensorScalar
|
491 | 477 |
|
492 |
| - if _arrow_supports_extension_scalars(): |
493 |
| - # TODO(Clark): Remove this version guard once we only support Arrow 8.0.0+. |
494 |
| - def _extension_scalar_to_ndarray( |
495 |
| - self, scalar: pa.ExtensionScalar |
496 |
| - ) -> np.ndarray: |
497 |
| - """ |
498 |
| - Convert an ExtensionScalar to a tensor element. |
499 |
| - """ |
500 |
| - raw_values = scalar.value.values |
501 |
| - shape = scalar.type.shape |
502 |
| - value_type = raw_values.type |
503 |
| - offset = raw_values.offset |
504 |
| - data_buffer = raw_values.buffers()[1] |
505 |
| - return _to_ndarray_helper(shape, value_type, offset, data_buffer) |
| 478 | + def _extension_scalar_to_ndarray(self, scalar: "pa.ExtensionScalar") -> np.ndarray: |
| 479 | + """ |
| 480 | + Convert an ExtensionScalar to a tensor element. |
| 481 | + """ |
| 482 | + raw_values = scalar.value.values |
| 483 | + shape = scalar.type.shape |
| 484 | + value_type = raw_values.type |
| 485 | + offset = raw_values.offset |
| 486 | + data_buffer = raw_values.buffers()[1] |
| 487 | + return _to_ndarray_helper(shape, value_type, offset, data_buffer) |
506 | 488 |
|
507 | 489 | def __str__(self) -> str:
|
508 | 490 | return (
|
@@ -657,42 +639,20 @@ def to_pylist(self):
|
657 | 639 | # support (see comment in __getitem__).
|
658 | 640 | return list(self)
|
659 | 641 |
|
660 |
| - if _arrow_supports_extension_scalars(): |
661 |
| - # NOTE(Clark): This __getitem__ override is only needed for Arrow 8.*, |
662 |
| - # before ExtensionScalar subclassing support was added. |
663 |
| - # TODO(Clark): Remove these methods once we only support Arrow 9.0.0+. |
664 |
| - def __getitem__(self, key): |
665 |
| - # This __getitem__ hook allows us to support proper indexing when |
666 |
| - # accessing a single tensor (a "scalar" item of the array). Without this |
667 |
| - # hook for integer keys, the indexing will fail on pyarrow < 9.0.0 due |
668 |
| - # to a lack of ExtensionScalar subclassing support. |
669 |
| - |
670 |
| - # NOTE(Clark): We'd like to override the pa.Array.getitem() helper |
671 |
| - # instead, which would obviate the need for overriding __iter__(), but |
672 |
| - # unfortunately overriding Cython cdef methods with normal Python |
673 |
| - # methods isn't allowed. |
674 |
| - item = super().__getitem__(key) |
675 |
| - if not isinstance(key, slice): |
676 |
| - item = item.type._extension_scalar_to_ndarray(item) |
677 |
| - return item |
| 642 | + def __getitem__(self, key): |
| 643 | + # This __getitem__ hook allows us to support proper indexing when |
| 644 | + # accessing a single tensor (a "scalar" item of the array). Without this |
| 645 | + # hook for integer keys, the indexing will fail on pyarrow < 9.0.0 due |
| 646 | + # to a lack of ExtensionScalar subclassing support. |
678 | 647 |
|
679 |
| - else: |
680 |
| - # NOTE(Clark): This __getitem__ override is only needed for Arrow < 8.0.0, |
681 |
| - # before any ExtensionScalar support was added. |
682 |
| - # TODO(Clark): Remove these methods once we only support Arrow 8.0.0+. |
683 |
| - def __getitem__(self, key): |
684 |
| - # This __getitem__ hook allows us to support proper indexing when |
685 |
| - # accessing a single tensor (a "scalar" item of the array). Without this |
686 |
| - # hook for integer keys, the indexing will fail on pyarrow < 8.0.0 due |
687 |
| - # to a lack of ExtensionScalar support. |
688 |
| - |
689 |
| - # NOTE(Clark): We'd like to override the pa.Array.getitem() helper |
690 |
| - # instead, which would obviate the need for overriding __iter__(), but |
691 |
| - # unfortunately overriding Cython cdef methods with normal Python |
692 |
| - # methods isn't allowed. |
693 |
| - if isinstance(key, slice): |
694 |
| - return super().__getitem__(key) |
695 |
| - return self._to_numpy(key) |
| 648 | + # NOTE(Clark): We'd like to override the pa.Array.getitem() helper |
| 649 | + # instead, which would obviate the need for overriding __iter__(), but |
| 650 | + # unfortunately overriding Cython cdef methods with normal Python |
| 651 | + # methods isn't allowed. |
| 652 | + item = super().__getitem__(key) |
| 653 | + if not isinstance(key, slice): |
| 654 | + item = item.type._extension_scalar_to_ndarray(item) |
| 655 | + return item |
696 | 656 |
|
697 | 657 |
|
698 | 658 | # NOTE: We need to inherit from the mixin before pa.ExtensionArray to ensure that the
|
@@ -1109,22 +1069,18 @@ def __str__(self) -> str:
|
1109 | 1069 | def __repr__(self) -> str:
|
1110 | 1070 | return str(self)
|
1111 | 1071 |
|
1112 |
| - if _arrow_supports_extension_scalars(): |
1113 |
| - # TODO(Clark): Remove this version guard once we only support Arrow 8.0.0+. |
1114 |
| - def _extension_scalar_to_ndarray( |
1115 |
| - self, scalar: pa.ExtensionScalar |
1116 |
| - ) -> np.ndarray: |
1117 |
| - """ |
1118 |
| - Convert an ExtensionScalar to a tensor element. |
1119 |
| - """ |
1120 |
| - data = scalar.value.get("data") |
1121 |
| - raw_values = data.values |
1122 |
| - |
1123 |
| - shape = tuple(scalar.value.get("shape").as_py()) |
1124 |
| - value_type = raw_values.type |
1125 |
| - offset = raw_values.offset |
1126 |
| - data_buffer = raw_values.buffers()[1] |
1127 |
| - return _to_ndarray_helper(shape, value_type, offset, data_buffer) |
| 1072 | + def _extension_scalar_to_ndarray(self, scalar: "pa.ExtensionScalar") -> np.ndarray: |
| 1073 | + """ |
| 1074 | + Convert an ExtensionScalar to a tensor element. |
| 1075 | + """ |
| 1076 | + data = scalar.value.get("data") |
| 1077 | + raw_values = data.values |
| 1078 | + |
| 1079 | + shape = tuple(scalar.value.get("shape").as_py()) |
| 1080 | + value_type = raw_values.type |
| 1081 | + offset = raw_values.offset |
| 1082 | + data_buffer = raw_values.buffers()[1] |
| 1083 | + return _to_ndarray_helper(shape, value_type, offset, data_buffer) |
1128 | 1084 |
|
1129 | 1085 |
|
1130 | 1086 | # NOTE: We need to inherit from the mixin before pa.ExtensionArray to ensure that the
|
|
0 commit comments