-
Notifications
You must be signed in to change notification settings - Fork 128
Add Arrow C streaming, DataFrame iteration, and OOM-safe streaming execution #1222
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from 10 commits
91ccd1e
f78e90b
e322521
31e8ed1
0130a72
03e530c
4a3f17d
f7a2407
b1d18a8
eeb2a37
748b7e2
5e650aa
ebd2191
6bae74b
f0cbe06
295d04a
475c031
06c9fc7
94432b5
31ed8e7
610aed3
1ebd3c1
2e4b963
d0ee865
16a249c
d91ecfa
21f286a
831f56f
7b5e461
d6e8132
8a250a4
7789322
07a8169
9e27cc6
9dc3fb2
d3c68cc
33f9024
7553b32
b6909a5
f4e76ea
b66b441
2794c88
17c4c2c
0ff4c0d
f450e1d
5dc5cfa
fd08dc4
9baa49e
78f6c8a
5a53633
ccc8633
98ac3a1
759fb86
57d4162
d66d496
d76a509
7433234
848665e
13ebaf9
dae501d
c36aa9a
914f17e
d120b2e
e7a7ec9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -145,10 +145,39 @@ To materialize the results of your DataFrame operations: | |
|
||
# Display results | ||
df.show() # Print tabular format to console | ||
|
||
# Count rows | ||
count = df.count() | ||
|
||
PyArrow Streaming | ||
----------------- | ||
|
||
DataFusion DataFrames implement the ``__arrow_c_stream__`` protocol, enabling | ||
zero-copy streaming into libraries like `PyArrow <https://arrow.apache.org/>`_. | ||
Earlier versions eagerly converted the entire DataFrame when exporting to | ||
PyArrow, which could exhaust memory on large datasets. With streaming, batches | ||
are produced lazily so you can process arbitrarily large results without | ||
out-of-memory errors. | ||
|
||
.. code-block:: python | ||
|
||
import pyarrow as pa | ||
|
||
# Create a PyArrow RecordBatchReader without materializing all batches | ||
reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) | ||
kosiew marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
for batch in reader: | ||
... # process each batch as it is produced | ||
|
||
DataFrames are also iterable, yielding :class:`pyarrow.RecordBatch` objects | ||
lazily so you can loop over results directly: | ||
|
||
.. code-block:: python | ||
|
||
for batch in df: | ||
... # process each batch as it is produced | ||
|
||
|
||
See :doc:`../io/arrow` for additional details on the Arrow interface. | ||
|
||
HTML Rendering | ||
-------------- | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,6 +26,7 @@ | |
TYPE_CHECKING, | ||
Any, | ||
Iterable, | ||
Iterator, | ||
Literal, | ||
Optional, | ||
Union, | ||
|
@@ -289,6 +290,9 @@ def __init__( | |
class DataFrame: | ||
"""Two dimensional table representation of data. | ||
|
||
DataFrame objects are iterable; iterating over a DataFrame yields | ||
:class:`pyarrow.RecordBatch` instances lazily. | ||
|
||
See :ref:`user_guide_concepts` in the online documentation for more information. | ||
""" | ||
|
||
|
@@ -1098,21 +1102,42 @@ def unnest_columns(self, *columns: str, preserve_nulls: bool = True) -> DataFram | |
return DataFrame(self.df.unnest_columns(columns, preserve_nulls=preserve_nulls)) | ||
|
||
def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: | ||
"""Export an Arrow PyCapsule Stream. | ||
"""Export the DataFrame as an Arrow C Stream. | ||
|
||
This will execute and collect the DataFrame. We will attempt to respect the | ||
requested schema, but only trivial transformations will be applied such as only | ||
returning the fields listed in the requested schema if their data types match | ||
those in the DataFrame. | ||
The DataFrame is executed using DataFusion's streaming APIs and exposed via | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Might be good to have a link somewhere in the docstring to https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added |
||
Arrow's C Stream interface. Record batches are produced incrementally, so the | ||
full result set is never materialized in memory. When ``requested_schema`` is | ||
provided, only straightforward projections such as column selection or | ||
reordering are applied. | ||
|
||
Args: | ||
requested_schema: Attempt to provide the DataFrame using this schema. | ||
|
||
Returns: | ||
Arrow PyCapsule object. | ||
Arrow PyCapsule object representing an ``ArrowArrayStream``. | ||
""" | ||
# ``DataFrame.__arrow_c_stream__`` in the Rust extension leverages | ||
# ``execute_stream_partitioned`` under the hood to stream batches while | ||
# preserving the original partition order. | ||
return self.df.__arrow_c_stream__(requested_schema) | ||
|
||
def __iter__(self) -> Iterator[pa.RecordBatch]: | ||
|
||
"""Yield record batches from the DataFrame without materializing results. | ||
|
||
This implementation streams record batches via the Arrow C Stream | ||
interface, allowing callers such as :func:`pyarrow.Table.from_batches` to | ||
consume results lazily. The DataFrame is executed using DataFusion's | ||
partitioned streaming APIs so ``collect`` is never invoked and batch | ||
order across partitions is preserved. | ||
""" | ||
from contextlib import closing | ||
|
||
import pyarrow as pa | ||
|
||
reader = pa.RecordBatchReader._import_from_c_capsule(self.__arrow_c_stream__()) | ||
with closing(reader): | ||
yield from reader | ||
|
||
def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: | ||
"""Apply a function to the current DataFrame which returns another DataFrame. | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The heading here is specific to PyArrow, but I think it would be good to provide a distinction here. Maybe "Zero-copy Streaming to other Arrow implementations"? Or something like that?
Then we can have a sub-section dedicated to pyarrow, but also explain that it works with any arrow-based python impl.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point!
Implemented your suggestion.