This is an automated email from the ASF dual-hosted git repository.
jorisvandenbossche pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/main by this push:
new a485bb7a7e GH-34787: [Python] Accept zero_copy_only=False for
ChunkedArray.to_numpy (#35582)
a485bb7a7e is described below
commit a485bb7a7e948b2089b935fe7d5ab098a0e615de
Author: Julien Jerphanion <[email protected]>
AuthorDate: Mon Jul 10 10:36:08 2023 +0200
GH-34787: [Python] Accept zero_copy_only=False for ChunkedArray.to_numpy
(#35582)
### Rationale for this change
Addresses GH-34787.
### What changes are included in this PR?
I think this proposes the minimal viable changes to have the same signature
as proposed in GH-34787.
### Are these changes tested?
Yes, for now, a single simple test has been introduced to test the correct
behavior of `ChunkedArray.to_numpy` with respect to the addition of
`zero_copy_only`.
* Closes: #34787
Lead-authored-by: Julien Jerphanion <[email protected]>
Co-authored-by: Joris Van den Bossche <[email protected]>
Signed-off-by: Joris Van den Bossche <[email protected]>
---
python/pyarrow/table.pxi | 12 +++++++++++-
python/pyarrow/tests/test_array.py | 15 +++++++++++++++
2 files changed, 26 insertions(+), 1 deletion(-)
diff --git a/python/pyarrow/table.pxi b/python/pyarrow/table.pxi
index 238fdb86bc..f08162089b 100644
--- a/python/pyarrow/table.pxi
+++ b/python/pyarrow/table.pxi
@@ -460,10 +460,16 @@ cdef class ChunkedArray(_PandasConvertible):
def _to_pandas(self, options, types_mapper=None, **kwargs):
return _array_like_to_pandas(self, options, types_mapper=types_mapper)
- def to_numpy(self):
+ def to_numpy(self, zero_copy_only=False):
"""
Return a NumPy copy of this array (experimental).
+ Parameters
+ ----------
+ zero_copy_only : bool, default False
+ Introduced for signature consistence with pyarrow.Array.to_numpy.
+ This must be False here since NumPy arrays' buffer must be
contiguous.
+
Returns
-------
array : numpy.ndarray
@@ -475,6 +481,10 @@ cdef class ChunkedArray(_PandasConvertible):
>>> n_legs.to_numpy()
array([ 2, 2, 4, 4, 5, 100])
"""
+ if zero_copy_only:
+ raise ValueError(
+ "zero_copy_only must be False for
pyarrow.ChunkedArray.to_numpy"
+ )
cdef:
PyObject* out
PandasOptions c_options
diff --git a/python/pyarrow/tests/test_array.py
b/python/pyarrow/tests/test_array.py
index ed29bf5cae..77eaca3dd1 100644
--- a/python/pyarrow/tests/test_array.py
+++ b/python/pyarrow/tests/test_array.py
@@ -179,6 +179,21 @@ def test_to_numpy_zero_copy():
np.testing.assert_array_equal(np_arr, expected)
+def test_chunked_array_to_numpy_zero_copy():
+ elements = [[2, 2, 4], [4, 5, 100]]
+
+ chunked_arr = pa.chunked_array(elements)
+
+ msg = "zero_copy_only must be False for pyarrow.ChunkedArray.to_numpy"
+
+ with pytest.raises(ValueError, match=msg):
+ chunked_arr.to_numpy(zero_copy_only=True)
+
+ np_arr = chunked_arr.to_numpy()
+ expected = [2, 2, 4, 4, 5, 100]
+ np.testing.assert_array_equal(np_arr, expected)
+
+
def test_to_numpy_unsupported_types():
# ARROW-2871: Some primitive types are not yet supported in to_numpy
bool_arr = pa.array([True, False, True])