paleolimbot commented on code in PR #39985:
URL: https://github.com/apache/arrow/pull/39985#discussion_r1488342175
##########
python/pyarrow/table.pxi:
##########
@@ -1327,6 +1327,68 @@ cdef class ChunkedArray(_PandasConvertible):
result += self.chunk(i).to_pylist()
return result
+ def __arrow_c_stream__(self, requested_schema=None):
+ """
+ Export to a C ArrowArrayStream PyCapsule.
+
+ Parameters
+ ----------
+ requested_schema : PyCapsule, default None
+ The schema to which the stream should be casted, passed as a
+ PyCapsule containing a C ArrowSchema representation of the
+ requested schema.
+
+ Returns
+ -------
+ PyCapsule
+ A capsule containing a C ArrowArrayStream struct.
+ """
+ cdef:
+ ArrowArrayStream* c_stream = NULL
+ ChunkedArray chunked = self
+
+ if requested_schema is not None:
+ out_type = DataType._import_from_c_capsule(requested_schema)
+ if self.type != out_type:
+ chunked = self.cast(out_type)
Review Comment:
Is this as much of a concern as with a `Table`? I can't think of any clean
way to do this lazily on a per-chunk basis, although I'm happy to remove the
feature if it's that bad of an idea.
A quick check suggests casting up front is faster...perhaps because it
obliterates the chunks:
```python
import pyarrow as pa
import numpy as np
n = int(1e6)
n_chunks = 1000
per_chunk = n // n_chunks
chunks = [np.random.random(per_chunk) for i in range(n_chunks)]
chunked = pa.chunked_array(chunks)
def roundtrip_chunked():
stream_capsule = chunked.__arrow_c_stream__()
chunked._import_from_c_capsule(stream_capsule)
%timeit roundtrip_chunked()
#> 3.72 ms ± 15.8 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
def roundtrip_chunked_cast():
stream_capsule = chunked.cast(pa.float()).__arrow_c_stream__()
chunked._import_from_c_capsule(stream_capsule)
%timeit roundtrip_chunked_cast()
#> 1.52 ms ± 1.79 µs per loop (mean ± std. dev. of 7 runs, 1,000 loops each)
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]