From 91ccd1ed4496c2d20435e28b2734cbc32f2279f2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 1 Sep 2025 11:28:04 +0800 Subject: [PATCH 01/67] feat: add streaming utilities, range support, and improve async handling in DataFrame - Add `range` method to SessionContext and iterator support to DataFrame - Introduce `spawn_stream` utility and refactor async execution for better signal handling - Add tests for `KeyboardInterrupt` in `__arrow_c_stream__` and incremental DataFrame streaming - Improve memory usage tracking in tests with psutil - Update DataFrame docs with PyArrow streaming section and enhance `__arrow_c_stream__` documentation - Replace Tokio runtime creation with `spawn_stream` in PySessionContext - Bump datafusion packages to 49.0.1 and update dependencies - Remove unused imports and restore main Cargo.toml --- docs/source/user-guide/dataframe/index.rst | 23 +++- examples/datafusion-ffi-example/Cargo.toml | 4 +- python/datafusion/context.py | 31 +++++ python/datafusion/dataframe.py | 29 ++++- python/tests/test_dataframe.py | 127 +++++++++++++++++++++ python/tests/test_io.py | 28 +++++ src/context.rs | 12 +- src/dataframe.rs | 94 ++++++++++----- src/utils.rs | 37 +++++- 9 files changed, 337 insertions(+), 48 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index f69485af7..00fe70567 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -145,10 +145,31 @@ 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 `_. +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(df.__arrow_c_stream__()) + for batch in reader: + ... # process each batch as it is produced + +See :doc:`../io/arrow` for additional details on the Arrow interface. + HTML Rendering -------------- diff --git a/examples/datafusion-ffi-example/Cargo.toml b/examples/datafusion-ffi-example/Cargo.toml index 647f6c51e..cd72ffbc3 100644 --- a/examples/datafusion-ffi-example/Cargo.toml +++ b/examples/datafusion-ffi-example/Cargo.toml @@ -21,8 +21,8 @@ version = "0.2.0" edition = "2021" [dependencies] -datafusion = { version = "49.0.2" } -datafusion-ffi = { version = "49.0.2" } +datafusion = { version = "49.0.1" } +datafusion-ffi = { version = "49.0.1" } pyo3 = { version = "0.23", features = ["extension-module", "abi3", "abi3-py39"] } arrow = { version = "55.0.0" } arrow-array = { version = "55.0.0" } diff --git a/python/datafusion/context.py b/python/datafusion/context.py index bce51d644..b0bd7c1f5 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -731,6 +731,37 @@ def from_polars(self, data: pl.DataFrame, name: str | None = None) -> DataFrame: """ return DataFrame(self.ctx.from_polars(data, name)) + def range( + self, + start: int, + stop: int | None = None, + step: int = 1, + partitions: int | None = None, + ) -> DataFrame: + """Create a DataFrame containing a sequence of numbers. + + This is backed by DataFusion's ``range`` table function, which generates + values lazily and therefore does not materialize the full range in + memory. When ``stop`` is omitted, ``start`` is treated as the stop value + and the sequence begins at zero. + + Args: + start: Starting value for the sequence or the exclusive stop if + ``stop`` is ``None``. + stop: Exclusive upper bound of the sequence. + step: Increment between successive values. + partitions: Optional number of partitions for the generated data. + + Returns: + DataFrame yielding the requested range of values. + """ + if stop is None: + start, stop = 0, start + + parts = f", {int(partitions)}" if partitions is not None else "" + sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" # noqa: S608 + return self.sql(sql) + # https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 # is the discussion on how we arrived at adding register_view def register_view(self, name: str, df: DataFrame) -> None: diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 61cb09438..8ac983451 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -26,6 +26,7 @@ TYPE_CHECKING, Any, Iterable, + Iterator, Literal, Optional, Union, @@ -1098,21 +1099,37 @@ 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 + 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`` under the hood to stream batches one at a time. 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 + streaming APIs so ``collect`` is never invoked. + """ + import pyarrow as pa + + reader = pa.RecordBatchReader._import_from_c(self.__arrow_c_stream__()) + yield from reader + def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 0cd56219a..17a804f2c 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1582,6 +1582,29 @@ def test_empty_to_arrow_table(df): assert set(pyarrow_table.column_names) == {"a", "b", "c"} +def test_arrow_c_stream_to_table(monkeypatch): + ctx = SessionContext() + + # Create a DataFrame with two separate record batches + batch1 = pa.record_batch([pa.array([1])], names=["a"]) + batch2 = pa.record_batch([pa.array([2])], names=["a"]) + df = ctx.create_dataframe([[batch1], [batch2]]) + + # Fail if the DataFrame is pre-collected + def fail_collect(self): # pragma: no cover - failure path + msg = "collect should not be called" + raise AssertionError(msg) + + monkeypatch.setattr(DataFrame, "collect", fail_collect) + + table = pa.Table.from_batches(df) + expected = pa.Table.from_batches([batch1, batch2]) + + assert table.equals(expected) + assert table.schema == df.schema() + assert table.column("a").num_chunks == 2 + + def test_to_pylist(df): # Convert datafusion dataframe to Python list pylist = df.to_pylist() @@ -2666,6 +2689,110 @@ def trigger_interrupt(): interrupt_thread.join(timeout=1.0) +def test_arrow_c_stream_interrupted(): + """__arrow_c_stream__ responds to ``KeyboardInterrupt`` signals. + + Similar to ``test_collect_interrupted`` this test issues a long running + query, but consumes the results via ``__arrow_c_stream__``. It then raises + ``KeyboardInterrupt`` in the main thread and verifies that the stream + iteration stops promptly with the appropriate exception. + """ + + ctx = SessionContext() + + batches = [] + for i in range(10): + batch = pa.RecordBatch.from_arrays( + [ + pa.array(list(range(i * 1000, (i + 1) * 1000))), + pa.array([f"value_{j}" for j in range(i * 1000, (i + 1) * 1000)]), + ], + names=["a", "b"], + ) + batches.append(batch) + + ctx.register_record_batches("t1", [batches]) + ctx.register_record_batches("t2", [batches]) + + df = ctx.sql( + """ + WITH t1_expanded AS ( + SELECT + a, + b, + CAST(a AS DOUBLE) / 1.5 AS c, + CAST(a AS DOUBLE) * CAST(a AS DOUBLE) AS d + FROM t1 + CROSS JOIN (SELECT 1 AS dummy FROM t1 LIMIT 5) + ), + t2_expanded AS ( + SELECT + a, + b, + CAST(a AS DOUBLE) * 2.5 AS e, + CAST(a AS DOUBLE) * CAST(a AS DOUBLE) * CAST(a AS DOUBLE) AS f + FROM t2 + CROSS JOIN (SELECT 1 AS dummy FROM t2 LIMIT 5) + ) + SELECT + t1.a, t1.b, t1.c, t1.d, + t2.a AS a2, t2.b AS b2, t2.e, t2.f + FROM t1_expanded t1 + JOIN t2_expanded t2 ON t1.a % 100 = t2.a % 100 + WHERE t1.a > 100 AND t2.a > 100 + """ + ) + + reader = pa.RecordBatchReader._import_from_c(df.__arrow_c_stream__()) + + interrupted = False + interrupt_error = None + query_started = threading.Event() + max_wait_time = 5.0 + + def trigger_interrupt(): + start_time = time.time() + while not query_started.is_set(): + time.sleep(0.1) + if time.time() - start_time > max_wait_time: + msg = f"Query did not start within {max_wait_time} seconds" + raise RuntimeError(msg) + + thread_id = threading.main_thread().ident + if thread_id is None: + msg = "Cannot get main thread ID" + raise RuntimeError(msg) + + exception = ctypes.py_object(KeyboardInterrupt) + res = ctypes.pythonapi.PyThreadState_SetAsyncExc( + ctypes.c_long(thread_id), exception + ) + if res != 1: + ctypes.pythonapi.PyThreadState_SetAsyncExc( + ctypes.c_long(thread_id), ctypes.py_object(0) + ) + msg = "Failed to raise KeyboardInterrupt in main thread" + raise RuntimeError(msg) + + interrupt_thread = threading.Thread(target=trigger_interrupt) + interrupt_thread.daemon = True + interrupt_thread.start() + + try: + query_started.set() + # consume the reader which should block and be interrupted + reader.read_all() + except KeyboardInterrupt: + interrupted = True + except Exception as e: # pragma: no cover - unexpected errors + interrupt_error = e + + if not interrupted: + pytest.fail(f"Stream was not interrupted; got error: {interrupt_error}") + + interrupt_thread.join(timeout=1.0) + + def test_show_select_where_no_rows(capsys) -> None: ctx = SessionContext() df = ctx.sql("SELECT 1 WHERE 1=0") diff --git a/python/tests/test_io.py b/python/tests/test_io.py index 7ca509689..91a0205fd 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -17,6 +17,7 @@ from pathlib import Path import pyarrow as pa +import pytest from datafusion import column from datafusion.io import read_avro, read_csv, read_json, read_parquet @@ -92,3 +93,30 @@ def test_read_avro(): path = Path.cwd() / "testing/data/avro/alltypes_plain.avro" avro_df = read_avro(path=path) assert avro_df is not None + + +def test_arrow_c_stream_large_dataset(ctx): + """DataFrame.__arrow_c_stream__ yields batches incrementally. + + This test constructs a DataFrame that would be far larger than available + memory if materialized. The ``__arrow_c_stream__`` method should expose a + stream of record batches without collecting the full dataset, so reading a + handful of batches should not exhaust process memory. + """ + # Create a very large DataFrame using range; this would be terabytes if collected + df = ctx.range(0, 1 << 40) + + reader = pa.RecordBatchReader._import_from_c(df.__arrow_c_stream__()) + + # Track RSS before consuming batches + psutil = pytest.importorskip("psutil") + process = psutil.Process() + start_rss = process.memory_info().rss + + for _ in range(5): + batch = reader.read_next_batch() + assert batch is not None + assert len(batch) > 0 + current_rss = process.memory_info().rss + # Ensure memory usage hasn't grown substantially (>50MB) + assert current_rss - start_rss < 50 * 1024 * 1024 diff --git a/src/context.rs b/src/context.rs index 36133a33d..3bc9f2989 100644 --- a/src/context.rs +++ b/src/context.rs @@ -34,7 +34,7 @@ use pyo3::prelude::*; use crate::catalog::{PyCatalog, PyTable, RustWrappedPyCatalogProvider}; use crate::dataframe::PyDataFrame; use crate::dataset::Dataset; -use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionResult}; +use crate::errors::{py_datafusion_err, PyDataFusionResult}; use crate::expr::sort_expr::PySortExpr; use crate::physical_plan::PyExecutionPlan; use crate::record_batch::PyRecordBatchStream; @@ -45,7 +45,7 @@ use crate::udaf::PyAggregateUDF; use crate::udf::PyScalarUDF; use crate::udtf::PyTableFunction; use crate::udwf::PyWindowUDF; -use crate::utils::{get_global_ctx, get_tokio_runtime, validate_pycapsule, wait_for_future}; +use crate::utils::{get_global_ctx, spawn_stream, validate_pycapsule, wait_for_future}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::arrow::record_batch::RecordBatch; @@ -66,7 +66,6 @@ use datafusion::execution::disk_manager::DiskManagerMode; use datafusion::execution::memory_pool::{FairSpillPool, GreedyMemoryPool, UnboundedMemoryPool}; use datafusion::execution::options::ReadOptions; use datafusion::execution::runtime_env::RuntimeEnvBuilder; -use datafusion::physical_plan::SendableRecordBatchStream; use datafusion::prelude::{ AvroReadOptions, CsvReadOptions, DataFrame, NdJsonReadOptions, ParquetReadOptions, }; @@ -74,7 +73,6 @@ use datafusion_ffi::catalog_provider::{FFI_CatalogProvider, ForeignCatalogProvid use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::types::{PyCapsule, PyDict, PyList, PyTuple, PyType}; use pyo3::IntoPyObjectExt; -use tokio::task::JoinHandle; /// Configuration options for a SessionContext #[pyclass(name = "SessionConfig", module = "datafusion", subclass)] @@ -1132,12 +1130,8 @@ impl PySessionContext { py: Python, ) -> PyDataFusionResult { let ctx: TaskContext = TaskContext::from(&self.ctx.state()); - // create a Tokio runtime to run the async code - let rt = &get_tokio_runtime().0; let plan = plan.plan.clone(); - let fut: JoinHandle> = - rt.spawn(async move { plan.execute(part, Arc::new(ctx)) }); - let stream = wait_for_future(py, async { fut.await.map_err(to_datafusion_err) })???; + let stream = spawn_stream(py, async move { plan.execute(part, Arc::new(ctx)) })?; Ok(PyRecordBatchStream::new(stream)) } } diff --git a/src/dataframe.rs b/src/dataframe.rs index 1437f5f82..6a09c8047 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -19,13 +19,13 @@ use std::collections::HashMap; use std::ffi::CString; use std::sync::Arc; -use arrow::array::{new_null_array, RecordBatch, RecordBatchIterator, RecordBatchReader}; +use arrow::array::{new_null_array, RecordBatch, RecordBatchReader}; use arrow::compute::can_cast_types; use arrow::error::ArrowError; use arrow::ffi::FFI_ArrowSchema; use arrow::ffi_stream::FFI_ArrowArrayStream; use arrow::pyarrow::FromPyArrow; -use datafusion::arrow::datatypes::Schema; +use datafusion::arrow::datatypes::{Schema, SchemaRef}; use datafusion::arrow::pyarrow::{PyArrowType, ToPyArrow}; use datafusion::arrow::util::pretty; use datafusion::common::UnnestOptions; @@ -42,16 +42,16 @@ use pyo3::exceptions::PyValueError; use pyo3::prelude::*; use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods}; -use tokio::task::JoinHandle; use crate::catalog::PyTable; -use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError}; +use crate::errors::{py_datafusion_err, PyDataFusionError}; use crate::expr::sort_expr::to_sort_expressions; use crate::physical_plan::PyExecutionPlan; use crate::record_batch::PyRecordBatchStream; use crate::sql::logical::PyLogicalPlan; use crate::utils::{ - get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, validate_pycapsule, wait_for_future, + get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, spawn_stream, spawn_streams, + validate_pycapsule, wait_for_future, }; use crate::{ errors::PyDataFusionResult, @@ -353,6 +353,54 @@ impl PyDataFrame { Ok(html_str) } } +/// Synchronous wrapper around a [`SendableRecordBatchStream`] used for +/// the `__arrow_c_stream__` implementation. +/// +/// It uses `runtime.block_on` to consume the underlying async stream, +/// providing synchronous iteration. When a `projection` is set, each +/// batch is converted via `record_batch_into_schema` to apply schema +/// changes per batch. +struct DataFrameStreamReader { + stream: SendableRecordBatchStream, + schema: SchemaRef, + projection: Option, +} + +impl Iterator for DataFrameStreamReader { + type Item = Result; + + fn next(&mut self) -> Option { + // Use wait_for_future to poll the underlying async stream while + // respecting Python signal handling (e.g. ``KeyboardInterrupt``). + // This mirrors the behaviour of other synchronous wrappers and + // prevents blocking indefinitely when a Python interrupt is raised. + let fut = self.stream.next(); + let result = Python::with_gil(|py| wait_for_future(py, fut)); + + match result { + Ok(Some(Ok(batch))) => { + let batch = if let Some(ref schema) = self.projection { + match record_batch_into_schema(batch, schema.as_ref()) { + Ok(b) => b, + Err(e) => return Some(Err(e)), + } + } else { + batch + }; + Some(Ok(batch)) + } + Ok(Some(Err(e))) => Some(Err(ArrowError::ExternalError(Box::new(e)))), + Ok(None) => None, + Err(e) => Some(Err(ArrowError::ExternalError(Box::new(e)))), + } + } +} + +impl RecordBatchReader for DataFrameStreamReader { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} #[pymethods] impl PyDataFrame { @@ -879,8 +927,11 @@ impl PyDataFrame { py: Python<'py>, requested_schema: Option>, ) -> PyDataFusionResult> { - let mut batches = wait_for_future(py, self.df.as_ref().clone().collect())??; + let df = self.df.as_ref().clone(); + let stream = spawn_stream(py, async move { df.execute_stream().await })?; + let mut schema: Schema = self.df.schema().to_owned().into(); + let mut projection: Option = None; if let Some(schema_capsule) = requested_schema { validate_pycapsule(&schema_capsule, "arrow_schema")?; @@ -889,16 +940,16 @@ impl PyDataFrame { let desired_schema = Schema::try_from(schema_ptr)?; schema = project_schema(schema, desired_schema)?; - - batches = batches - .into_iter() - .map(|record_batch| record_batch_into_schema(record_batch, &schema)) - .collect::, ArrowError>>()?; + projection = Some(Arc::new(schema.clone())); } - let batches_wrapped = batches.into_iter().map(Ok); + let schema_ref = projection.clone().unwrap_or_else(|| Arc::new(schema)); - let reader = RecordBatchIterator::new(batches_wrapped, Arc::new(schema)); + let reader = DataFrameStreamReader { + stream, + schema: schema_ref, + projection, + }; let reader: Box = Box::new(reader); let ffi_stream = FFI_ArrowArrayStream::new(reader); @@ -907,26 +958,15 @@ impl PyDataFrame { } fn execute_stream(&self, py: Python) -> PyDataFusionResult { - // create a Tokio runtime to run the async code - let rt = &get_tokio_runtime().0; let df = self.df.as_ref().clone(); - let fut: JoinHandle> = - rt.spawn(async move { df.execute_stream().await }); - let stream = wait_for_future(py, async { fut.await.map_err(to_datafusion_err) })???; + let stream = spawn_stream(py, async move { df.execute_stream().await })?; Ok(PyRecordBatchStream::new(stream)) } fn execute_stream_partitioned(&self, py: Python) -> PyResult> { - // create a Tokio runtime to run the async code - let rt = &get_tokio_runtime().0; let df = self.df.as_ref().clone(); - let fut: JoinHandle>> = - rt.spawn(async move { df.execute_stream_partitioned().await }); - let stream = wait_for_future(py, async { fut.await.map_err(to_datafusion_err) })? - .map_err(py_datafusion_err)? - .map_err(py_datafusion_err)?; - - Ok(stream.into_iter().map(PyRecordBatchStream::new).collect()) + let streams = spawn_streams(py, async move { df.execute_stream_partitioned().await })?; + Ok(streams.into_iter().map(PyRecordBatchStream::new).collect()) } /// Convert to pandas dataframe with pyarrow diff --git a/src/utils.rs b/src/utils.rs index 3b30de5de..b0789cb7d 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -17,16 +17,17 @@ use crate::{ common::data_type::PyScalarValue, - errors::{PyDataFusionError, PyDataFusionResult}, + errors::{to_datafusion_err, PyDataFusionError, PyDataFusionResult}, TokioRuntime, }; use datafusion::{ - common::ScalarValue, execution::context::SessionContext, logical_expr::Volatility, + common::ScalarValue, execution::context::SessionContext, execution::SendableRecordBatchStream, + logical_expr::Volatility, }; use pyo3::prelude::*; use pyo3::{exceptions::PyValueError, types::PyCapsule}; use std::{future::Future, sync::OnceLock, time::Duration}; -use tokio::{runtime::Runtime, time::sleep}; +use tokio::{runtime::Runtime, task::JoinHandle, time::sleep}; /// Utility to get the Tokio Runtime from Python #[inline] pub(crate) fn get_tokio_runtime() -> &'static TokioRuntime { @@ -84,6 +85,36 @@ where }) } +/// Spawn a [`SendableRecordBatchStream`] on the Tokio runtime and wait for completion +/// while respecting Python signal handling. +pub(crate) fn spawn_stream(py: Python, fut: F) -> PyDataFusionResult +where + F: Future> + Send + 'static, +{ + let rt = &get_tokio_runtime().0; + let handle: JoinHandle> = rt.spawn(fut); + Ok(wait_for_future(py, async { + handle.await.map_err(to_datafusion_err) + })???) +} + +/// Spawn a partitioned [`SendableRecordBatchStream`] on the Tokio runtime and wait for completion +/// while respecting Python signal handling. +pub(crate) fn spawn_streams( + py: Python, + fut: F, +) -> PyDataFusionResult> +where + F: Future>> + Send + 'static, +{ + let rt = &get_tokio_runtime().0; + let handle: JoinHandle>> = + rt.spawn(fut); + Ok(wait_for_future(py, async { + handle.await.map_err(to_datafusion_err) + })???) +} + pub(crate) fn parse_volatility(value: &str) -> PyDataFusionResult { Ok(match value { "immutable" => Volatility::Immutable, From f78e90b9bab4d7e76b360df7e0b5032dd3c3504f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 1 Sep 2025 13:14:57 +0800 Subject: [PATCH 02/67] refactor: improve DataFrame streaming, memory management, and error handling - Refactor record batch streaming to use `poll_next_batch` for clearer error handling - Improve `spawn_future`/`spawn_stream` functions for better Python exception integration and code reuse - Update `datafusion` and `datafusion-ffi` dependencies to 49.0.2 - Fix PyArrow `RecordBatchReader` import to use `_import_from_c_capsule` for safer memory handling - Refactor `ArrowArrayStream` handling to use `PyCapsule` with destructor for improved memory management - Refactor projection initialization in `PyDataFrame` for clarity - Move `range` functionality into `_testing.py` helper - Rename test column in `test_table_from_batches_stream` for accuracy - Add tests for `RecordBatchReader` and enhance DataFrame stream handling --- docs/source/user-guide/dataframe/index.rst | 2 +- examples/datafusion-ffi-example/Cargo.toml | 4 +- python/datafusion/_testing.py | 42 +++++++++++++++++ python/datafusion/context.py | 30 ------------ python/datafusion/dataframe.py | 2 +- python/tests/test_dataframe.py | 10 +++- python/tests/test_io.py | 21 +++++++-- src/dataframe.rs | 53 ++++++++++++++++++---- src/record_batch.rs | 14 ++++-- src/utils.rs | 42 ++++++++++++----- 10 files changed, 157 insertions(+), 63 deletions(-) create mode 100644 python/datafusion/_testing.py diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 00fe70567..18b1ba411 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -164,7 +164,7 @@ out-of-memory errors. import pyarrow as pa # Create a PyArrow RecordBatchReader without materializing all batches - reader = pa.RecordBatchReader._import_from_c(df.__arrow_c_stream__()) + reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) for batch in reader: ... # process each batch as it is produced diff --git a/examples/datafusion-ffi-example/Cargo.toml b/examples/datafusion-ffi-example/Cargo.toml index cd72ffbc3..647f6c51e 100644 --- a/examples/datafusion-ffi-example/Cargo.toml +++ b/examples/datafusion-ffi-example/Cargo.toml @@ -21,8 +21,8 @@ version = "0.2.0" edition = "2021" [dependencies] -datafusion = { version = "49.0.1" } -datafusion-ffi = { version = "49.0.1" } +datafusion = { version = "49.0.2" } +datafusion-ffi = { version = "49.0.2" } pyo3 = { version = "0.23", features = ["extension-module", "abi3", "abi3-py39"] } arrow = { version = "55.0.0" } arrow-array = { version = "55.0.0" } diff --git a/python/datafusion/_testing.py b/python/datafusion/_testing.py new file mode 100644 index 000000000..fa4007cc6 --- /dev/null +++ b/python/datafusion/_testing.py @@ -0,0 +1,42 @@ +"""Testing-only helpers for datafusion-python. + +This module contains utilities used by the test-suite that should not be +exposed as part of the public API. Keep the implementation minimal and +documented so reviewers can easily see it's test-only. +""" +from __future__ import annotations + +from typing import Any + +from .context import SessionContext + + +def range_table( + ctx: SessionContext, + start: int, + stop: int | None = None, + step: int = 1, + partitions: int | None = None, +) -> Any: + """Create a DataFrame containing a sequence of numbers using SQL RANGE. + + This mirrors the previous ``SessionContext.range`` convenience method but + lives in a testing-only module so it doesn't expand the public surface. + + Args: + ctx: SessionContext instance to run the SQL against. + start: Starting value for the sequence or exclusive stop when ``stop`` + is ``None``. + stop: Exclusive upper bound of the sequence. + step: Increment between successive values. + partitions: Optional number of partitions for the generated data. + + Returns: + DataFrame produced by the range table function. + """ + if stop is None: + start, stop = 0, start + + parts = f", {int(partitions)}" if partitions is not None else "" + sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" + return ctx.sql(sql) diff --git a/python/datafusion/context.py b/python/datafusion/context.py index b0bd7c1f5..c67edc7d2 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -731,36 +731,6 @@ def from_polars(self, data: pl.DataFrame, name: str | None = None) -> DataFrame: """ return DataFrame(self.ctx.from_polars(data, name)) - def range( - self, - start: int, - stop: int | None = None, - step: int = 1, - partitions: int | None = None, - ) -> DataFrame: - """Create a DataFrame containing a sequence of numbers. - - This is backed by DataFusion's ``range`` table function, which generates - values lazily and therefore does not materialize the full range in - memory. When ``stop`` is omitted, ``start`` is treated as the stop value - and the sequence begins at zero. - - Args: - start: Starting value for the sequence or the exclusive stop if - ``stop`` is ``None``. - stop: Exclusive upper bound of the sequence. - step: Increment between successive values. - partitions: Optional number of partitions for the generated data. - - Returns: - DataFrame yielding the requested range of values. - """ - if stop is None: - start, stop = 0, start - - parts = f", {int(partitions)}" if partitions is not None else "" - sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" # noqa: S608 - return self.sql(sql) # https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 # is the discussion on how we arrived at adding register_view diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 8ac983451..9a08e1364 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1127,7 +1127,7 @@ def __iter__(self) -> Iterator[pa.RecordBatch]: """ import pyarrow as pa - reader = pa.RecordBatchReader._import_from_c(self.__arrow_c_stream__()) + reader = pa.RecordBatchReader._import_from_c_capsule(self.__arrow_c_stream__()) yield from reader def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 17a804f2c..4b0672bfb 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1605,6 +1605,14 @@ def fail_collect(self): # pragma: no cover - failure path assert table.column("a").num_chunks == 2 +def test_arrow_c_stream_reader(df): + reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) + assert isinstance(reader, pa.RecordBatchReader) + table = pa.Table.from_batches(reader) + expected = pa.Table.from_batches(df.collect()) + assert table.equals(expected) + + def test_to_pylist(df): # Convert datafusion dataframe to Python list pylist = df.to_pylist() @@ -2743,7 +2751,7 @@ def test_arrow_c_stream_interrupted(): """ ) - reader = pa.RecordBatchReader._import_from_c(df.__arrow_c_stream__()) + reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) interrupted = False interrupt_error = None diff --git a/python/tests/test_io.py b/python/tests/test_io.py index 91a0205fd..cd1fe4fb4 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -18,7 +18,8 @@ import pyarrow as pa import pytest -from datafusion import column +from datafusion import DataFrame, column +from datafusion._testing import range_table from datafusion.io import read_avro, read_csv, read_json, read_parquet @@ -104,9 +105,9 @@ def test_arrow_c_stream_large_dataset(ctx): handful of batches should not exhaust process memory. """ # Create a very large DataFrame using range; this would be terabytes if collected - df = ctx.range(0, 1 << 40) + df = range_table(ctx, 0, 1 << 40) - reader = pa.RecordBatchReader._import_from_c(df.__arrow_c_stream__()) + reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) # Track RSS before consuming batches psutil = pytest.importorskip("psutil") @@ -120,3 +121,17 @@ def test_arrow_c_stream_large_dataset(ctx): current_rss = process.memory_info().rss # Ensure memory usage hasn't grown substantially (>50MB) assert current_rss - start_rss < 50 * 1024 * 1024 + + +def test_table_from_batches_stream(ctx, monkeypatch): + df = range_table(ctx, 0, 10) + + def fail_collect(self): # pragma: no cover - failure path + msg = "collect should not be called" + raise AssertionError(msg) + + monkeypatch.setattr(DataFrame, "collect", fail_collect) + + table = pa.Table.from_batches(df) + assert table.shape == (10, 1) + assert table.column_names == ["value"] diff --git a/src/dataframe.rs b/src/dataframe.rs index 6a09c8047..bf11b59c9 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -16,7 +16,7 @@ // under the License. use std::collections::HashMap; -use std::ffi::CString; +use std::ffi::{c_void, CStr, CString}; use std::sync::Arc; use arrow::array::{new_null_array, RecordBatch, RecordBatchReader}; @@ -39,6 +39,7 @@ use datafusion::prelude::*; use datafusion_ffi::table_provider::FFI_TableProvider; use futures::{StreamExt, TryStreamExt}; use pyo3::exceptions::PyValueError; +use pyo3::ffi; use pyo3::prelude::*; use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods}; @@ -47,7 +48,7 @@ use crate::catalog::PyTable; use crate::errors::{py_datafusion_err, PyDataFusionError}; use crate::expr::sort_expr::to_sort_expressions; use crate::physical_plan::PyExecutionPlan; -use crate::record_batch::PyRecordBatchStream; +use crate::record_batch::{poll_next_batch, PyRecordBatchStream}; use crate::sql::logical::PyLogicalPlan; use crate::utils::{ get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, spawn_stream, spawn_streams, @@ -58,6 +59,21 @@ use crate::{ expr::{sort_expr::PySortExpr, PyExpr}, }; +#[allow(clippy::manual_c_str_literals)] +static ARROW_STREAM_NAME: &CStr = + unsafe { CStr::from_bytes_with_nul_unchecked(b"arrow_array_stream\0") }; + +unsafe extern "C" fn drop_stream(capsule: *mut ffi::PyObject) { + if capsule.is_null() { + return; + } + let stream_ptr = + ffi::PyCapsule_GetPointer(capsule, ARROW_STREAM_NAME.as_ptr()) as *mut FFI_ArrowArrayStream; + if !stream_ptr.is_null() { + drop(Box::from_raw(stream_ptr)); + } +} + // https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 // - we have not decided on the table_provider approach yet // this is an interim implementation @@ -374,11 +390,11 @@ impl Iterator for DataFrameStreamReader { // respecting Python signal handling (e.g. ``KeyboardInterrupt``). // This mirrors the behaviour of other synchronous wrappers and // prevents blocking indefinitely when a Python interrupt is raised. - let fut = self.stream.next(); + let fut = poll_next_batch(&mut self.stream); let result = Python::with_gil(|py| wait_for_future(py, fut)); match result { - Ok(Some(Ok(batch))) => { + Ok(Ok(Some(batch))) => { let batch = if let Some(ref schema) = self.projection { match record_batch_into_schema(batch, schema.as_ref()) { Ok(b) => b, @@ -389,8 +405,8 @@ impl Iterator for DataFrameStreamReader { }; Some(Ok(batch)) } - Ok(Some(Err(e))) => Some(Err(ArrowError::ExternalError(Box::new(e)))), - Ok(None) => None, + Ok(Ok(None)) => None, + Ok(Err(e)) => Some(Err(ArrowError::ExternalError(Box::new(e)))), Err(e) => Some(Err(ArrowError::ExternalError(Box::new(e)))), } } @@ -943,7 +959,7 @@ impl PyDataFrame { projection = Some(Arc::new(schema.clone())); } - let schema_ref = projection.clone().unwrap_or_else(|| Arc::new(schema)); + let schema_ref = Arc::new(schema.clone()); let reader = DataFrameStreamReader { stream, @@ -952,9 +968,26 @@ impl PyDataFrame { }; let reader: Box = Box::new(reader); - let ffi_stream = FFI_ArrowArrayStream::new(reader); - let stream_capsule_name = CString::new("arrow_array_stream").unwrap(); - PyCapsule::new(py, ffi_stream, Some(stream_capsule_name)).map_err(PyDataFusionError::from) + let stream = Box::new(FFI_ArrowArrayStream::new(reader)); + let stream_ptr = Box::into_raw(stream); + assert!( + !stream_ptr.is_null(), + "ArrowArrayStream pointer should never be null" + ); + let capsule = unsafe { + ffi::PyCapsule_New( + stream_ptr as *mut c_void, + ARROW_STREAM_NAME.as_ptr(), + Some(drop_stream), + ) + }; + if capsule.is_null() { + unsafe { drop(Box::from_raw(stream_ptr)) }; + Err(PyErr::fetch(py).into()) + } else { + let any = unsafe { Bound::from_owned_ptr(py, capsule) }; + Ok(any.downcast_into::().unwrap()) + } } fn execute_stream(&self, py: Python) -> PyDataFusionResult { diff --git a/src/record_batch.rs b/src/record_batch.rs index a85f05423..563b0f4b2 100644 --- a/src/record_batch.rs +++ b/src/record_batch.rs @@ -84,15 +84,20 @@ impl PyRecordBatchStream { } } +pub(crate) async fn poll_next_batch( + stream: &mut SendableRecordBatchStream, +) -> datafusion::error::Result> { + stream.next().await.transpose() +} + async fn next_stream( stream: Arc>, sync: bool, ) -> PyResult { let mut stream = stream.lock().await; - match stream.next().await { - Some(Ok(batch)) => Ok(batch.into()), - Some(Err(e)) => Err(PyDataFusionError::from(e))?, - None => { + match poll_next_batch(&mut stream).await { + Ok(Some(batch)) => Ok(batch.into()), + Ok(None) => { // Depending on whether the iteration is sync or not, we raise either a // StopIteration or a StopAsyncIteration if sync { @@ -101,5 +106,6 @@ async fn next_stream( Err(PyStopAsyncIteration::new_err("stream exhausted")) } } + Err(e) => Err(PyDataFusionError::from(e))?, } } diff --git a/src/utils.rs b/src/utils.rs index b0789cb7d..5ee648389 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -85,17 +85,42 @@ where }) } +/// Spawn a [`Future`] on the Tokio runtime and wait for completion +/// while respecting Python signal handling. +pub(crate) fn spawn_future(py: Python, fut: F) -> PyDataFusionResult +where + F: Future> + Send + 'static, + T: Send + 'static, +{ + let rt = &get_tokio_runtime().0; + let handle: JoinHandle> = rt.spawn(fut); + // Wait for the join handle while respecting Python signal handling. + // We handle errors in two steps so `?` maps the error types correctly: + // 1) convert any Python-related error from `wait_for_future` into `PyDataFusionError` + // 2) convert any DataFusion error (inner result) into `PyDataFusionError` + let inner_result = wait_for_future(py, async { + // handle.await yields `Result, JoinError>` + // map JoinError into a DataFusion error so the async block returns + // `datafusion::common::Result` (i.e. Result) + match handle.await { + Ok(inner) => inner, + Err(join_err) => Err(to_datafusion_err(join_err)), + } + })?; // converts PyErr -> PyDataFusionError + + // `inner_result` is `datafusion::common::Result`; use `?` to convert + // the inner DataFusion error into `PyDataFusionError` via `From` and + // return the inner `T` on success. + Ok(inner_result?) +} + /// Spawn a [`SendableRecordBatchStream`] on the Tokio runtime and wait for completion /// while respecting Python signal handling. pub(crate) fn spawn_stream(py: Python, fut: F) -> PyDataFusionResult where F: Future> + Send + 'static, { - let rt = &get_tokio_runtime().0; - let handle: JoinHandle> = rt.spawn(fut); - Ok(wait_for_future(py, async { - handle.await.map_err(to_datafusion_err) - })???) + spawn_future(py, fut) } /// Spawn a partitioned [`SendableRecordBatchStream`] on the Tokio runtime and wait for completion @@ -107,12 +132,7 @@ pub(crate) fn spawn_streams( where F: Future>> + Send + 'static, { - let rt = &get_tokio_runtime().0; - let handle: JoinHandle>> = - rt.spawn(fut); - Ok(wait_for_future(py, async { - handle.await.map_err(to_datafusion_err) - })???) + spawn_future(py, fut) } pub(crate) fn parse_volatility(value: &str) -> PyDataFusionResult { From e32252113f5815a0ee62947ce37467e36166a189 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 11:56:43 +0800 Subject: [PATCH 03/67] feat: enhance DataFrame streaming and improve robustness, tests, and docs - Preserve partition order in DataFrame streaming and update related tests - Add tests for record batch ordering and DataFrame batch iteration - Improve `drop_stream` to correctly handle PyArrow ownership transfer and null pointers - Replace `assert` with `debug_assert` for safer ArrowArrayStream validation - Add documentation for `poll_next_batch` in PyRecordBatchStream - Refactor tests to use `fail_collect` fixture for DataFrame collect - Refactor `range_table` return type to `DataFrame` for clearer type hints - Minor cleanup in SessionContext (remove extra blank line) --- docs/source/user-guide/dataframe/index.rst | 8 ++ python/datafusion/_testing.py | 10 +- python/datafusion/context.py | 1 - python/datafusion/dataframe.py | 9 +- python/tests/conftest.py | 11 ++- python/tests/test_dataframe.py | 40 ++++++-- python/tests/test_io.py | 10 +- src/dataframe.rs | 108 +++++++++++++-------- src/record_batch.rs | 1 + 9 files changed, 135 insertions(+), 63 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 18b1ba411..37f559186 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -168,6 +168,14 @@ out-of-memory errors. 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 diff --git a/python/datafusion/_testing.py b/python/datafusion/_testing.py index fa4007cc6..224f2b921 100644 --- a/python/datafusion/_testing.py +++ b/python/datafusion/_testing.py @@ -4,12 +4,16 @@ exposed as part of the public API. Keep the implementation minimal and documented so reviewers can easily see it's test-only. """ + from __future__ import annotations -from typing import Any +from typing import TYPE_CHECKING from .context import SessionContext +if TYPE_CHECKING: + from datafusion import DataFrame + def range_table( ctx: SessionContext, @@ -17,7 +21,7 @@ def range_table( stop: int | None = None, step: int = 1, partitions: int | None = None, -) -> Any: +) -> DataFrame: """Create a DataFrame containing a sequence of numbers using SQL RANGE. This mirrors the previous ``SessionContext.range`` convenience method but @@ -38,5 +42,5 @@ def range_table( start, stop = 0, start parts = f", {int(partitions)}" if partitions is not None else "" - sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" + sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" # noqa: S608 return ctx.sql(sql) diff --git a/python/datafusion/context.py b/python/datafusion/context.py index c67edc7d2..bce51d644 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -731,7 +731,6 @@ def from_polars(self, data: pl.DataFrame, name: str | None = None) -> DataFrame: """ return DataFrame(self.ctx.from_polars(data, name)) - # https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 # is the discussion on how we arrived at adding register_view def register_view(self, name: str, df: DataFrame) -> None: diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 9a08e1364..a196e38a3 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -290,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. """ @@ -1114,7 +1117,8 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: Arrow PyCapsule object representing an ``ArrowArrayStream``. """ # ``DataFrame.__arrow_c_stream__`` in the Rust extension leverages - # ``execute_stream`` under the hood to stream batches one at a time. + # ``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]: @@ -1123,7 +1127,8 @@ def __iter__(self) -> Iterator[pa.RecordBatch]: 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 - streaming APIs so ``collect`` is never invoked. + partitioned streaming APIs so ``collect`` is never invoked and batch + order across partitions is preserved. """ import pyarrow as pa diff --git a/python/tests/conftest.py b/python/tests/conftest.py index 9548fbfe4..26ed7281d 100644 --- a/python/tests/conftest.py +++ b/python/tests/conftest.py @@ -17,7 +17,7 @@ import pyarrow as pa import pytest -from datafusion import SessionContext +from datafusion import DataFrame, SessionContext from pyarrow.csv import write_csv @@ -49,3 +49,12 @@ def database(ctx, tmp_path): delimiter=",", schema_infer_max_records=10, ) + + +@pytest.fixture +def fail_collect(monkeypatch): + def _fail_collect(self, *args, **kwargs): # pragma: no cover - failure path + msg = "collect should not be called" + raise AssertionError(msg) + + monkeypatch.setattr(DataFrame, "collect", _fail_collect) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 4b0672bfb..cf8e2e5b7 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1582,7 +1582,19 @@ def test_empty_to_arrow_table(df): assert set(pyarrow_table.column_names) == {"a", "b", "c"} -def test_arrow_c_stream_to_table(monkeypatch): +def test_iter_batches_dataframe(fail_collect): + ctx = SessionContext() + + batch1 = pa.record_batch([pa.array([1])], names=["a"]) + batch2 = pa.record_batch([pa.array([2])], names=["a"]) + df = ctx.create_dataframe([[batch1], [batch2]]) + + expected = [batch1, batch2] + for got, exp in zip(df, expected): + assert got.equals(exp) + + +def test_arrow_c_stream_to_table(fail_collect): ctx = SessionContext() # Create a DataFrame with two separate record batches @@ -1590,19 +1602,31 @@ def test_arrow_c_stream_to_table(monkeypatch): batch2 = pa.record_batch([pa.array([2])], names=["a"]) df = ctx.create_dataframe([[batch1], [batch2]]) - # Fail if the DataFrame is pre-collected - def fail_collect(self): # pragma: no cover - failure path - msg = "collect should not be called" - raise AssertionError(msg) + table = pa.Table.from_batches(df) + batches = table.to_batches() + + assert len(batches) == 2 + assert batches[0].equals(batch1) + assert batches[1].equals(batch2) + assert table.schema == df.schema() + assert table.column("a").num_chunks == 2 + + +def test_arrow_c_stream_order(): + ctx = SessionContext() - monkeypatch.setattr(DataFrame, "collect", fail_collect) + batch1 = pa.record_batch([pa.array([1])], names=["a"]) + batch2 = pa.record_batch([pa.array([2])], names=["a"]) + + df = ctx.create_dataframe([[batch1, batch2]]) table = pa.Table.from_batches(df) expected = pa.Table.from_batches([batch1, batch2]) assert table.equals(expected) - assert table.schema == df.schema() - assert table.column("a").num_chunks == 2 + col = table.column("a") + assert col.chunk(0)[0].as_py() == 1 + assert col.chunk(1)[0].as_py() == 2 def test_arrow_c_stream_reader(df): diff --git a/python/tests/test_io.py b/python/tests/test_io.py index cd1fe4fb4..65480ca34 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -18,7 +18,7 @@ import pyarrow as pa import pytest -from datafusion import DataFrame, column +from datafusion import column from datafusion._testing import range_table from datafusion.io import read_avro, read_csv, read_json, read_parquet @@ -123,15 +123,9 @@ def test_arrow_c_stream_large_dataset(ctx): assert current_rss - start_rss < 50 * 1024 * 1024 -def test_table_from_batches_stream(ctx, monkeypatch): +def test_table_from_batches_stream(ctx, fail_collect): df = range_table(ctx, 0, 10) - def fail_collect(self): # pragma: no cover - failure path - msg = "collect should not be called" - raise AssertionError(msg) - - monkeypatch.setattr(DataFrame, "collect", fail_collect) - table = pa.Table.from_batches(df) assert table.shape == (10, 1) assert table.column_names == ["value"] diff --git a/src/dataframe.rs b/src/dataframe.rs index bf11b59c9..fe03e98b3 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -67,11 +67,25 @@ unsafe extern "C" fn drop_stream(capsule: *mut ffi::PyObject) { if capsule.is_null() { return; } - let stream_ptr = - ffi::PyCapsule_GetPointer(capsule, ARROW_STREAM_NAME.as_ptr()) as *mut FFI_ArrowArrayStream; - if !stream_ptr.is_null() { - drop(Box::from_raw(stream_ptr)); + + // When PyArrow imports this capsule it steals the raw stream pointer and + // sets the capsule's internal pointer to NULL. In that case + // `PyCapsule_IsValid` returns 0 and this destructor must not drop the + // stream as ownership has been transferred to PyArrow. If the capsule was + // never imported, the pointer remains valid and we are responsible for + // freeing the stream here. + if ffi::PyCapsule_IsValid(capsule, ARROW_STREAM_NAME.as_ptr()) == 1 { + let stream_ptr = ffi::PyCapsule_GetPointer(capsule, ARROW_STREAM_NAME.as_ptr()) + as *mut FFI_ArrowArrayStream; + if !stream_ptr.is_null() { + drop(Box::from_raw(stream_ptr)); + } } + + // `PyCapsule_GetPointer` sets a Python error on failure. Clear it only + // after the stream has been released (or determined to be owned + // elsewhere). + ffi::PyErr_Clear(); } // https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 @@ -369,50 +383,59 @@ impl PyDataFrame { Ok(html_str) } } -/// Synchronous wrapper around a [`SendableRecordBatchStream`] used for -/// the `__arrow_c_stream__` implementation. + +/// Synchronous wrapper around partitioned [`SendableRecordBatchStream`]s used +/// for the `__arrow_c_stream__` implementation. /// -/// It uses `runtime.block_on` to consume the underlying async stream, -/// providing synchronous iteration. When a `projection` is set, each -/// batch is converted via `record_batch_into_schema` to apply schema -/// changes per batch. -struct DataFrameStreamReader { - stream: SendableRecordBatchStream, +/// It drains each partition's stream sequentially, yielding record batches in +/// their original partition order. When a `projection` is set, each batch is +/// converted via `record_batch_into_schema` to apply schema changes per batch. +struct PartitionedDataFrameStreamReader { + streams: Vec, schema: SchemaRef, projection: Option, + current: usize, } -impl Iterator for DataFrameStreamReader { +impl Iterator for PartitionedDataFrameStreamReader { type Item = Result; fn next(&mut self) -> Option { - // Use wait_for_future to poll the underlying async stream while - // respecting Python signal handling (e.g. ``KeyboardInterrupt``). - // This mirrors the behaviour of other synchronous wrappers and - // prevents blocking indefinitely when a Python interrupt is raised. - let fut = poll_next_batch(&mut self.stream); - let result = Python::with_gil(|py| wait_for_future(py, fut)); - - match result { - Ok(Ok(Some(batch))) => { - let batch = if let Some(ref schema) = self.projection { - match record_batch_into_schema(batch, schema.as_ref()) { - Ok(b) => b, - Err(e) => return Some(Err(e)), - } - } else { - batch - }; - Some(Ok(batch)) + while self.current < self.streams.len() { + let stream = &mut self.streams[self.current]; + let fut = poll_next_batch(stream); + let result = Python::with_gil(|py| wait_for_future(py, fut)); + + match result { + Ok(Ok(Some(batch))) => { + let batch = if let Some(ref schema) = self.projection { + match record_batch_into_schema(batch, schema.as_ref()) { + Ok(b) => b, + Err(e) => return Some(Err(e)), + } + } else { + batch + }; + return Some(Ok(batch)); + } + Ok(Ok(None)) => { + self.current += 1; + continue; + } + Ok(Err(e)) => { + return Some(Err(ArrowError::ExternalError(Box::new(e)))); + } + Err(e) => { + return Some(Err(ArrowError::ExternalError(Box::new(e)))); + } } - Ok(Ok(None)) => None, - Ok(Err(e)) => Some(Err(ArrowError::ExternalError(Box::new(e)))), - Err(e) => Some(Err(ArrowError::ExternalError(Box::new(e)))), } + + None } } -impl RecordBatchReader for DataFrameStreamReader { +impl RecordBatchReader for PartitionedDataFrameStreamReader { fn schema(&self) -> SchemaRef { self.schema.clone() } @@ -944,7 +967,7 @@ impl PyDataFrame { requested_schema: Option>, ) -> PyDataFusionResult> { let df = self.df.as_ref().clone(); - let stream = spawn_stream(py, async move { df.execute_stream().await })?; + let streams = spawn_streams(py, async move { df.execute_stream_partitioned().await })?; let mut schema: Schema = self.df.schema().to_owned().into(); let mut projection: Option = None; @@ -961,19 +984,24 @@ impl PyDataFrame { let schema_ref = Arc::new(schema.clone()); - let reader = DataFrameStreamReader { - stream, + let reader = PartitionedDataFrameStreamReader { + streams, schema: schema_ref, projection, + current: 0, }; let reader: Box = Box::new(reader); let stream = Box::new(FFI_ArrowArrayStream::new(reader)); let stream_ptr = Box::into_raw(stream); - assert!( + debug_assert!( !stream_ptr.is_null(), - "ArrowArrayStream pointer should never be null" + "ArrowArrayStream pointer should never be null", ); + // The returned capsule allows zero-copy hand-off to PyArrow. When + // PyArrow imports the capsule it assumes ownership of the stream and + // nulls out the capsule's internal pointer so `drop_stream` knows not to + // free it. let capsule = unsafe { ffi::PyCapsule_New( stream_ptr as *mut c_void, diff --git a/src/record_batch.rs b/src/record_batch.rs index 563b0f4b2..2a5a0039f 100644 --- a/src/record_batch.rs +++ b/src/record_batch.rs @@ -84,6 +84,7 @@ impl PyRecordBatchStream { } } +/// Polls the next batch from a `SendableRecordBatchStream`, converting the `Option>` form. pub(crate) async fn poll_next_batch( stream: &mut SendableRecordBatchStream, ) -> datafusion::error::Result> { From 31e8ed19e793e81d1257a8daad2536d4174a7068 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 16:45:40 +0800 Subject: [PATCH 04/67] feat: add testing utilities for DataFrame range generation --- python/tests/test_io.py | 4 +++- python/{datafusion/_testing.py => tests/utils.py} | 5 ++--- 2 files changed, 5 insertions(+), 4 deletions(-) rename python/{datafusion/_testing.py => tests/utils.py} (95%) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index 65480ca34..d89578cd9 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -14,14 +14,16 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + from pathlib import Path import pyarrow as pa import pytest from datafusion import column -from datafusion._testing import range_table from datafusion.io import read_avro, read_csv, read_json, read_parquet +from .utils import range_table + def test_read_json_global_ctx(ctx): path = Path(__file__).parent.resolve() diff --git a/python/datafusion/_testing.py b/python/tests/utils.py similarity index 95% rename from python/datafusion/_testing.py rename to python/tests/utils.py index 224f2b921..612f666d4 100644 --- a/python/datafusion/_testing.py +++ b/python/tests/utils.py @@ -9,10 +9,9 @@ from typing import TYPE_CHECKING -from .context import SessionContext - if TYPE_CHECKING: from datafusion import DataFrame + from datafusion.context import SessionContext def range_table( @@ -42,5 +41,5 @@ def range_table( start, stop = 0, start parts = f", {int(partitions)}" if partitions is not None else "" - sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" # noqa: S608 + sql = f"SELECT * FROM range({int(start)}, {int(stop)}, {int(step)}{parts})" return ctx.sql(sql) From 0130a722241968be6165b5959234d4a8040ada03 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 16:47:28 +0800 Subject: [PATCH 05/67] feat: ensure proper resource management in DataFrame streaming --- python/datafusion/dataframe.py | 5 +- python/tests/test_dataframe_iter_stream.py | 55 ++++++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 python/tests/test_dataframe_iter_stream.py diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index a196e38a3..27414a782 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1130,10 +1130,13 @@ def __iter__(self) -> Iterator[pa.RecordBatch]: 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__()) - yield from reader + 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. diff --git a/python/tests/test_dataframe_iter_stream.py b/python/tests/test_dataframe_iter_stream.py new file mode 100644 index 000000000..8f1d20d82 --- /dev/null +++ b/python/tests/test_dataframe_iter_stream.py @@ -0,0 +1,55 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +import pyarrow as pa + + +def test_iter_releases_reader(monkeypatch, ctx): + batches = [ + pa.RecordBatch.from_pydict({"a": [1]}), + pa.RecordBatch.from_pydict({"a": [2]}), + ] + + class DummyReader: + def __init__(self, batches): + self._iter = iter(batches) + self.closed = False + + def __iter__(self): + return self + + def __next__(self): + return next(self._iter) + + def close(self): + self.closed = True + + dummy_reader = DummyReader(batches) + + class FakeRecordBatchReader: + @staticmethod + def _import_from_c_capsule(*_args, **_kwargs): + return dummy_reader + + monkeypatch.setattr(pa, "RecordBatchReader", FakeRecordBatchReader) + + df = ctx.from_pydict({"a": [1, 2]}) + + for _ in df: + break + + assert dummy_reader.closed From 03e530c9fb1abeef17f8893c82ae01a7853f34aa Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 16:49:04 +0800 Subject: [PATCH 06/67] refactor: replace spawn_stream and spawn_streams with spawn_future for consistency --- src/context.rs | 4 ++-- src/dataframe.rs | 10 +++++----- src/utils.rs | 24 +----------------------- 3 files changed, 8 insertions(+), 30 deletions(-) diff --git a/src/context.rs b/src/context.rs index 3bc9f2989..561fb37fa 100644 --- a/src/context.rs +++ b/src/context.rs @@ -45,7 +45,7 @@ use crate::udaf::PyAggregateUDF; use crate::udf::PyScalarUDF; use crate::udtf::PyTableFunction; use crate::udwf::PyWindowUDF; -use crate::utils::{get_global_ctx, spawn_stream, validate_pycapsule, wait_for_future}; +use crate::utils::{get_global_ctx, spawn_future, validate_pycapsule, wait_for_future}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::arrow::record_batch::RecordBatch; @@ -1131,7 +1131,7 @@ impl PySessionContext { ) -> PyDataFusionResult { let ctx: TaskContext = TaskContext::from(&self.ctx.state()); let plan = plan.plan.clone(); - let stream = spawn_stream(py, async move { plan.execute(part, Arc::new(ctx)) })?; + let stream = spawn_future(py, async move { plan.execute(part, Arc::new(ctx)) })?; Ok(PyRecordBatchStream::new(stream)) } } diff --git a/src/dataframe.rs b/src/dataframe.rs index fe03e98b3..f7a0bfadf 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -51,8 +51,8 @@ use crate::physical_plan::PyExecutionPlan; use crate::record_batch::{poll_next_batch, PyRecordBatchStream}; use crate::sql::logical::PyLogicalPlan; use crate::utils::{ - get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, spawn_stream, spawn_streams, - validate_pycapsule, wait_for_future, + get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, spawn_future, validate_pycapsule, + wait_for_future, }; use crate::{ errors::PyDataFusionResult, @@ -967,7 +967,7 @@ impl PyDataFrame { requested_schema: Option>, ) -> PyDataFusionResult> { let df = self.df.as_ref().clone(); - let streams = spawn_streams(py, async move { df.execute_stream_partitioned().await })?; + let streams = spawn_future(py, async move { df.execute_stream_partitioned().await })?; let mut schema: Schema = self.df.schema().to_owned().into(); let mut projection: Option = None; @@ -1020,13 +1020,13 @@ impl PyDataFrame { fn execute_stream(&self, py: Python) -> PyDataFusionResult { let df = self.df.as_ref().clone(); - let stream = spawn_stream(py, async move { df.execute_stream().await })?; + let stream = spawn_future(py, async move { df.execute_stream().await })?; Ok(PyRecordBatchStream::new(stream)) } fn execute_stream_partitioned(&self, py: Python) -> PyResult> { let df = self.df.as_ref().clone(); - let streams = spawn_streams(py, async move { df.execute_stream_partitioned().await })?; + let streams = spawn_future(py, async move { df.execute_stream_partitioned().await })?; Ok(streams.into_iter().map(PyRecordBatchStream::new).collect()) } diff --git a/src/utils.rs b/src/utils.rs index 5ee648389..483095d3c 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -21,8 +21,7 @@ use crate::{ TokioRuntime, }; use datafusion::{ - common::ScalarValue, execution::context::SessionContext, execution::SendableRecordBatchStream, - logical_expr::Volatility, + common::ScalarValue, execution::context::SessionContext, logical_expr::Volatility, }; use pyo3::prelude::*; use pyo3::{exceptions::PyValueError, types::PyCapsule}; @@ -114,27 +113,6 @@ where Ok(inner_result?) } -/// Spawn a [`SendableRecordBatchStream`] on the Tokio runtime and wait for completion -/// while respecting Python signal handling. -pub(crate) fn spawn_stream(py: Python, fut: F) -> PyDataFusionResult -where - F: Future> + Send + 'static, -{ - spawn_future(py, fut) -} - -/// Spawn a partitioned [`SendableRecordBatchStream`] on the Tokio runtime and wait for completion -/// while respecting Python signal handling. -pub(crate) fn spawn_streams( - py: Python, - fut: F, -) -> PyDataFusionResult> -where - F: Future>> + Send + 'static, -{ - spawn_future(py, fut) -} - pub(crate) fn parse_volatility(value: &str) -> PyDataFusionResult { Ok(match value { "immutable" => Volatility::Immutable, From 4a3f17d33956fd7910729d7dcd8ce58323faf88e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 18:01:55 +0800 Subject: [PATCH 07/67] feat: add test for Arrow C stream schema selection in DataFrame --- python/tests/test_dataframe.py | 40 ++++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index cf8e2e5b7..990359b55 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -46,6 +46,8 @@ from datafusion.expr import Window from pyarrow.csv import write_csv +pa_cffi = pytest.importorskip("pyarrow.cffi") + MB = 1024 * 1024 @@ -1637,6 +1639,44 @@ def test_arrow_c_stream_reader(df): assert table.equals(expected) +def test_arrow_c_stream_schema_selection(fail_collect): + ctx = SessionContext() + + batch = pa.RecordBatch.from_arrays( + [ + pa.array([1, 2]), + pa.array([3, 4]), + pa.array([5, 6]), + ], + names=["a", "b", "c"], + ) + df = ctx.create_dataframe([[batch]]) + + requested_schema = pa.schema([("c", pa.int64()), ("a", pa.int64())]) + + c_schema = pa_cffi.ffi.new("struct ArrowSchema*") + address = int(pa_cffi.ffi.cast("uintptr_t", c_schema)) + requested_schema._export_to_c(address) + capsule_new = ctypes.pythonapi.PyCapsule_New + capsule_new.restype = ctypes.py_object + capsule_new.argtypes = [ctypes.c_void_p, ctypes.c_char_p, ctypes.c_void_p] + schema_capsule = capsule_new(ctypes.c_void_p(address), b"arrow_schema", None) + + reader = pa.RecordBatchReader._import_from_c_capsule( + df.__arrow_c_stream__(schema_capsule) + ) + + assert reader.schema == requested_schema + + batches = list(reader) + + assert len(batches) == 1 + expected_batch = pa.record_batch( + [pa.array([5, 6]), pa.array([1, 2])], names=["c", "a"] + ) + assert batches[0].equals(expected_batch) + + def test_to_pylist(df): # Convert datafusion dataframe to Python list pylist = df.to_pylist() From f7a24074a363903c41b10d00f6e660b8e31f5e5e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 18:37:24 +0800 Subject: [PATCH 08/67] test: rename and extend test_arrow_c_stream_to_table to include RecordBatchReader validation --- python/tests/test_dataframe.py | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 990359b55..b7cbb4919 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1596,7 +1596,7 @@ def test_iter_batches_dataframe(fail_collect): assert got.equals(exp) -def test_arrow_c_stream_to_table(fail_collect): +def test_arrow_c_stream_to_table_and_reader(fail_collect): ctx = SessionContext() # Create a DataFrame with two separate record batches @@ -1613,6 +1613,12 @@ def test_arrow_c_stream_to_table(fail_collect): assert table.schema == df.schema() assert table.column("a").num_chunks == 2 + reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) + assert isinstance(reader, pa.RecordBatchReader) + reader_table = pa.Table.from_batches(reader) + expected = pa.Table.from_batches([batch1, batch2]) + assert reader_table.equals(expected) + def test_arrow_c_stream_order(): ctx = SessionContext() @@ -1631,14 +1637,6 @@ def test_arrow_c_stream_order(): assert col.chunk(1)[0].as_py() == 2 -def test_arrow_c_stream_reader(df): - reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) - assert isinstance(reader, pa.RecordBatchReader) - table = pa.Table.from_batches(reader) - expected = pa.Table.from_batches(df.collect()) - assert table.equals(expected) - - def test_arrow_c_stream_schema_selection(fail_collect): ctx = SessionContext() From b1d18a8d862f0ffea75240bd207245cd686aeb36 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 18:51:18 +0800 Subject: [PATCH 09/67] test: add validation for schema mismatch in Arrow C stream --- python/tests/test_dataframe.py | 23 +++++++++++++++++++++++ python/tests/utils.py | 18 +++++++++++++++++- 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index b7cbb4919..27af0dacc 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1675,6 +1675,29 @@ def test_arrow_c_stream_schema_selection(fail_collect): assert batches[0].equals(expected_batch) +def test_arrow_c_stream_schema_mismatch(fail_collect): + ctx = SessionContext() + + batch = pa.RecordBatch.from_arrays( + [pa.array([1, 2]), pa.array([3, 4])], names=["a", "b"] + ) + df = ctx.create_dataframe([[batch]]) + + bad_schema = pa.schema([("a", pa.string())]) + + c_schema = pa_cffi.ffi.new("struct ArrowSchema*") + address = int(pa_cffi.ffi.cast("uintptr_t", c_schema)) + bad_schema._export_to_c(address) + + capsule_new = ctypes.pythonapi.PyCapsule_New + capsule_new.restype = ctypes.py_object + capsule_new.argtypes = [ctypes.c_void_p, ctypes.c_char_p, ctypes.c_void_p] + bad_capsule = capsule_new(ctypes.c_void_p(address), b"arrow_schema", None) + + with pytest.raises(Exception, match="Fail to merge schema"): + df.__arrow_c_stream__(bad_capsule) + + def test_to_pylist(df): # Convert datafusion dataframe to Python list pylist = df.to_pylist() diff --git a/python/tests/utils.py b/python/tests/utils.py index 612f666d4..c085cb76a 100644 --- a/python/tests/utils.py +++ b/python/tests/utils.py @@ -1,10 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + """Testing-only helpers for datafusion-python. This module contains utilities used by the test-suite that should not be exposed as part of the public API. Keep the implementation minimal and documented so reviewers can easily see it's test-only. """ - from __future__ import annotations from typing import TYPE_CHECKING From eeb2a37cce5819cbf5d5ecc225682c7ca9099a0f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 2 Sep 2025 20:54:47 +0800 Subject: [PATCH 10/67] fix Ruff errors --- python/tests/utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/tests/utils.py b/python/tests/utils.py index c085cb76a..00efb6555 100644 --- a/python/tests/utils.py +++ b/python/tests/utils.py @@ -21,6 +21,7 @@ exposed as part of the public API. Keep the implementation minimal and documented so reviewers can easily see it's test-only. """ + from __future__ import annotations from typing import TYPE_CHECKING From 748b7e2ebe99c2490897d37c75b77733864b4a4a Mon Sep 17 00:00:00 2001 From: kosiew Date: Sun, 7 Sep 2025 18:43:05 +0800 Subject: [PATCH 11/67] Update docs/source/user-guide/dataframe/index.rst Co-authored-by: Kyle Barron --- docs/source/user-guide/dataframe/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 37f559186..604d156d4 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -164,7 +164,7 @@ out-of-memory errors. import pyarrow as pa # Create a PyArrow RecordBatchReader without materializing all batches - reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) + reader = pa.RecordBatchReader.from_stream(df) for batch in reader: ... # process each batch as it is produced From 5e650aa5827b0fdfa173099f579a3673c259cb9e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 20:17:28 +0800 Subject: [PATCH 12/67] test: add batch iteration test for DataFrame --- python/tests/test_dataframe.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 27af0dacc..e7a6e441e 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -379,6 +379,23 @@ def test_cast(df): assert df.schema() == expected +def test_iter_batches(df): + batches = [] + for batch in df: + batches.append(batch) # noqa: PERF402 + + # Delete DataFrame to ensure RecordBatches remain valid + del df + + assert len(batches) == 1 + + batch = batches[0] + assert isinstance(batch, pa.RecordBatch) + assert batch.column(0).to_pylist() == [1, 2, 3] + assert batch.column(1).to_pylist() == [4, 5, 6] + assert batch.column(2).to_pylist() == [8, 5, 8] + + def test_with_column_renamed(df): df = df.with_column("c", column("a") + column("b")).with_column_renamed("c", "sum") From ebd2191497c5cf09cec2f52168e77159fc6e1ce1 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 18:39:03 +0800 Subject: [PATCH 13/67] refactor: simplify stream capsule creation in PyDataFrame --- src/dataframe.rs | 58 ++++-------------------------------------------- 1 file changed, 4 insertions(+), 54 deletions(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index f7a0bfadf..e685f388b 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -16,7 +16,7 @@ // under the License. use std::collections::HashMap; -use std::ffi::{c_void, CStr, CString}; +use std::ffi::CString; use std::sync::Arc; use arrow::array::{new_null_array, RecordBatch, RecordBatchReader}; @@ -39,7 +39,6 @@ use datafusion::prelude::*; use datafusion_ffi::table_provider::FFI_TableProvider; use futures::{StreamExt, TryStreamExt}; use pyo3::exceptions::PyValueError; -use pyo3::ffi; use pyo3::prelude::*; use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods}; @@ -59,35 +58,6 @@ use crate::{ expr::{sort_expr::PySortExpr, PyExpr}, }; -#[allow(clippy::manual_c_str_literals)] -static ARROW_STREAM_NAME: &CStr = - unsafe { CStr::from_bytes_with_nul_unchecked(b"arrow_array_stream\0") }; - -unsafe extern "C" fn drop_stream(capsule: *mut ffi::PyObject) { - if capsule.is_null() { - return; - } - - // When PyArrow imports this capsule it steals the raw stream pointer and - // sets the capsule's internal pointer to NULL. In that case - // `PyCapsule_IsValid` returns 0 and this destructor must not drop the - // stream as ownership has been transferred to PyArrow. If the capsule was - // never imported, the pointer remains valid and we are responsible for - // freeing the stream here. - if ffi::PyCapsule_IsValid(capsule, ARROW_STREAM_NAME.as_ptr()) == 1 { - let stream_ptr = ffi::PyCapsule_GetPointer(capsule, ARROW_STREAM_NAME.as_ptr()) - as *mut FFI_ArrowArrayStream; - if !stream_ptr.is_null() { - drop(Box::from_raw(stream_ptr)); - } - } - - // `PyCapsule_GetPointer` sets a Python error on failure. Clear it only - // after the stream has been released (or determined to be owned - // elsewhere). - ffi::PyErr_Clear(); -} - // https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 // - we have not decided on the table_provider approach yet // this is an interim implementation @@ -993,29 +963,9 @@ impl PyDataFrame { let reader: Box = Box::new(reader); let stream = Box::new(FFI_ArrowArrayStream::new(reader)); - let stream_ptr = Box::into_raw(stream); - debug_assert!( - !stream_ptr.is_null(), - "ArrowArrayStream pointer should never be null", - ); - // The returned capsule allows zero-copy hand-off to PyArrow. When - // PyArrow imports the capsule it assumes ownership of the stream and - // nulls out the capsule's internal pointer so `drop_stream` knows not to - // free it. - let capsule = unsafe { - ffi::PyCapsule_New( - stream_ptr as *mut c_void, - ARROW_STREAM_NAME.as_ptr(), - Some(drop_stream), - ) - }; - if capsule.is_null() { - unsafe { drop(Box::from_raw(stream_ptr)) }; - Err(PyErr::fetch(py).into()) - } else { - let any = unsafe { Bound::from_owned_ptr(py, capsule) }; - Ok(any.downcast_into::().unwrap()) - } + + let stream_capsule_name = CString::new("arrow_array_stream").unwrap(); + Ok(PyCapsule::new(py, stream, Some(stream_capsule_name))?) } fn execute_stream(&self, py: Python) -> PyDataFusionResult { From 6bae74b96f4a68159becab772ffc287135699d22 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 20:41:01 +0800 Subject: [PATCH 14/67] refactor: enhance stream capsule management in PyDataFrame --- src/dataframe.rs | 52 ++++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 48 insertions(+), 4 deletions(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index e685f388b..badfc90c3 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -16,7 +16,7 @@ // under the License. use std::collections::HashMap; -use std::ffi::CString; +use std::ffi::{c_void, CString}; use std::sync::Arc; use arrow::array::{new_null_array, RecordBatch, RecordBatchReader}; @@ -42,6 +42,7 @@ use pyo3::exceptions::PyValueError; use pyo3::prelude::*; use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods}; +use pyo3::PyErr; use crate::catalog::PyTable; use crate::errors::{py_datafusion_err, PyDataFusionError}; @@ -962,10 +963,53 @@ impl PyDataFrame { }; let reader: Box = Box::new(reader); - let stream = Box::new(FFI_ArrowArrayStream::new(reader)); + // Create a stream and transfer it to a raw pointer. The capsule takes + // ownership and is responsible for freeing the stream unless PyArrow + // steals it. PyArrow will set the capsule's pointer to NULL when it + // imports the stream, signaling that it now owns the resources. + let raw_stream = Box::into_raw(Box::new(FFI_ArrowArrayStream::new(reader))); + + // Name used both for capsule creation and lookup in the destructor. + const STREAM_NAME: &[u8] = b"arrow_array_stream\0"; + + unsafe extern "C" fn drop_stream_capsule(capsule: *mut pyo3::ffi::PyObject) { + // Attempt to recover the raw stream pointer. If PyArrow imported the + // stream it will have set the capsule pointer to NULL, in which case + // `PyCapsule_GetPointer` returns NULL and we simply clear the error. + let ptr = pyo3::ffi::PyCapsule_GetPointer(capsule, STREAM_NAME.as_ptr() as *const _) + as *mut FFI_ArrowArrayStream; + + if ptr.is_null() { + // Ignore any exception raised by `PyCapsule_GetPointer` when the + // pointer is already NULL. + pyo3::ffi::PyErr_Clear(); + } else { + // Reconstruct the Box and drop it so resources are released. + drop(Box::from_raw(ptr)); + } + } + + let capsule_ptr = unsafe { + pyo3::ffi::PyCapsule_New( + raw_stream as *mut c_void, + STREAM_NAME.as_ptr() as *const std::ffi::c_char, + Some(drop_stream_capsule), + ) + }; - let stream_capsule_name = CString::new("arrow_array_stream").unwrap(); - Ok(PyCapsule::new(py, stream, Some(stream_capsule_name))?) + if capsule_ptr.is_null() { + // Reclaim ownership to avoid leaking on failure + unsafe { + drop(Box::from_raw(raw_stream)); + } + return Err(PyErr::fetch(py).into()); + } + + // Safety: `capsule_ptr` is a new reference from `PyCapsule_New` + let capsule = unsafe { + Bound::from_owned_ptr(py, capsule_ptr).downcast_into_unchecked::() + }; + Ok(capsule) } fn execute_stream(&self, py: Python) -> PyDataFusionResult { From f0cbe068868f18cc76a82b1eabdf017a1f6e0514 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 22:05:47 +0800 Subject: [PATCH 15/67] refactor: enhance DataFrame and RecordBatchStream iteration support --- docs/source/user-guide/dataframe/index.rst | 23 ++++++++++++++++-- python/datafusion/dataframe.py | 27 +++++++++------------- python/datafusion/record_batch.py | 23 +++++++++++++++++- 3 files changed, 54 insertions(+), 19 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 604d156d4..1779f0b81 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -168,14 +168,33 @@ out-of-memory errors. 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: +DataFrames are also iterable, yielding :class:`datafusion.RecordBatch` +objects lazily so you can loop over results directly without importing +PyArrow: .. code-block:: python for batch in df: + ... # each batch is a ``RecordBatch`` + +Asynchronous iteration is supported as well, allowing integration with +``asyncio`` event loops: + +.. code-block:: python + + async for batch in df: ... # process each batch as it is produced +To work with the stream directly, use +``to_record_batch_stream()``, which returns a +:class:`~datafusion.RecordBatchStream`: + +.. code-block:: python + + stream = df.to_record_batch_stream() + for batch in stream: + ... + See :doc:`../io/arrow` for additional details on the Arrow interface. HTML Rendering diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 27414a782..ed8c8ea36 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -43,7 +43,7 @@ from datafusion._internal import ParquetWriterOptions as ParquetWriterOptionsInternal from datafusion.expr import Expr, SortExpr, sort_or_default from datafusion.plan import ExecutionPlan, LogicalPlan -from datafusion.record_batch import RecordBatchStream +from datafusion.record_batch import RecordBatch, RecordBatchStream if TYPE_CHECKING: import pathlib @@ -1030,6 +1030,10 @@ def execute_stream(self) -> RecordBatchStream: """ return RecordBatchStream(self.df.execute_stream()) + def to_record_batch_stream(self) -> RecordBatchStream: + """Return a :class:`RecordBatchStream` executing this DataFrame.""" + return self.execute_stream() + def execute_stream_partitioned(self) -> list[RecordBatchStream]: """Executes this DataFrame and returns a stream for each partition. @@ -1121,22 +1125,13 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: # 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 + def __iter__(self) -> Iterator[RecordBatch]: + """Yield :class:`RecordBatch` objects by streaming execution.""" + yield from self.to_record_batch_stream() - reader = pa.RecordBatchReader._import_from_c_capsule(self.__arrow_c_stream__()) - with closing(reader): - yield from reader + async def __aiter__(self) -> RecordBatchStream: + """Return an asynchronous iterator over streamed ``RecordBatch`` objects.""" + return await self.to_record_batch_stream().__aiter__() def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. diff --git a/python/datafusion/record_batch.py b/python/datafusion/record_batch.py index 556eaa786..7934de4d3 100644 --- a/python/datafusion/record_batch.py +++ b/python/datafusion/record_batch.py @@ -46,6 +46,26 @@ def to_pyarrow(self) -> pa.RecordBatch: """Convert to :py:class:`pa.RecordBatch`.""" return self.record_batch.to_pyarrow() + def __arrow_c_array__( + self, requested_schema: object | None = None + ) -> tuple[object, object]: + """Export the record batch via the Arrow C Data Interface. + + This allows zero-copy interchange with libraries that support the + `Arrow PyCapsule interface `_. + + Args: + requested_schema: Attempt to provide the record batch using this + schema. Only straightforward projections such as column + selection or reordering are applied. + + Returns: + Two Arrow PyCapsule objects representing the ``ArrowArray`` and + ``ArrowSchema``. + """ + return self.record_batch.__arrow_c_array__(requested_schema) + class RecordBatchStream: """This class represents a stream of record batches. @@ -72,8 +92,9 @@ def __next__(self) -> RecordBatch: next_batch = next(self.rbs) return RecordBatch(next_batch) - def __aiter__(self) -> typing_extensions.Self: + async def __aiter__(self) -> typing_extensions.Self: """Async iterator function.""" + await self.rbs.__aiter__() return self def __iter__(self) -> typing_extensions.Self: From 295d04a290a31c1bd3ec914f5baf73ccba59452b Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 22:07:40 +0800 Subject: [PATCH 16/67] refactor: improve docstrings for DataFrame and RecordBatchStream methods --- docs/source/conf.py | 6 ++++++ python/datafusion/dataframe.py | 2 +- python/datafusion/record_batch.py | 11 +++++------ 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/docs/source/conf.py b/docs/source/conf.py index 28db17d35..e7b27a769 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -72,6 +72,12 @@ suppress_warnings = ["autoapi.python_import_resolution"] autoapi_python_class_content = "both" autoapi_keep_files = False # set to True for debugging generated files +autoapi_options = [ + "members", + "undoc-members", + "special-members", + "show-inheritance", +] def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa: ARG001 diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index ed8c8ea36..036dc699c 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -309,7 +309,7 @@ def into_view(self) -> pa.Table: return self.df.into_view() def __getitem__(self, key: str | list[str]) -> DataFrame: - """Return a new :py:class`DataFrame` with the specified column or columns. + """Return a new :py:class:`DataFrame` with the specified column or columns. Args: key: Column name or list of column names to select. diff --git a/python/datafusion/record_batch.py b/python/datafusion/record_batch.py index 7934de4d3..c24cde0ac 100644 --- a/python/datafusion/record_batch.py +++ b/python/datafusion/record_batch.py @@ -83,20 +83,19 @@ def next(self) -> RecordBatch: return next(self) async def __anext__(self) -> RecordBatch: - """Async iterator function.""" + """Return the next :py:class:`RecordBatch` in the stream asynchronously.""" next_batch = await self.rbs.__anext__() return RecordBatch(next_batch) def __next__(self) -> RecordBatch: - """Iterator function.""" + """Return the next :py:class:`RecordBatch` in the stream.""" next_batch = next(self.rbs) return RecordBatch(next_batch) - async def __aiter__(self) -> typing_extensions.Self: - """Async iterator function.""" - await self.rbs.__aiter__() + def __aiter__(self) -> typing_extensions.Self: + """Return an asynchronous iterator over record batches.""" return self def __iter__(self) -> typing_extensions.Self: - """Iterator function.""" + """Return an iterator over record batches.""" return self From 475c031da78e8df7dc5401ab795872932ffa6ddc Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 22:09:27 +0800 Subject: [PATCH 17/67] refactor: add to_record_batch_stream method and improve iteration support in DataFrame --- python/datafusion/dataframe.py | 20 ++++++-- python/tests/test_dataframe.py | 24 ++++++++-- python/tests/test_dataframe_iter_stream.py | 55 ---------------------- 3 files changed, 34 insertions(+), 65 deletions(-) delete mode 100644 python/tests/test_dataframe_iter_stream.py diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 036dc699c..0cefb237b 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -25,6 +25,7 @@ from typing import ( TYPE_CHECKING, Any, + AsyncIterator, Iterable, Iterator, Literal, @@ -1043,6 +1044,15 @@ def execute_stream_partitioned(self) -> list[RecordBatchStream]: streams = self.df.execute_stream_partitioned() return [RecordBatchStream(rbs) for rbs in streams] + def to_record_batch_stream(self) -> RecordBatchStream: + """Return a :py:class:`RecordBatchStream` over this DataFrame's results. + + Returns: + A ``RecordBatchStream`` representing the lazily generated record + batches for this DataFrame. + """ + return self.execute_stream() + def to_pandas(self) -> pd.DataFrame: """Execute the :py:class:`DataFrame` and convert it into a Pandas DataFrame. @@ -1126,12 +1136,12 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: return self.df.__arrow_c_stream__(requested_schema) def __iter__(self) -> Iterator[RecordBatch]: - """Yield :class:`RecordBatch` objects by streaming execution.""" - yield from self.to_record_batch_stream() + """Return an iterator over this DataFrame's record batches.""" + return iter(self.to_record_batch_stream()) - async def __aiter__(self) -> RecordBatchStream: - """Return an asynchronous iterator over streamed ``RecordBatch`` objects.""" - return await self.to_record_batch_stream().__aiter__() + def __aiter__(self) -> AsyncIterator[RecordBatch]: + """Return an async iterator over this DataFrame's record batches.""" + return self.to_record_batch_stream().__aiter__() def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index e7a6e441e..cb44d15c2 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -29,6 +29,7 @@ DataFrame, ParquetColumnOptions, ParquetWriterOptions, + RecordBatch, SessionContext, WindowFrame, column, @@ -390,10 +391,23 @@ def test_iter_batches(df): assert len(batches) == 1 batch = batches[0] - assert isinstance(batch, pa.RecordBatch) - assert batch.column(0).to_pylist() == [1, 2, 3] - assert batch.column(1).to_pylist() == [4, 5, 6] - assert batch.column(2).to_pylist() == [8, 5, 8] + assert isinstance(batch, RecordBatch) + pa_batch = batch.to_pyarrow() + assert pa_batch.column(0).to_pylist() == [1, 2, 3] + assert pa_batch.column(1).to_pylist() == [4, 5, 6] + assert pa_batch.column(2).to_pylist() == [8, 5, 8] + + +def test_to_record_batch_stream(df): + stream = df.to_record_batch_stream() + batches = list(stream) + + assert len(batches) == 1 + assert isinstance(batches[0], RecordBatch) + pa_batch = batches[0].to_pyarrow() + assert pa_batch.column(0).to_pylist() == [1, 2, 3] + assert pa_batch.column(1).to_pylist() == [4, 5, 6] + assert pa_batch.column(2).to_pylist() == [8, 5, 8] def test_with_column_renamed(df): @@ -1331,7 +1345,7 @@ def test_execution_plan(aggregate_df): @pytest.mark.asyncio async def test_async_iteration_of_df(aggregate_df): rows_returned = 0 - async for batch in aggregate_df.execute_stream(): + async for batch in aggregate_df: assert batch is not None rows_returned += len(batch.to_pyarrow()[0]) diff --git a/python/tests/test_dataframe_iter_stream.py b/python/tests/test_dataframe_iter_stream.py deleted file mode 100644 index 8f1d20d82..000000000 --- a/python/tests/test_dataframe_iter_stream.py +++ /dev/null @@ -1,55 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -import pyarrow as pa - - -def test_iter_releases_reader(monkeypatch, ctx): - batches = [ - pa.RecordBatch.from_pydict({"a": [1]}), - pa.RecordBatch.from_pydict({"a": [2]}), - ] - - class DummyReader: - def __init__(self, batches): - self._iter = iter(batches) - self.closed = False - - def __iter__(self): - return self - - def __next__(self): - return next(self._iter) - - def close(self): - self.closed = True - - dummy_reader = DummyReader(batches) - - class FakeRecordBatchReader: - @staticmethod - def _import_from_c_capsule(*_args, **_kwargs): - return dummy_reader - - monkeypatch.setattr(pa, "RecordBatchReader", FakeRecordBatchReader) - - df = ctx.from_pydict({"a": [1, 2]}) - - for _ in df: - break - - assert dummy_reader.closed From 06c9fc7256c97266a562aa2a513bc7c2a698d7c8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 23:02:21 +0800 Subject: [PATCH 18/67] test: update test_iter_batches_dataframe to assert RecordBatch type and conversion --- python/tests/test_dataframe.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index cb44d15c2..75d7122dd 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1624,7 +1624,8 @@ def test_iter_batches_dataframe(fail_collect): expected = [batch1, batch2] for got, exp in zip(df, expected): - assert got.equals(exp) + assert isinstance(got, RecordBatch) + assert got.to_pyarrow().equals(exp) def test_arrow_c_stream_to_table_and_reader(fail_collect): From 94432b546e8676503b09315ba02e571ec48e962c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 23:02:55 +0800 Subject: [PATCH 19/67] fix: update table creation from batches to use to_pyarrow conversion --- python/tests/test_dataframe.py | 4 ++-- python/tests/test_io.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 75d7122dd..2cd7731c4 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1636,7 +1636,7 @@ def test_arrow_c_stream_to_table_and_reader(fail_collect): batch2 = pa.record_batch([pa.array([2])], names=["a"]) df = ctx.create_dataframe([[batch1], [batch2]]) - table = pa.Table.from_batches(df) + table = pa.Table.from_batches(batch.to_pyarrow() for batch in df) batches = table.to_batches() assert len(batches) == 2 @@ -1660,7 +1660,7 @@ def test_arrow_c_stream_order(): df = ctx.create_dataframe([[batch1, batch2]]) - table = pa.Table.from_batches(df) + table = pa.Table.from_batches(batch.to_pyarrow() for batch in df) expected = pa.Table.from_batches([batch1, batch2]) assert table.equals(expected) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index d89578cd9..d292b59bc 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -128,6 +128,6 @@ def test_arrow_c_stream_large_dataset(ctx): def test_table_from_batches_stream(ctx, fail_collect): df = range_table(ctx, 0, 10) - table = pa.Table.from_batches(df) + table = pa.Table.from_batches(batch.to_pyarrow() for batch in df) assert table.shape == (10, 1) assert table.column_names == ["value"] From 31ed8e7770c66254f0f7379553c126f19c66a80f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 23:03:25 +0800 Subject: [PATCH 20/67] test: add test_iter_returns_datafusion_recordbatch to verify RecordBatch type --- python/tests/test_dataframe.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 2cd7731c4..8e37a828a 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -45,6 +45,7 @@ reset_formatter, ) from datafusion.expr import Window +from datafusion.record_batch import RecordBatch as DataFusionRecordBatch from pyarrow.csv import write_csv pa_cffi = pytest.importorskip("pyarrow.cffi") @@ -398,6 +399,11 @@ def test_iter_batches(df): assert pa_batch.column(2).to_pylist() == [8, 5, 8] +def test_iter_returns_datafusion_recordbatch(df): + for batch in df: + assert isinstance(batch, DataFusionRecordBatch) + + def test_to_record_batch_stream(df): stream = df.to_record_batch_stream() batches = list(stream) From 610aed3b46e3a24aa95db129273c46a0186c76df Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 8 Sep 2025 23:03:53 +0800 Subject: [PATCH 21/67] docs: clarify RecordBatch reference and add PyArrow conversion example --- docs/source/user-guide/dataframe/index.rst | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 1779f0b81..03ff1977b 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -175,7 +175,15 @@ PyArrow: .. code-block:: python for batch in df: - ... # each batch is a ``RecordBatch`` + ... # each batch is a ``datafusion.RecordBatch`` + +Each batch exposes ``to_pyarrow()``, allowing conversion to a PyArrow +table without collecting everything eagerly: + +.. code-block:: python + + import pyarrow as pa + table = pa.Table.from_batches(b.to_pyarrow() for b in df) Asynchronous iteration is supported as well, allowing integration with ``asyncio`` event loops: From 1ebd3c17c4e8dabfba8187e0c9e83a683a416c1c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 9 Sep 2025 07:50:19 +0800 Subject: [PATCH 22/67] test: improve test_iter_batches_dataframe to validate RecordBatch conversion --- python/tests/test_dataframe.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 8e37a828a..2ff33a1a9 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1629,9 +1629,11 @@ def test_iter_batches_dataframe(fail_collect): df = ctx.create_dataframe([[batch1], [batch2]]) expected = [batch1, batch2] - for got, exp in zip(df, expected): - assert isinstance(got, RecordBatch) - assert got.to_pyarrow().equals(exp) + results = [b.to_pyarrow() for b in df] + + assert len(results) == len(expected) + for exp in expected: + assert any(got.equals(exp) for got in results) def test_arrow_c_stream_to_table_and_reader(fail_collect): From 2e4b963496599f0de1a7640b9579b05c8b8c3239 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 9 Sep 2025 08:44:50 +0800 Subject: [PATCH 23/67] test: enhance test_arrow_c_stream_to_table_and_reader for batch equality validation --- python/tests/test_dataframe.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 2ff33a1a9..ae09a90ad 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1648,8 +1648,9 @@ def test_arrow_c_stream_to_table_and_reader(fail_collect): batches = table.to_batches() assert len(batches) == 2 - assert batches[0].equals(batch1) - assert batches[1].equals(batch2) + expected = [batch1, batch2] + for exp in expected: + assert any(got.equals(exp) for got in batches) assert table.schema == df.schema() assert table.column("a").num_chunks == 2 From d0ee865dc9df83809f1929fa1171a0267bf93813 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 9 Sep 2025 21:18:46 +0800 Subject: [PATCH 24/67] Shelve unrelated changes --- docs/source/conf.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docs/source/conf.py b/docs/source/conf.py index e7b27a769..28db17d35 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -72,12 +72,6 @@ suppress_warnings = ["autoapi.python_import_resolution"] autoapi_python_class_content = "both" autoapi_keep_files = False # set to True for debugging generated files -autoapi_options = [ - "members", - "undoc-members", - "special-members", - "show-inheritance", -] def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa: ARG001 From 16a249c6da3461597d46e357d3c80e461f01c121 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 9 Sep 2025 21:33:41 +0800 Subject: [PATCH 25/67] Fix documentation to reference datafusion.RecordBatch instead of pyarrow.RecordBatch --- python/datafusion/dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 0cefb237b..71cd8f537 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -292,7 +292,7 @@ class DataFrame: """Two dimensional table representation of data. DataFrame objects are iterable; iterating over a DataFrame yields - :class:`pyarrow.RecordBatch` instances lazily. + :class:`datafusion.RecordBatch` instances lazily. See :ref:`user_guide_concepts` in the online documentation for more information. """ @@ -1044,7 +1044,7 @@ def execute_stream_partitioned(self) -> list[RecordBatchStream]: streams = self.df.execute_stream_partitioned() return [RecordBatchStream(rbs) for rbs in streams] - def to_record_batch_stream(self) -> RecordBatchStream: + def to_record_batch_stream(self) -> RecordBatchStream: # noqa: F811 """Return a :py:class:`RecordBatchStream` over this DataFrame's results. Returns: From d91ecfad2f0267fc58c63f15a6cd2a0444b3b37e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 9 Sep 2025 21:46:05 +0800 Subject: [PATCH 26/67] Remove redundant to_record_batch_stream method from DataFrame class --- python/datafusion/dataframe.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 71cd8f537..492e0ca55 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1031,10 +1031,6 @@ def execute_stream(self) -> RecordBatchStream: """ return RecordBatchStream(self.df.execute_stream()) - def to_record_batch_stream(self) -> RecordBatchStream: - """Return a :class:`RecordBatchStream` executing this DataFrame.""" - return self.execute_stream() - def execute_stream_partitioned(self) -> list[RecordBatchStream]: """Executes this DataFrame and returns a stream for each partition. From 21f286a03b2f75f14777601cbb426ebc713e6be2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 12:56:10 +0800 Subject: [PATCH 27/67] Refactor Arrow stream creation in PyDataFrame to use PyCapsule directly --- src/dataframe.rs | 54 +++++++----------------------------------------- 1 file changed, 7 insertions(+), 47 deletions(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index badfc90c3..f1772e262 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -16,7 +16,7 @@ // under the License. use std::collections::HashMap; -use std::ffi::{c_void, CString}; +use std::ffi::CString; use std::sync::Arc; use arrow::array::{new_null_array, RecordBatch, RecordBatchReader}; @@ -963,52 +963,12 @@ impl PyDataFrame { }; let reader: Box = Box::new(reader); - // Create a stream and transfer it to a raw pointer. The capsule takes - // ownership and is responsible for freeing the stream unless PyArrow - // steals it. PyArrow will set the capsule's pointer to NULL when it - // imports the stream, signaling that it now owns the resources. - let raw_stream = Box::into_raw(Box::new(FFI_ArrowArrayStream::new(reader))); - - // Name used both for capsule creation and lookup in the destructor. - const STREAM_NAME: &[u8] = b"arrow_array_stream\0"; - - unsafe extern "C" fn drop_stream_capsule(capsule: *mut pyo3::ffi::PyObject) { - // Attempt to recover the raw stream pointer. If PyArrow imported the - // stream it will have set the capsule pointer to NULL, in which case - // `PyCapsule_GetPointer` returns NULL and we simply clear the error. - let ptr = pyo3::ffi::PyCapsule_GetPointer(capsule, STREAM_NAME.as_ptr() as *const _) - as *mut FFI_ArrowArrayStream; - - if ptr.is_null() { - // Ignore any exception raised by `PyCapsule_GetPointer` when the - // pointer is already NULL. - pyo3::ffi::PyErr_Clear(); - } else { - // Reconstruct the Box and drop it so resources are released. - drop(Box::from_raw(ptr)); - } - } - - let capsule_ptr = unsafe { - pyo3::ffi::PyCapsule_New( - raw_stream as *mut c_void, - STREAM_NAME.as_ptr() as *const std::ffi::c_char, - Some(drop_stream_capsule), - ) - }; - - if capsule_ptr.is_null() { - // Reclaim ownership to avoid leaking on failure - unsafe { - drop(Box::from_raw(raw_stream)); - } - return Err(PyErr::fetch(py).into()); - } - - // Safety: `capsule_ptr` is a new reference from `PyCapsule_New` - let capsule = unsafe { - Bound::from_owned_ptr(py, capsule_ptr).downcast_into_unchecked::() - }; + // Create the Arrow stream and wrap it in a PyCapsule. The default + // destructor provided by PyO3 will drop the stream unless ownership is + // transferred to PyArrow during import. + let stream = FFI_ArrowArrayStream::new(reader); + let name = CString::new("arrow_array_stream").unwrap(); + let capsule = PyCapsule::new(py, stream, Some(name))?; Ok(capsule) } From 831f56f45a0086c4d6486a59ade7d58a8e620463 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 13:07:20 +0800 Subject: [PATCH 28/67] Add `once_cell` dependency and refactor Arrow array stream capsule name handling --- Cargo.lock | 1 + Cargo.toml | 41 +++++++++++++++++++++++++++++++++-------- src/dataframe.rs | 8 ++++++-- 3 files changed, 40 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9497218c9..bab60fb1d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1552,6 +1552,7 @@ dependencies = [ "log", "mimalloc", "object_store", + "once_cell", "prost", "prost-types", "pyo3", diff --git a/Cargo.toml b/Cargo.toml index e51f4ddea..c5224ac58 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,17 +26,34 @@ readme = "README.md" license = "Apache-2.0" edition = "2021" rust-version = "1.78" -include = ["/src", "/datafusion", "/LICENSE.txt", "build.rs", "pyproject.toml", "Cargo.toml", "Cargo.lock"] +include = [ + "/src", + "/datafusion", + "/LICENSE.txt", + "build.rs", + "pyproject.toml", + "Cargo.toml", + "Cargo.lock", +] [features] default = ["mimalloc"] -protoc = [ "datafusion-substrait/protoc" ] +protoc = ["datafusion-substrait/protoc"] substrait = ["dep:datafusion-substrait"] [dependencies] -tokio = { version = "1.45", features = ["macros", "rt", "rt-multi-thread", "sync"] } -pyo3 = { version = "0.24", features = ["extension-module", "abi3", "abi3-py39"] } -pyo3-async-runtimes = { version = "0.24", features = ["tokio-runtime"]} +tokio = { version = "1.45", features = [ + "macros", + "rt", + "rt-multi-thread", + "sync", +] } +pyo3 = { version = "0.24", features = [ + "extension-module", + "abi3", + "abi3-py39", +] } +pyo3-async-runtimes = { version = "0.24", features = ["tokio-runtime"] } pyo3-log = "0.12.4" arrow = { version = "55.1.0", features = ["pyarrow"] } datafusion = { version = "49.0.2", features = ["avro", "unicode_expressions"] } @@ -45,15 +62,23 @@ datafusion-proto = { version = "49.0.2" } datafusion-ffi = { version = "49.0.2" } prost = "0.13.1" # keep in line with `datafusion-substrait` uuid = { version = "1.18", features = ["v4"] } -mimalloc = { version = "0.1", optional = true, default-features = false, features = ["local_dynamic_tls"] } +mimalloc = { version = "0.1", optional = true, default-features = false, features = [ + "local_dynamic_tls", +] } async-trait = "0.1.89" +once_cell = "1.18" futures = "0.3" -object_store = { version = "0.12.3", features = ["aws", "gcp", "azure", "http"] } +object_store = { version = "0.12.3", features = [ + "aws", + "gcp", + "azure", + "http", +] } url = "2" log = "0.4.27" [build-dependencies] -prost-types = "0.13.1" # keep in line with `datafusion-substrait` +prost-types = "0.13.1" # keep in line with `datafusion-substrait` pyo3-build-config = "0.24" [lib] diff --git a/src/dataframe.rs b/src/dataframe.rs index f1772e262..20d3fb532 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use once_cell::sync::Lazy; use std::collections::HashMap; use std::ffi::CString; use std::sync::Arc; @@ -59,6 +60,10 @@ use crate::{ expr::{sort_expr::PySortExpr, PyExpr}, }; +/// File-level static CString for the Arrow array stream capsule name. +static ARROW_ARRAY_STREAM_NAME: Lazy = + Lazy::new(|| CString::new("arrow_array_stream").unwrap()); + // https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 // - we have not decided on the table_provider approach yet // this is an interim implementation @@ -967,8 +972,7 @@ impl PyDataFrame { // destructor provided by PyO3 will drop the stream unless ownership is // transferred to PyArrow during import. let stream = FFI_ArrowArrayStream::new(reader); - let name = CString::new("arrow_array_stream").unwrap(); - let capsule = PyCapsule::new(py, stream, Some(name))?; + let capsule = PyCapsule::new(py, stream, Some(ARROW_ARRAY_STREAM_NAME.clone()))?; Ok(capsule) } From 7b5e4614da71b71d629a0918f32f15d69567a9b0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 14:11:20 +0800 Subject: [PATCH 29/67] Add `cstr` dependency and refactor Arrow array stream capsule name handling --- Cargo.lock | 12 +++++++++++- Cargo.toml | 2 +- src/dataframe.rs | 12 ++++++------ 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bab60fb1d..ca99964e2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -837,6 +837,16 @@ dependencies = [ "typenum", ] +[[package]] +name = "cstr" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68523903c8ae5aacfa32a0d9ae60cadeb764e1da14ee0d26b1f3089f13a54636" +dependencies = [ + "proc-macro2", + "quote", +] + [[package]] name = "csv" version = "1.3.1" @@ -1544,6 +1554,7 @@ version = "49.0.0" dependencies = [ "arrow", "async-trait", + "cstr", "datafusion", "datafusion-ffi", "datafusion-proto", @@ -1552,7 +1563,6 @@ dependencies = [ "log", "mimalloc", "object_store", - "once_cell", "prost", "prost-types", "pyo3", diff --git a/Cargo.toml b/Cargo.toml index c5224ac58..0be83a31d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -66,8 +66,8 @@ mimalloc = { version = "0.1", optional = true, default-features = false, feature "local_dynamic_tls", ] } async-trait = "0.1.89" -once_cell = "1.18" futures = "0.3" +cstr = "0.2" object_store = { version = "0.12.3", features = [ "aws", "gcp", diff --git a/src/dataframe.rs b/src/dataframe.rs index 20d3fb532..46119141b 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use once_cell::sync::Lazy; +use cstr::cstr; use std::collections::HashMap; -use std::ffi::CString; +use std::ffi::{CStr, CString}; use std::sync::Arc; use arrow::array::{new_null_array, RecordBatch, RecordBatchReader}; @@ -60,9 +60,8 @@ use crate::{ expr::{sort_expr::PySortExpr, PyExpr}, }; -/// File-level static CString for the Arrow array stream capsule name. -static ARROW_ARRAY_STREAM_NAME: Lazy = - Lazy::new(|| CString::new("arrow_array_stream").unwrap()); +/// File-level static CStr for the Arrow array stream capsule name. +static ARROW_ARRAY_STREAM_NAME: &CStr = cstr!("arrow_array_stream"); // https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 // - we have not decided on the table_provider approach yet @@ -972,7 +971,8 @@ impl PyDataFrame { // destructor provided by PyO3 will drop the stream unless ownership is // transferred to PyArrow during import. let stream = FFI_ArrowArrayStream::new(reader); - let capsule = PyCapsule::new(py, stream, Some(ARROW_ARRAY_STREAM_NAME.clone()))?; + let name = CString::new(ARROW_ARRAY_STREAM_NAME.to_bytes()).unwrap(); + let capsule = PyCapsule::new(py, stream, Some(name))?; Ok(capsule) } From d6e8132383410ebde44b92a2ce8fa96919f1573a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 14:33:41 +0800 Subject: [PATCH 30/67] Refactor test_iter_returns_datafusion_recordbatch to use RecordBatch directly --- python/tests/test_dataframe.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index ae09a90ad..5ca76f14a 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -45,7 +45,6 @@ reset_formatter, ) from datafusion.expr import Window -from datafusion.record_batch import RecordBatch as DataFusionRecordBatch from pyarrow.csv import write_csv pa_cffi = pytest.importorskip("pyarrow.cffi") @@ -401,7 +400,7 @@ def test_iter_batches(df): def test_iter_returns_datafusion_recordbatch(df): for batch in df: - assert isinstance(batch, DataFusionRecordBatch) + assert isinstance(batch, RecordBatch) def test_to_record_batch_stream(df): From 8a250a4f65fdba562370eb84f33b7ee0f0d91ecd Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 14:34:28 +0800 Subject: [PATCH 31/67] Add streaming execution examples to DataFrame documentation --- docs/source/user-guide/dataframe/index.rst | 24 ++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 03ff1977b..53d8cc0d4 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -203,6 +203,30 @@ To work with the stream directly, use for batch in stream: ... +Execute as Stream +^^^^^^^^^^^^^^^^^ + +For finer control over streaming execution, use +:py:meth:`~datafusion.DataFrame.execute_stream` to obtain a +:py:class:`pyarrow.RecordBatchReader`: + +.. code-block:: python + + reader = df.execute_stream() + for batch in reader: + ... # process each batch as it is produced + +When partition boundaries are important, +:py:meth:`~datafusion.DataFrame.execute_stream_partitioned` +returns an iterable of :py:class:`pyarrow.RecordBatchReader` objects, one per +partition: + +.. code-block:: python + + for stream in df.execute_stream_partitioned(): + for batch in stream: + ... # each stream yields RecordBatches + See :doc:`../io/arrow` for additional details on the Arrow interface. HTML Rendering From 77893223237f85ece3fa95ab463a4b0b44bc24cc Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 14:35:28 +0800 Subject: [PATCH 32/67] Rename `to_record_batch_stream` to `execute_stream` and update references in the codebase; mark the old method as deprecated. --- docs/source/user-guide/dataframe/index.rst | 4 ++-- python/datafusion/dataframe.py | 9 ++++++--- python/tests/test_dataframe.py | 4 ++-- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 53d8cc0d4..2a06a5666 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -194,12 +194,12 @@ Asynchronous iteration is supported as well, allowing integration with ... # process each batch as it is produced To work with the stream directly, use -``to_record_batch_stream()``, which returns a +``execute_stream()``, which returns a :class:`~datafusion.RecordBatchStream`: .. code-block:: python - stream = df.to_record_batch_stream() + stream = df.execute_stream() for batch in stream: ... diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 492e0ca55..f11db7676 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1040,9 +1040,12 @@ def execute_stream_partitioned(self) -> list[RecordBatchStream]: streams = self.df.execute_stream_partitioned() return [RecordBatchStream(rbs) for rbs in streams] - def to_record_batch_stream(self) -> RecordBatchStream: # noqa: F811 + @deprecated("Use execute_stream() instead") + def to_record_batch_stream(self) -> RecordBatchStream: """Return a :py:class:`RecordBatchStream` over this DataFrame's results. + This method is deprecated. Use :py:meth:`execute_stream` instead. + Returns: A ``RecordBatchStream`` representing the lazily generated record batches for this DataFrame. @@ -1133,11 +1136,11 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: def __iter__(self) -> Iterator[RecordBatch]: """Return an iterator over this DataFrame's record batches.""" - return iter(self.to_record_batch_stream()) + return iter(self.execute_stream()) def __aiter__(self) -> AsyncIterator[RecordBatch]: """Return an async iterator over this DataFrame's record batches.""" - return self.to_record_batch_stream().__aiter__() + return self.execute_stream().__aiter__() def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 5ca76f14a..19fb274f1 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -403,8 +403,8 @@ def test_iter_returns_datafusion_recordbatch(df): assert isinstance(batch, RecordBatch) -def test_to_record_batch_stream(df): - stream = df.to_record_batch_stream() +def test_execute_stream_basic(df): + stream = df.execute_stream() batches = list(stream) assert len(batches) == 1 From 07a81697f77384df612bf7149405589ab20d359c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 14:36:05 +0800 Subject: [PATCH 33/67] Clean up formatting in Cargo.toml for improved readability --- Cargo.toml | 38 +++++++------------------------------- 1 file changed, 7 insertions(+), 31 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 0be83a31d..b7a4a9b15 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,33 +26,16 @@ readme = "README.md" license = "Apache-2.0" edition = "2021" rust-version = "1.78" -include = [ - "/src", - "/datafusion", - "/LICENSE.txt", - "build.rs", - "pyproject.toml", - "Cargo.toml", - "Cargo.lock", -] +include = ["/src", "/datafusion", "/LICENSE.txt", "build.rs", "pyproject.toml", "Cargo.toml", "Cargo.lock"] [features] default = ["mimalloc"] -protoc = ["datafusion-substrait/protoc"] +protoc = [ "datafusion-substrait/protoc" ] substrait = ["dep:datafusion-substrait"] [dependencies] -tokio = { version = "1.45", features = [ - "macros", - "rt", - "rt-multi-thread", - "sync", -] } -pyo3 = { version = "0.24", features = [ - "extension-module", - "abi3", - "abi3-py39", -] } +tokio = { version = "1.45", features = ["macros", "rt", "rt-multi-thread", "sync"] } +pyo3 = { version = "0.24", features = ["extension-module", "abi3", "abi3-py39"] } pyo3-async-runtimes = { version = "0.24", features = ["tokio-runtime"] } pyo3-log = "0.12.4" arrow = { version = "55.1.0", features = ["pyarrow"] } @@ -62,23 +45,16 @@ datafusion-proto = { version = "49.0.2" } datafusion-ffi = { version = "49.0.2" } prost = "0.13.1" # keep in line with `datafusion-substrait` uuid = { version = "1.18", features = ["v4"] } -mimalloc = { version = "0.1", optional = true, default-features = false, features = [ - "local_dynamic_tls", -] } +mimalloc = { version = "0.1", optional = true, default-features = false, features = ["local_dynamic_tls"] } async-trait = "0.1.89" futures = "0.3" cstr = "0.2" -object_store = { version = "0.12.3", features = [ - "aws", - "gcp", - "azure", - "http", -] } +object_store = { version = "0.12.3", features = ["aws", "gcp", "azure", "http"] } url = "2" log = "0.4.27" [build-dependencies] -prost-types = "0.13.1" # keep in line with `datafusion-substrait` +prost-types = "0.13.1" # keep in line with `datafusion-substrait` pyo3-build-config = "0.24" [lib] From 9e27cc65adb76fbaea0750095cf98f656edf9c20 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 10 Sep 2025 15:06:34 +0800 Subject: [PATCH 34/67] Refactor Cargo.toml for improved formatting and readability --- Cargo.toml | 38 +++++++++++++++++++++++++++++++------- 1 file changed, 31 insertions(+), 7 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index b7a4a9b15..0be83a31d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,16 +26,33 @@ readme = "README.md" license = "Apache-2.0" edition = "2021" rust-version = "1.78" -include = ["/src", "/datafusion", "/LICENSE.txt", "build.rs", "pyproject.toml", "Cargo.toml", "Cargo.lock"] +include = [ + "/src", + "/datafusion", + "/LICENSE.txt", + "build.rs", + "pyproject.toml", + "Cargo.toml", + "Cargo.lock", +] [features] default = ["mimalloc"] -protoc = [ "datafusion-substrait/protoc" ] +protoc = ["datafusion-substrait/protoc"] substrait = ["dep:datafusion-substrait"] [dependencies] -tokio = { version = "1.45", features = ["macros", "rt", "rt-multi-thread", "sync"] } -pyo3 = { version = "0.24", features = ["extension-module", "abi3", "abi3-py39"] } +tokio = { version = "1.45", features = [ + "macros", + "rt", + "rt-multi-thread", + "sync", +] } +pyo3 = { version = "0.24", features = [ + "extension-module", + "abi3", + "abi3-py39", +] } pyo3-async-runtimes = { version = "0.24", features = ["tokio-runtime"] } pyo3-log = "0.12.4" arrow = { version = "55.1.0", features = ["pyarrow"] } @@ -45,16 +62,23 @@ datafusion-proto = { version = "49.0.2" } datafusion-ffi = { version = "49.0.2" } prost = "0.13.1" # keep in line with `datafusion-substrait` uuid = { version = "1.18", features = ["v4"] } -mimalloc = { version = "0.1", optional = true, default-features = false, features = ["local_dynamic_tls"] } +mimalloc = { version = "0.1", optional = true, default-features = false, features = [ + "local_dynamic_tls", +] } async-trait = "0.1.89" futures = "0.3" cstr = "0.2" -object_store = { version = "0.12.3", features = ["aws", "gcp", "azure", "http"] } +object_store = { version = "0.12.3", features = [ + "aws", + "gcp", + "azure", + "http", +] } url = "2" log = "0.4.27" [build-dependencies] -prost-types = "0.13.1" # keep in line with `datafusion-substrait` +prost-types = "0.13.1" # keep in line with `datafusion-substrait` pyo3-build-config = "0.24" [lib] From d3c68ccbc937035583e4dcf6bb75dce46221e99f Mon Sep 17 00:00:00 2001 From: kosiew Date: Sat, 13 Sep 2025 16:37:20 +0800 Subject: [PATCH 35/67] Update python/tests/test_io.py Co-authored-by: Kyle Barron --- python/tests/test_io.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index d292b59bc..ce6c181fc 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -109,7 +109,7 @@ def test_arrow_c_stream_large_dataset(ctx): # Create a very large DataFrame using range; this would be terabytes if collected df = range_table(ctx, 0, 1 << 40) - reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) + reader = pa.RecordBatchReader.from_stream(df) # Track RSS before consuming batches psutil = pytest.importorskip("psutil") From 33f90247598c47fcafbf07ff5913a669748c7e4f Mon Sep 17 00:00:00 2001 From: kosiew Date: Sat, 13 Sep 2025 17:19:59 +0800 Subject: [PATCH 36/67] Update python/datafusion/dataframe.py Co-authored-by: Kyle Barron --- python/datafusion/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index f3fff991e..9bd8e5ff2 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1147,7 +1147,7 @@ def __iter__(self) -> Iterator[RecordBatch]: def __aiter__(self) -> AsyncIterator[RecordBatch]: """Return an async iterator over this DataFrame's record batches.""" - return self.execute_stream().__aiter__() + return aiter(self.execute_stream()) def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. From 7553b3217609a410951740b94511a909fe47fb55 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 10:26:30 +0800 Subject: [PATCH 37/67] Refactor test_table_from_batches_stream to use pa.table for improved clarity --- python/tests/test_io.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index ce6c181fc..bf3371b5c 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -128,6 +128,6 @@ def test_arrow_c_stream_large_dataset(ctx): def test_table_from_batches_stream(ctx, fail_collect): df = range_table(ctx, 0, 10) - table = pa.Table.from_batches(batch.to_pyarrow() for batch in df) + table = pa.table(df) assert table.shape == (10, 1) assert table.column_names == ["value"] From b6909a590acee253153c33b972a16aa0c1dda316 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 10:27:19 +0800 Subject: [PATCH 38/67] Remove deprecated to_record_batch_stream method; use execute_stream instead --- dev/changelog/50.0.0.md | 24 ++++++++++++++++++++++++ python/datafusion/dataframe.py | 12 ------------ 2 files changed, 24 insertions(+), 12 deletions(-) create mode 100644 dev/changelog/50.0.0.md diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md new file mode 100644 index 000000000..0d6f251b8 --- /dev/null +++ b/dev/changelog/50.0.0.md @@ -0,0 +1,24 @@ + + +# Apache DataFusion Python 50.0.0 Changelog + +**Other:** + +- remove deprecated `DataFrame.to_record_batch_stream`; use `execute_stream` instead. diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 9bd8e5ff2..49d34cfab 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1047,18 +1047,6 @@ def execute_stream_partitioned(self) -> list[RecordBatchStream]: streams = self.df.execute_stream_partitioned() return [RecordBatchStream(rbs) for rbs in streams] - @deprecated("Use execute_stream() instead") - def to_record_batch_stream(self) -> RecordBatchStream: - """Return a :py:class:`RecordBatchStream` over this DataFrame's results. - - This method is deprecated. Use :py:meth:`execute_stream` instead. - - Returns: - A ``RecordBatchStream`` representing the lazily generated record - batches for this DataFrame. - """ - return self.execute_stream() - def to_pandas(self) -> pd.DataFrame: """Execute the :py:class:`DataFrame` and convert it into a Pandas DataFrame. From f4e76eac7217aa2c6e29feee25cad1dc0baa096c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 10:28:17 +0800 Subject: [PATCH 39/67] Add example for concurrent processing of partitioned streams using asyncio --- docs/source/user-guide/dataframe/index.rst | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 2a06a5666..78a56ff99 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -227,6 +227,20 @@ partition: for batch in stream: ... # each stream yields RecordBatches +To process partitions concurrently, first collect the streams into a list +and then poll each one in a separate ``asyncio`` task: + +.. code-block:: python + + import asyncio + + async def consume(stream): + async for batch in stream: + ... + + streams = list(df.execute_stream_partitioned()) + await asyncio.gather(*(consume(s) for s in streams)) + See :doc:`../io/arrow` for additional details on the Arrow interface. HTML Rendering From b66b4412c3a7825b48a71e595a1b470d29bd60ce Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 10:28:48 +0800 Subject: [PATCH 40/67] Update documentation to reflect changes in execute_stream return type and usage --- docs/source/user-guide/dataframe/index.rst | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 78a56ff99..ce89c03c3 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -208,17 +208,22 @@ Execute as Stream For finer control over streaming execution, use :py:meth:`~datafusion.DataFrame.execute_stream` to obtain a -:py:class:`pyarrow.RecordBatchReader`: +:py:class:`datafusion.RecordBatchStream`: .. code-block:: python - reader = df.execute_stream() - for batch in reader: + stream = df.execute_stream() + for batch in stream: ... # process each batch as it is produced +.. tip:: + + To get a PyArrow reader instead, call + ``pa.RecordBatchReader.from_stream(df)``. + When partition boundaries are important, :py:meth:`~datafusion.DataFrame.execute_stream_partitioned` -returns an iterable of :py:class:`pyarrow.RecordBatchReader` objects, one per +returns an iterable of :py:class:`datafusion.RecordBatchStream` objects, one per partition: .. code-block:: python From 2794c88b370a11566854f3a9ce9579c78dd2fe74 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 10:29:24 +0800 Subject: [PATCH 41/67] Update PyArrow streaming example to use pa.table for eager collection --- docs/source/user-guide/dataframe/index.rst | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index ce89c03c3..cf09e9a75 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -178,12 +178,13 @@ PyArrow: ... # each batch is a ``datafusion.RecordBatch`` Each batch exposes ``to_pyarrow()``, allowing conversion to a PyArrow -table without collecting everything eagerly: +table. ``pa.table(df)`` collects the entire DataFrame eagerly into a +PyArrow table: .. code-block:: python import pyarrow as pa - table = pa.Table.from_batches(b.to_pyarrow() for b in df) + table = pa.table(df) Asynchronous iteration is supported as well, allowing integration with ``asyncio`` event loops: From 17c4c2c42ba1a7f68f305db9f51b8f89bd1a463a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 11:36:51 +0800 Subject: [PATCH 42/67] Enhance documentation for DataFrame streaming API, clarifying schema handling and limitations --- python/datafusion/dataframe.py | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 49d34cfab..fd43f06a9 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1114,15 +1114,27 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: The DataFrame is executed using DataFusion's streaming APIs and exposed via 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. + full result set is never materialized in memory. + + When ``requested_schema`` is provided, DataFusion applies only simple + projections such as selecting a subset of existing columns or reordering + them. Column renaming, computed expressions, or type coercion are not + supported through this interface. Args: - requested_schema: Attempt to provide the DataFrame using this schema. + requested_schema: Either a :py:class:`pyarrow.Schema` or an Arrow C + Schema capsule (``PyCapsule``) produced by + ``schema._export_to_c_capsule()``. The DataFrame will attempt to + align its output with the fields and order specified by this schema. Returns: - Arrow PyCapsule object representing an ``ArrowArrayStream``. + Arrow ``PyCapsule`` object representing an ``ArrowArrayStream``. + + Examples: + >>> schema = df.schema() + >>> stream = df.__arrow_c_stream__(schema) + >>> capsule = schema._export_to_c_capsule() + >>> stream = df.__arrow_c_stream__(capsule) """ # ``DataFrame.__arrow_c_stream__`` in the Rust extension leverages # ``execute_stream_partitioned`` under the hood to stream batches while From 0ff4c0de322223d74b08c58d5ff024f526f5605f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 11:37:55 +0800 Subject: [PATCH 43/67] Clarify behavior of __arrow_c_stream__ execution, emphasizing incremental batch processing and memory efficiency --- docs/source/user-guide/io/arrow.rst | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/docs/source/user-guide/io/arrow.rst b/docs/source/user-guide/io/arrow.rst index d571aa99c..7b32208a6 100644 --- a/docs/source/user-guide/io/arrow.rst +++ b/docs/source/user-guide/io/arrow.rst @@ -60,10 +60,16 @@ Exporting from DataFusion DataFusion DataFrames implement ``__arrow_c_stream__`` PyCapsule interface, so any Python library that accepts these can import a DataFusion DataFrame directly. -.. warning:: - It is important to note that this will cause the DataFrame execution to happen, which may be - a time consuming task. That is, you will cause a - :py:func:`datafusion.dataframe.DataFrame.collect` operation call to occur. +.. note:: + Invoking ``__arrow_c_stream__`` still triggers execution of the underlying + query, but batches are yielded incrementally rather than materialized all at + once in memory. Consumers can process the stream as it arrives, avoiding the + memory overhead of a full + :py:func:`datafusion.dataframe.DataFrame.collect`. + + For an example of this streamed execution and its memory safety, see the + ``test_arrow_c_stream_large_dataset`` unit test in + :mod:`python.tests.test_io`. .. ipython:: python From f450e1dc015d5dc612c9c377ebdffe46771e42f0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 11:39:08 +0800 Subject: [PATCH 44/67] Add note on limitations of `arrow::compute::cast` for schema transformations --- src/dataframe.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index cbb6749b7..db41a204f 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -1085,7 +1085,11 @@ fn project_schema(from_schema: Schema, to_schema: Schema) -> Result Date: Sat, 13 Sep 2025 16:37:20 +0800 Subject: [PATCH 45/67] Update python/tests/test_io.py Co-authored-by: Kyle Barron From fd08dc44657a842a997dd5274364546fccb2204f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 17:14:46 +0800 Subject: [PATCH 46/67] Rename test function for clarity: update `test_table_from_batches_stream` to `test_table_from_arrow_c_stream` --- python/tests/test_io.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index bf3371b5c..eb41e8840 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -125,7 +125,7 @@ def test_arrow_c_stream_large_dataset(ctx): assert current_rss - start_rss < 50 * 1024 * 1024 -def test_table_from_batches_stream(ctx, fail_collect): +def test_table_from_arrow_c_stream(ctx, fail_collect): df = range_table(ctx, 0, 10) table = pa.table(df) From 9baa49ecac9254c6605003ddeda8b3cf5dffcf28 Mon Sep 17 00:00:00 2001 From: kosiew Date: Sat, 13 Sep 2025 17:19:59 +0800 Subject: [PATCH 47/67] Update python/datafusion/dataframe.py Co-authored-by: Kyle Barron From 78f6c8aa543e26f923a7707a55795ec2f1bc110c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 17:23:57 +0800 Subject: [PATCH 48/67] Add documentation note for Arrow C Data Interface PyCapsule in DataFrame class --- python/datafusion/dataframe.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index fd43f06a9..1e8de0045 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1135,6 +1135,11 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: >>> stream = df.__arrow_c_stream__(schema) >>> capsule = schema._export_to_c_capsule() >>> stream = df.__arrow_c_stream__(capsule) + + Notes: + The Arrow C Data Interface PyCapsule details are documented by Apache + Arrow and can be found at: + https://arrow.apache.org/docs/format/CDataInterface/PyCapsuleInterface.html """ # ``DataFrame.__arrow_c_stream__`` in the Rust extension leverages # ``execute_stream_partitioned`` under the hood to stream batches while From 5a53633a434700c7ab244c0742032abdf6b9cf82 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 17:47:49 +0800 Subject: [PATCH 49/67] Enhance documentation on zero-copy streaming to Arrow-based Python libraries, clarifying the protocol and adding implementation-agnostic notes. --- dev/changelog/50.0.0.md | 24 --------------- docs/source/user-guide/dataframe/index.rst | 34 ++++++++++++++-------- 2 files changed, 22 insertions(+), 36 deletions(-) delete mode 100644 dev/changelog/50.0.0.md diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md deleted file mode 100644 index 0d6f251b8..000000000 --- a/dev/changelog/50.0.0.md +++ /dev/null @@ -1,24 +0,0 @@ - - -# Apache DataFusion Python 50.0.0 Changelog - -**Other:** - -- remove deprecated `DataFrame.to_record_batch_stream`; use `execute_stream` instead. diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index cf09e9a75..8f2e1ba7c 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -149,15 +149,26 @@ To materialize the results of your DataFrame operations: # Count rows count = df.count() -PyArrow Streaming +Zero-copy streaming to Arrow-based Python libraries ----------------- DataFusion DataFrames implement the ``__arrow_c_stream__`` protocol, enabling -zero-copy streaming into libraries like `PyArrow `_. -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. +zero-copy, lazy streaming into Arrow-based Python libraries. Earlier versions +eagerly converted the entire DataFrame when exporting to Python Arrow APIs, +which could exhaust memory on large results. With the streaming protocol, +batches are produced on demand so you can process arbitrarily large results +without out-of-memory errors. + +.. note:: + + The protocol is implementation-agnostic and works with any Python library + that understands the Arrow C streaming interface (for example, PyArrow + or other Arrow-compatible implementations). The sections below provide a + short PyArrow-specific example and general guidance for other + implementations. + +PyArrow +------- .. code-block:: python @@ -170,7 +181,7 @@ out-of-memory errors. DataFrames are also iterable, yielding :class:`datafusion.RecordBatch` objects lazily so you can loop over results directly without importing -PyArrow: +PyArrow:: .. code-block:: python @@ -179,7 +190,7 @@ PyArrow: Each batch exposes ``to_pyarrow()``, allowing conversion to a PyArrow table. ``pa.table(df)`` collects the entire DataFrame eagerly into a -PyArrow table: +PyArrow table:: .. code-block:: python @@ -187,16 +198,15 @@ PyArrow table: table = pa.table(df) Asynchronous iteration is supported as well, allowing integration with -``asyncio`` event loops: +``asyncio`` event loops:: .. code-block:: python async for batch in df: ... # process each batch as it is produced -To work with the stream directly, use -``execute_stream()``, which returns a -:class:`~datafusion.RecordBatchStream`: +To work with the stream directly, use ``execute_stream()``, which returns a +:class:`~datafusion.RecordBatchStream`:: .. code-block:: python From ccc8633b41778752ad1c519b7f11c33ec337b00e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 18:11:19 +0800 Subject: [PATCH 50/67] Fix formatting of section header for zero-copy streaming in DataFrame documentation --- docs/source/user-guide/dataframe/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 8f2e1ba7c..5e7487d7a 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -150,7 +150,7 @@ To materialize the results of your DataFrame operations: count = df.count() Zero-copy streaming to Arrow-based Python libraries ------------------ +--------------------------------------------------- DataFusion DataFrames implement the ``__arrow_c_stream__`` protocol, enabling zero-copy, lazy streaming into Arrow-based Python libraries. Earlier versions From 98ac3a1f4eab0815c810cdea846f17e7243eb341 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 18:12:22 +0800 Subject: [PATCH 51/67] Refine zero-copy streaming documentation by removing outdated information about eager conversion, emphasizing on-demand batch processing to prevent memory exhaustion. --- docs/source/user-guide/dataframe/index.rst | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 5e7487d7a..7dcbc539c 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -153,11 +153,9 @@ Zero-copy streaming to Arrow-based Python libraries --------------------------------------------------- DataFusion DataFrames implement the ``__arrow_c_stream__`` protocol, enabling -zero-copy, lazy streaming into Arrow-based Python libraries. Earlier versions -eagerly converted the entire DataFrame when exporting to Python Arrow APIs, -which could exhaust memory on large results. With the streaming protocol, -batches are produced on demand so you can process arbitrarily large results -without out-of-memory errors. +zero-copy, lazy streaming into Arrow-based Python libraries. With the streaming +protocol, batches are produced on demand so you can process arbitrarily large +results without out-of-memory errors. .. note:: From 759fb86874370f962d293592ab62e51f364514ce Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 20:46:49 +0800 Subject: [PATCH 52/67] Add alternative method for creating RecordBatchReader from Arrow C stream --- python/tests/test_dataframe.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index 371020144..c723805b1 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1654,6 +1654,8 @@ def test_arrow_c_stream_to_table_and_reader(fail_collect): assert table.column("a").num_chunks == 2 reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) + # alternatively, we could use + # reader = pa.RecordBatchReader.from_stream(df) assert isinstance(reader, pa.RecordBatchReader) reader_table = pa.Table.from_batches(reader) expected = pa.Table.from_batches([batch1, batch2]) From 57d41622b5890942b09213826f6c918da8b9f865 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 20:54:42 +0800 Subject: [PATCH 53/67] Refactor tests to use RecordBatchReader.from_stream instead of deprecated _import_from_c_capsule method --- python/tests/test_dataframe.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index c723805b1..af09d58ef 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1653,9 +1653,7 @@ def test_arrow_c_stream_to_table_and_reader(fail_collect): assert table.schema == df.schema() assert table.column("a").num_chunks == 2 - reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) - # alternatively, we could use - # reader = pa.RecordBatchReader.from_stream(df) + reader = pa.RecordBatchReader.from_stream(df) assert isinstance(reader, pa.RecordBatchReader) reader_table = pa.Table.from_batches(reader) expected = pa.Table.from_batches([batch1, batch2]) @@ -2878,7 +2876,7 @@ def test_arrow_c_stream_interrupted(): """ ) - reader = pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__()) + reader = pa.RecordBatchReader.from_stream(df) interrupted = False interrupt_error = None From d66d496a50542b4d02ae94212c46d4cbe3f5b876 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 21:10:46 +0800 Subject: [PATCH 54/67] Replace deprecated _import_from_c_capsule method with from_stream for RecordBatchReader in test_arrow_c_stream_schema_selection --- python/tests/test_dataframe.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/python/tests/test_dataframe.py b/python/tests/test_dataframe.py index af09d58ef..262795530 100644 --- a/python/tests/test_dataframe.py +++ b/python/tests/test_dataframe.py @@ -1698,11 +1698,8 @@ def test_arrow_c_stream_schema_selection(fail_collect): capsule_new = ctypes.pythonapi.PyCapsule_New capsule_new.restype = ctypes.py_object capsule_new.argtypes = [ctypes.c_void_p, ctypes.c_char_p, ctypes.c_void_p] - schema_capsule = capsule_new(ctypes.c_void_p(address), b"arrow_schema", None) - reader = pa.RecordBatchReader._import_from_c_capsule( - df.__arrow_c_stream__(schema_capsule) - ) + reader = pa.RecordBatchReader.from_stream(df, schema=requested_schema) assert reader.schema == requested_schema From d76a50936d6f961fdcedc421726f01fa6440c59f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 21:17:08 +0800 Subject: [PATCH 55/67] Update test description for arrow_c_stream_large_dataset to clarify streaming method and usage of public API --- python/tests/test_io.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index eb41e8840..c74156799 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -99,12 +99,14 @@ def test_read_avro(): def test_arrow_c_stream_large_dataset(ctx): - """DataFrame.__arrow_c_stream__ yields batches incrementally. + """DataFrame streaming yields batches incrementally using Arrow APIs. This test constructs a DataFrame that would be far larger than available - memory if materialized. The ``__arrow_c_stream__`` method should expose a - stream of record batches without collecting the full dataset, so reading a - handful of batches should not exhaust process memory. + memory if materialized. Use the public API + ``pa.RecordBatchReader.from_stream(df)`` (which is same as + ``pa.RecordBatchReader._import_from_c_capsule(df.__arrow_c_stream__())``) + to read record batches incrementally without collecting the full dataset, + so reading a handful of batches should not exhaust process memory. """ # Create a very large DataFrame using range; this would be terabytes if collected df = range_table(ctx, 0, 1 << 40) From 743323422e1441ec60b5ea7f656b88ede4890ba2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 21:29:16 +0800 Subject: [PATCH 56/67] Add comments to clarify RSS measurement in test_arrow_c_stream_large_dataset --- python/tests/test_io.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/tests/test_io.py b/python/tests/test_io.py index c74156799..5cca0219a 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -114,6 +114,9 @@ def test_arrow_c_stream_large_dataset(ctx): reader = pa.RecordBatchReader.from_stream(df) # Track RSS before consuming batches + # RSS is a practical measure of RAM usage visible to the OS. It excludes memory + # that has been swapped out and provides a simple cross-platform-ish indicator + # (psutil normalizes per-OS sources). psutil = pytest.importorskip("psutil") process = psutil.Process() start_rss = process.memory_info().rss From 848665e8b10adf2ea816239397ff74df7abdd888 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 22:00:48 +0800 Subject: [PATCH 57/67] Fix ruff errors --- python/datafusion/dataframe.py | 2 +- python/tests/test_io.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 1e8de0045..cd5ac32f1 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1135,7 +1135,7 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: >>> stream = df.__arrow_c_stream__(schema) >>> capsule = schema._export_to_c_capsule() >>> stream = df.__arrow_c_stream__(capsule) - + Notes: The Arrow C Data Interface PyCapsule details are documented by Apache Arrow and can be found at: diff --git a/python/tests/test_io.py b/python/tests/test_io.py index 5cca0219a..9f56f74d7 100644 --- a/python/tests/test_io.py +++ b/python/tests/test_io.py @@ -114,8 +114,8 @@ def test_arrow_c_stream_large_dataset(ctx): reader = pa.RecordBatchReader.from_stream(df) # Track RSS before consuming batches - # RSS is a practical measure of RAM usage visible to the OS. It excludes memory - # that has been swapped out and provides a simple cross-platform-ish indicator + # RSS is a practical measure of RAM usage visible to the OS. It excludes memory + # that has been swapped out and provides a simple cross-platform-ish indicator # (psutil normalizes per-OS sources). psutil = pytest.importorskip("psutil") process = psutil.Process() From 13ebaf99fd349c82ff45752f98c012288e44ae2d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 13 Sep 2025 22:17:54 +0800 Subject: [PATCH 58/67] Update async iterator implementation in DataFrame to ensure compatibility with Python < 3.10 --- python/datafusion/dataframe.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index cd5ac32f1..8874e71a4 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1151,8 +1151,13 @@ def __iter__(self) -> Iterator[RecordBatch]: return iter(self.execute_stream()) def __aiter__(self) -> AsyncIterator[RecordBatch]: - """Return an async iterator over this DataFrame's record batches.""" - return aiter(self.execute_stream()) + """Return an async iterator over this DataFrame's record batches. + + `RecordBatchStream` implements ``__aiter__``, so return the stream + directly to remain compatible with Python < 3.10 (this project + supports Python >= 3.6). + """ + return self.execute_stream() def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. From dae501d87ae634155143c425cc1834a3d7acc3d8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 15 Sep 2025 13:30:47 +0800 Subject: [PATCH 59/67] Fix async iterator implementation in DataFrame for compatibility with Python < 3.10 --- python/datafusion/dataframe.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 8874e71a4..4b3d77d7a 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1153,11 +1153,10 @@ def __iter__(self) -> Iterator[RecordBatch]: def __aiter__(self) -> AsyncIterator[RecordBatch]: """Return an async iterator over this DataFrame's record batches. - `RecordBatchStream` implements ``__aiter__``, so return the stream - directly to remain compatible with Python < 3.10 (this project - supports Python >= 3.6). + We're using __aiter__ becaause we support Python < 3.10 where aiter() is not + available. """ - return self.execute_stream() + return self.execute_stream().__aiter__() def transform(self, func: Callable[..., DataFrame], *args: Any) -> DataFrame: """Apply a function to the current DataFrame which returns another DataFrame. From c36aa9ae43743dd2b20b847bc38358587673a74a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 15 Sep 2025 14:36:36 +0800 Subject: [PATCH 60/67] fix typo --- python/datafusion/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 4b3d77d7a..ece8290c2 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1153,7 +1153,7 @@ def __iter__(self) -> Iterator[RecordBatch]: def __aiter__(self) -> AsyncIterator[RecordBatch]: """Return an async iterator over this DataFrame's record batches. - We're using __aiter__ becaause we support Python < 3.10 where aiter() is not + We're using __aiter__ because we support Python < 3.10 where aiter() is not available. """ return self.execute_stream().__aiter__() From 914f17e937206ecd4c50da37ae476037aa71791e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 15 Sep 2025 16:23:04 +0800 Subject: [PATCH 61/67] Fix formatting in DataFrame documentation and add example usage for Arrow integration --- docs/source/user-guide/dataframe/index.rst | 2 +- docs/source/user-guide/io/arrow.rst | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 7dcbc539c..2b573ea4e 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -179,7 +179,7 @@ PyArrow DataFrames are also iterable, yielding :class:`datafusion.RecordBatch` objects lazily so you can loop over results directly without importing -PyArrow:: +PyArrow: .. code-block:: python diff --git a/docs/source/user-guide/io/arrow.rst b/docs/source/user-guide/io/arrow.rst index 7b32208a6..372c0d5af 100644 --- a/docs/source/user-guide/io/arrow.rst +++ b/docs/source/user-guide/io/arrow.rst @@ -74,6 +74,8 @@ Python library that accepts these can import a DataFusion DataFrame directly. .. ipython:: python + from datafusion import col, lit + df = df.select((col("a") * lit(1.5)).alias("c"), lit("df").alias("d")) pa.table(df) From e7a7ec958e9d2713a26453d7db8c79fe622bbb08 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 20 Sep 2025 14:03:26 +0800 Subject: [PATCH 62/67] fix: correct formatting in documentation for RecordBatchStream --- docs/source/user-guide/dataframe/index.rst | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index c3ffb7995..e7ee581ce 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -254,7 +254,7 @@ Asynchronous iteration is supported as well, allowing integration with ... # process each batch as it is produced To work with the stream directly, use ``execute_stream()``, which returns a -:class:`~datafusion.RecordBatchStream`:: +:class:`~datafusion.RecordBatchStream`. .. code-block:: python @@ -278,8 +278,9 @@ For finer control over streaming execution, use .. tip:: To get a PyArrow reader instead, call - ``pa.RecordBatchReader.from_stream(df)``. + ``pa.RecordBatchReader.from_stream(df)``. + When partition boundaries are important, :py:meth:`~datafusion.DataFrame.execute_stream_partitioned` returns an iterable of :py:class:`datafusion.RecordBatchStream` objects, one per From 51d88bd1af0cc7f6ddce145427667f2a0784dc70 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 1 Nov 2025 12:22:53 +0800 Subject: [PATCH 63/67] refactor: remove unused import from errors module in dataframe.rs --- src/dataframe.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index 6cd4119eb..a93aa0185 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -46,15 +46,14 @@ use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods}; use pyo3::PyErr; -use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError}; +use crate::errors::{py_datafusion_err, PyDataFusionError}; use crate::expr::sort_expr::to_sort_expressions; use crate::physical_plan::PyExecutionPlan; use crate::record_batch::{poll_next_batch, PyRecordBatchStream}; use crate::sql::logical::PyLogicalPlan; use crate::table::{PyTable, TempViewTable}; use crate::utils::{ - get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, spawn_future, validate_pycapsule, - wait_for_future, + is_ipython_env, py_obj_to_scalar_value, spawn_future, validate_pycapsule, wait_for_future, }; use crate::{ errors::PyDataFusionResult, From e9994cf65906cb475e6f71e1f50b3521695563ac Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 1 Nov 2025 13:26:04 +0800 Subject: [PATCH 64/67] Simplified the streaming protocol description by removing the clause about arbitrarily large results while keeping the paragraph smooth. --- docs/source/user-guide/dataframe/index.rst | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/source/user-guide/dataframe/index.rst b/docs/source/user-guide/dataframe/index.rst index 70976af0d..659589cf0 100644 --- a/docs/source/user-guide/dataframe/index.rst +++ b/docs/source/user-guide/dataframe/index.rst @@ -204,9 +204,8 @@ Zero-copy streaming to Arrow-based Python libraries --------------------------------------------------- DataFusion DataFrames implement the ``__arrow_c_stream__`` protocol, enabling -zero-copy, lazy streaming into Arrow-based Python libraries. With the streaming -protocol, batches are produced on demand so you can process arbitrarily large -results without out-of-memory errors. +zero-copy, lazy streaming into Arrow-based Python libraries. With the streaming +protocol, batches are produced on demand. .. note:: From 62f9df267f3983415a0651b1c7aeae99c0ec8f4d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 1 Nov 2025 13:26:35 +0800 Subject: [PATCH 65/67] Updated the Arrow streaming documentation to describe incremental execution, remove the note block, and highlight lazy batch retrieval when using __arrow_c_stream__ --- docs/source/user-guide/io/arrow.rst | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/docs/source/user-guide/io/arrow.rst b/docs/source/user-guide/io/arrow.rst index 372c0d5af..9196fcea7 100644 --- a/docs/source/user-guide/io/arrow.rst +++ b/docs/source/user-guide/io/arrow.rst @@ -60,16 +60,10 @@ Exporting from DataFusion DataFusion DataFrames implement ``__arrow_c_stream__`` PyCapsule interface, so any Python library that accepts these can import a DataFusion DataFrame directly. -.. note:: - Invoking ``__arrow_c_stream__`` still triggers execution of the underlying - query, but batches are yielded incrementally rather than materialized all at - once in memory. Consumers can process the stream as it arrives, avoiding the - memory overhead of a full - :py:func:`datafusion.dataframe.DataFrame.collect`. - - For an example of this streamed execution and its memory safety, see the - ``test_arrow_c_stream_large_dataset`` unit test in - :mod:`python.tests.test_io`. +Invoking ``__arrow_c_stream__`` triggers execution of the underlying query, but +batches are yielded incrementally rather than materialized all at once in memory. +Consumers can process the stream as it arrives. The stream executes lazily, +letting downstream readers pull batches on demand. .. ipython:: python From 2f4d41b847734774bb22afbef9627c371b734f2f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 1 Nov 2025 13:27:06 +0800 Subject: [PATCH 66/67] Replaced the DataFrame.__arrow_c_stream__ docstring example with a link to the Apache Arrow streaming documentation for practical guidance. --- python/datafusion/dataframe.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index dd429e587..dac71a93c 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1319,11 +1319,10 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: Returns: Arrow ``PyCapsule`` object representing an ``ArrowArrayStream``. - Examples: - >>> schema = df.schema() - >>> stream = df.__arrow_c_stream__(schema) - >>> capsule = schema._export_to_c_capsule() - >>> stream = df.__arrow_c_stream__(capsule) + For practical usage patterns, see the Apache Arrow streaming + documentation: https://arrow.apache.org/docs/python/ipc.html#streaming + and the DataFusion user guide sections on Arrow I/O and DataFrame + streaming (:doc:`../io/arrow` and :doc:`../dataframe/index`). Notes: The Arrow C Data Interface PyCapsule details are documented by Apache From d2bbd608bf08da180f63c79ffb56d774baf75114 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 1 Nov 2025 16:06:25 +0800 Subject: [PATCH 67/67] fix: update user guide links in DataFrame class documentation for clarity --- python/datafusion/dataframe.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index dac71a93c..2549e3c18 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -1320,9 +1320,10 @@ def __arrow_c_stream__(self, requested_schema: object | None = None) -> object: Arrow ``PyCapsule`` object representing an ``ArrowArrayStream``. For practical usage patterns, see the Apache Arrow streaming - documentation: https://arrow.apache.org/docs/python/ipc.html#streaming - and the DataFusion user guide sections on Arrow I/O and DataFrame - streaming (:doc:`../io/arrow` and :doc:`../dataframe/index`). + documentation: https://arrow.apache.org/docs/python/ipc.html#streaming. + + For details on DataFusion's Arrow integration and DataFrame streaming, + see the user guide (user-guide/io/arrow and user-guide/dataframe/index). Notes: The Arrow C Data Interface PyCapsule details are documented by Apache