From 9b4f1442df7a94220ba63ce68d1a6b72756e1019 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 15 Sep 2025 15:43:20 +0800 Subject: [PATCH 01/34] =?UTF-8?q?Migrate=20Table=20=E2=86=92=20TableProvid?= =?UTF-8?q?er;=20refactor=20registration=20and=20access,=20update=20docs/t?= =?UTF-8?q?ests,=20add=20DataFrame=20view=20support,=20and=20improve=20Sen?= =?UTF-8?q?d/concurrency=20support.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit migrates the codebase from using `Table` to a `TableProvider`-based API, refactors registration and access paths to simplify catalog/context interactions, and updates documentation and examples. DataFrame view handling is improved (`into_view` is now public), the test-suite is expanded to cover new registration and async SQL scenarios, and `TableProvider` now supports the `Send` trait across modules for safer concurrency. Minor import cleanup and utility adjustments (including a refined `pyany_to_table_provider`) are included. --- docs/source/contributor-guide/ffi.rst | 2 +- docs/source/user-guide/data-sources.rst | 4 +- docs/source/user-guide/io/table_provider.rst | 25 +++- python/datafusion/__init__.py | 3 +- python/datafusion/catalog.py | 10 +- python/datafusion/context.py | 28 +++-- python/datafusion/dataframe.py | 11 +- python/tests/test_context.py | 30 +++++ src/catalog.rs | 64 ++++++++--- src/context.rs | 57 +++++----- src/dataframe.rs | 52 ++------- src/lib.rs | 3 + src/table.rs | 114 +++++++++++++++++++ src/udtf.rs | 15 ++- tests/dataframe_into_view.rs | 39 +++++++ 15 files changed, 347 insertions(+), 110 deletions(-) create mode 100644 src/table.rs create mode 100644 tests/dataframe_into_view.rs diff --git a/docs/source/contributor-guide/ffi.rst b/docs/source/contributor-guide/ffi.rst index e201db71e..e8abde00d 100644 --- a/docs/source/contributor-guide/ffi.rst +++ b/docs/source/contributor-guide/ffi.rst @@ -34,7 +34,7 @@ as performant as possible and to utilize the features of DataFusion, you may dec your source in Rust and then expose it through `PyO3 `_ as a Python library. At first glance, it may appear the best way to do this is to add the ``datafusion-python`` -crate as a dependency, provide a ``PyTable``, and then to register it with the +crate as a dependency, produce a DataFusion table in Rust, and then register it with the ``SessionContext``. Unfortunately, this will not work. When you produce your code as a Python library and it needs to interact with the DataFusion diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index 7d07c67df..7df0be98d 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -152,9 +152,11 @@ as Delta Lake. This will require a recent version of .. code-block:: python from deltalake import DeltaTable + from datafusion import TableProvider delta_table = DeltaTable("path_to_table") - ctx.register_table_provider("my_delta_table", delta_table) + provider = TableProvider.from_capsule(delta_table) + ctx.register_table("my_delta_table", provider) df = ctx.table("my_delta_table") df.show() diff --git a/docs/source/user-guide/io/table_provider.rst b/docs/source/user-guide/io/table_provider.rst index bd1d6b80f..6b81c27c7 100644 --- a/docs/source/user-guide/io/table_provider.rst +++ b/docs/source/user-guide/io/table_provider.rst @@ -47,12 +47,29 @@ A complete example can be found in the `examples folder Table: """Return the table with the given ``name`` from this schema.""" return Table(self._raw_schema.table(name)) - def register_table(self, name, table) -> None: - """Register a table provider in this schema.""" + def register_table(self, name, table: Table | TableProvider) -> None: + """Register a table or table provider in this schema.""" if isinstance(table, Table): return self._raw_schema.register_table(name, table.table) return self._raw_schema.register_table(name, table) @@ -219,8 +221,8 @@ def table(self, name: str) -> Table | None: """Retrieve a specific table from this schema.""" ... - def register_table(self, name: str, table: Table) -> None: # noqa: B027 - """Add a table from this schema. + def register_table(self, name: str, table: Table | TableProvider) -> None: # noqa: B027 + """Add a table to this schema. This method is optional. If your schema provides a fixed list of tables, you do not need to implement this method. diff --git a/python/datafusion/context.py b/python/datafusion/context.py index bce51d644..0a3944a68 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -46,6 +46,7 @@ import pandas as pd import polars as pl + from datafusion import TableProvider from datafusion.plan import ExecutionPlan, LogicalPlan @@ -743,16 +744,21 @@ def register_view(self, name: str, df: DataFrame) -> None: view = df.into_view() self.ctx.register_table(name, view) - def register_table(self, name: str, table: Table) -> None: - """Register a :py:class: `~datafusion.catalog.Table` as a table. + def register_table(self, name: str, table: Table | TableProvider) -> None: + """Register a :py:class:`~datafusion.catalog.Table` or ``TableProvider``. - The registered table can be referenced from SQL statement executed against. + The registered table can be referenced from SQL statements executed against + this context. Args: name: Name of the resultant table. - table: DataFusion table to add to the session context. + table: DataFusion :class:`Table` or :class:`TableProvider` to add to the + session context. """ - self.ctx.register_table(name, table.table) + if isinstance(table, Table): + self.ctx.register_table(name, table.table) + else: + self.ctx.register_table(name, table) def deregister_table(self, name: str) -> None: """Remove a table from the session.""" @@ -772,14 +778,18 @@ def register_catalog_provider( self.ctx.register_catalog_provider(name, provider) def register_table_provider( - self, name: str, provider: TableProviderExportable + self, name: str, provider: TableProviderExportable | TableProvider ) -> None: """Register a table provider. - This table provider must have a method called ``__datafusion_table_provider__`` - which returns a PyCapsule that exposes a ``FFI_TableProvider``. + Deprecated: use :meth:`register_table` instead. """ - self.ctx.register_table_provider(name, provider) + warnings.warn( + "register_table_provider is deprecated; use register_table", + DeprecationWarning, + stacklevel=2, + ) + self.register_table(name, provider) def register_udtf(self, func: TableFunction) -> None: """Register a user defined table function.""" diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 181c29db4..0656384eb 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -40,6 +40,7 @@ from datafusion._internal import DataFrame as DataFrameInternal from datafusion._internal import ParquetColumnOptions as ParquetColumnOptionsInternal from datafusion._internal import ParquetWriterOptions as ParquetWriterOptionsInternal +from datafusion._internal import TableProvider as TableProviderInternal from datafusion.expr import Expr, SortExpr, sort_or_default from datafusion.plan import ExecutionPlan, LogicalPlan from datafusion.record_batch import RecordBatchStream @@ -307,8 +308,14 @@ def __init__(self, df: DataFrameInternal) -> None: """ self.df = df - def into_view(self) -> pa.Table: - """Convert DataFrame as a ViewTable which can be used in register_table.""" + def into_view(self) -> TableProviderInternal: + """Convert ``DataFrame`` into a ``TableProvider`` view for registration. + + This is the preferred way to obtain a view for + :py:meth:`~datafusion.context.SessionContext.register_table`. + ``TableProvider.from_dataframe`` calls this method under the hood, + and the older ``TableProvider.from_view`` helper is deprecated. + """ return self.df.into_view() def __getitem__(self, key: str | list[str]) -> DataFrame: diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 6dbcc0d5e..91692bbe7 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -27,6 +27,7 @@ SessionConfig, SessionContext, SQLOptions, + TableProvider, column, literal, ) @@ -330,6 +331,35 @@ def test_deregister_table(ctx, database): assert public.names() == {"csv1", "csv2"} +def test_register_table_from_dataframe_into_view(ctx): + df = ctx.from_pydict({"a": [1, 2]}) + provider = df.into_view() + ctx.register_table("view_tbl", provider) + result = ctx.sql("SELECT * FROM view_tbl").collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + +def test_table_provider_from_capsule(ctx): + df = ctx.from_pydict({"a": [1, 2]}) + provider = df.into_view() + capsule = provider.__datafusion_table_provider__() + provider2 = TableProvider.from_capsule(capsule) + ctx.register_table("capsule_tbl", provider2) + result = ctx.sql("SELECT * FROM capsule_tbl").collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + +def test_table_provider_from_capsule_invalid(): + with pytest.raises(Exception): # noqa: B017 + TableProvider.from_capsule(object()) + + +def test_register_table_with_dataframe_errors(ctx): + df = ctx.from_pydict({"a": [1]}) + with pytest.raises(Exception): # noqa: B017 + ctx.register_table("bad", df) + + def test_register_dataset(ctx): # create a RecordBatch and register it as a pyarrow.dataset.Dataset batch = pa.RecordBatch.from_arrays( diff --git a/src/catalog.rs b/src/catalog.rs index 17d4ec3b8..0f8d33d6a 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -17,6 +17,7 @@ use crate::dataset::Dataset; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; +use crate::table::PyTableProvider; use crate::utils::{validate_pycapsule, wait_for_future}; use async_trait::async_trait; use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; @@ -51,7 +52,7 @@ pub struct PySchema { #[pyclass(name = "RawTable", module = "datafusion.catalog", subclass)] #[derive(Clone)] pub struct PyTable { - pub table: Arc, + pub table: Arc, } impl From> for PyCatalog { @@ -67,11 +68,11 @@ impl From> for PySchema { } impl PyTable { - pub fn new(table: Arc) -> Self { + pub fn new(table: Arc) -> Self { Self { table } } - pub fn table(&self) -> Arc { + pub fn table(&self) -> Arc { self.table.clone() } } @@ -205,15 +206,18 @@ impl PySchema { let provider = unsafe { capsule.reference::() }; let provider: ForeignTableProvider = provider.into(); - Arc::new(provider) as Arc + Arc::new(provider) as Arc } else { match table_provider.extract::() { Ok(py_table) => py_table.table, - Err(_) => { - let py = table_provider.py(); - let provider = Dataset::new(&table_provider, py)?; - Arc::new(provider) as Arc - } + Err(_) => match table_provider.extract::() { + Ok(py_provider) => py_provider.into_inner(), + Err(_) => { + let py = table_provider.py(); + let provider = Dataset::new(&table_provider, py)?; + Arc::new(provider) as Arc + } + }, } }; @@ -294,7 +298,7 @@ impl RustWrappedPySchemaProvider { } } - fn table_inner(&self, name: &str) -> PyResult>> { + fn table_inner(&self, name: &str) -> PyResult>> { Python::with_gil(|py| { let provider = self.schema_provider.bind(py); let py_table_method = provider.getattr("table")?; @@ -305,14 +309,14 @@ impl RustWrappedPySchemaProvider { } if py_table.hasattr("__datafusion_table_provider__")? { - let capsule = provider.getattr("__datafusion_table_provider__")?.call0()?; + let capsule = py_table.getattr("__datafusion_table_provider__")?.call0()?; let capsule = capsule.downcast::().map_err(py_datafusion_err)?; validate_pycapsule(capsule, "datafusion_table_provider")?; let provider = unsafe { capsule.reference::() }; let provider: ForeignTableProvider = provider.into(); - Ok(Some(Arc::new(provider) as Arc)) + Ok(Some(Arc::new(provider) as Arc)) } else { if let Ok(inner_table) = py_table.getattr("table") { if let Ok(inner_table) = inner_table.extract::() { @@ -320,11 +324,15 @@ impl RustWrappedPySchemaProvider { } } + if let Ok(py_provider) = py_table.extract::() { + return Ok(Some(py_provider.into_inner())); + } + match py_table.extract::() { Ok(py_table) => Ok(Some(py_table.table)), Err(_) => { let ds = Dataset::new(&py_table, py).map_err(py_datafusion_err)?; - Ok(Some(Arc::new(ds) as Arc)) + Ok(Some(Arc::new(ds) as Arc)) } } } @@ -360,7 +368,17 @@ impl SchemaProvider for RustWrappedPySchemaProvider { &self, name: &str, ) -> datafusion::common::Result>, DataFusionError> { - self.table_inner(name).map_err(to_datafusion_err) + // Convert from our internal Send type to the trait expected type + match self.table_inner(name).map_err(to_datafusion_err)? { + Some(table) => { + // Safe conversion: we're widening the bounds (removing Send) + let raw = Arc::into_raw(table); + let wide: *const dyn TableProvider = raw as *const _; + let arc = unsafe { Arc::from_raw(wide) }; + Ok(Some(arc)) + } + None => Ok(None), + } } fn register_table( @@ -368,7 +386,14 @@ impl SchemaProvider for RustWrappedPySchemaProvider { name: String, table: Arc, ) -> datafusion::common::Result>> { - let py_table = PyTable::new(table); + // Convert from trait type to our internal Send type + let send_table = { + let raw = Arc::into_raw(table); + let send: *const (dyn TableProvider + Send) = raw as *const _; + unsafe { Arc::from_raw(send) } + }; + + let py_table = PyTable::new(send_table); Python::with_gil(|py| { let provider = self.schema_provider.bind(py); let _ = provider @@ -397,7 +422,14 @@ impl SchemaProvider for RustWrappedPySchemaProvider { // If we can turn this table provider into a `Dataset`, return it. // Otherwise, return None. let dataset = match Dataset::new(&table, py) { - Ok(dataset) => Some(Arc::new(dataset) as Arc), + Ok(dataset) => { + // Convert from our internal Send type to trait expected type + let send_table = Arc::new(dataset) as Arc; + let raw = Arc::into_raw(send_table); + let wide: *const dyn TableProvider = raw as *const _; + let arc = unsafe { Arc::from_raw(wide) }; + Some(arc) + } Err(_) => None, }; diff --git a/src/context.rs b/src/context.rs index 36133a33d..670b079f3 100644 --- a/src/context.rs +++ b/src/context.rs @@ -41,6 +41,7 @@ use crate::record_batch::PyRecordBatchStream; use crate::sql::exceptions::py_value_err; use crate::sql::logical::PyLogicalPlan; use crate::store::StorageContexts; +use crate::table::PyTableProvider; use crate::udaf::PyAggregateUDF; use crate::udf::PyScalarUDF; use crate::udtf::PyTableFunction; @@ -417,12 +418,7 @@ impl PySessionContext { .with_listing_options(options) .with_schema(resolved_schema); let table = ListingTable::try_new(config)?; - self.register_table( - name, - &PyTable { - table: Arc::new(table), - }, - )?; + self.ctx.register_table(name, Arc::new(table))?; Ok(()) } @@ -607,8 +603,32 @@ impl PySessionContext { Ok(df) } - pub fn register_table(&mut self, name: &str, table: &PyTable) -> PyDataFusionResult<()> { - self.ctx.register_table(name, table.table())?; + pub fn register_table( + &mut self, + name: &str, + table_provider: Bound<'_, PyAny>, + ) -> PyDataFusionResult<()> { + let provider = if table_provider.hasattr("__datafusion_table_provider__")? { + let capsule = table_provider + .getattr("__datafusion_table_provider__")? + .call0()?; + let capsule = capsule.downcast::().map_err(py_datafusion_err)?; + validate_pycapsule(capsule, "datafusion_table_provider")?; + + let provider = unsafe { capsule.reference::() }; + let provider: ForeignTableProvider = provider.into(); + Arc::new(provider) as Arc + } else if let Ok(py_table) = table_provider.extract::() { + py_table.table() + } else if let Ok(py_provider) = table_provider.extract::() { + py_provider.into_inner() + } else { + return Err(crate::errors::PyDataFusionError::Common( + "Expected a Table or TableProvider.".to_string(), + )); + }; + + self.ctx.register_table(name, provider)?; Ok(()) } @@ -651,23 +671,8 @@ impl PySessionContext { name: &str, provider: Bound<'_, PyAny>, ) -> PyDataFusionResult<()> { - if provider.hasattr("__datafusion_table_provider__")? { - let capsule = provider.getattr("__datafusion_table_provider__")?.call0()?; - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - - let _ = self.ctx.register_table(name, Arc::new(provider))?; - - Ok(()) - } else { - Err(crate::errors::PyDataFusionError::Common( - "__datafusion_table_provider__ does not exist on Table Provider object." - .to_string(), - )) - } + // Deprecated: use `register_table` instead + self.register_table(name, provider) } pub fn register_record_batches( @@ -853,7 +858,7 @@ impl PySessionContext { dataset: &Bound<'_, PyAny>, py: Python, ) -> PyDataFusionResult<()> { - let table: Arc = Arc::new(Dataset::new(dataset, py)?); + let table: Arc = Arc::new(Dataset::new(dataset, py)?); self.ctx.register_table(name, table)?; diff --git a/src/dataframe.rs b/src/dataframe.rs index 46fba137c..76cb53ec0 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -36,7 +36,6 @@ use datafusion::error::DataFusionError; use datafusion::execution::SendableRecordBatchStream; use datafusion::parquet::basic::{BrotliLevel, Compression, GzipLevel, ZstdLevel}; use datafusion::prelude::*; -use datafusion_ffi::table_provider::FFI_TableProvider; use futures::{StreamExt, TryStreamExt}; use pyo3::exceptions::PyValueError; use pyo3::prelude::*; @@ -44,12 +43,12 @@ 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::expr::sort_expr::to_sort_expressions; use crate::physical_plan::PyExecutionPlan; use crate::record_batch::PyRecordBatchStream; use crate::sql::logical::PyLogicalPlan; +pub use crate::table::PyTableProvider; use crate::utils::{ get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, validate_pycapsule, wait_for_future, }; @@ -58,40 +57,6 @@ use crate::{ expr::{sort_expr::PySortExpr, PyExpr}, }; -// 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 -#[pyclass(name = "TableProvider", module = "datafusion")] -pub struct PyTableProvider { - provider: Arc, -} - -impl PyTableProvider { - pub fn new(provider: Arc) -> Self { - Self { provider } - } - - pub fn as_table(&self) -> PyTable { - let table_provider: Arc = self.provider.clone(); - PyTable::new(table_provider) - } -} - -#[pymethods] -impl PyTableProvider { - fn __datafusion_table_provider__<'py>( - &self, - py: Python<'py>, - ) -> PyResult> { - let name = CString::new("datafusion_table_provider").unwrap(); - - let runtime = get_tokio_runtime().0.handle().clone(); - let provider = FFI_TableProvider::new(Arc::clone(&self.provider), false, Some(runtime)); - - PyCapsule::new(py, provider, Some(name.clone())) - } -} - /// Configuration for DataFrame display formatting #[derive(Debug, Clone)] pub struct FormatterConfig { @@ -303,6 +268,11 @@ impl PyDataFrame { } } + #[allow(clippy::wrong_self_convention)] + pub(crate) fn into_view_provider(&self) -> Arc { + self.df.as_ref().clone().into_view() + } + fn prepare_repr_string(&mut self, py: Python, as_html: bool) -> PyDataFusionResult { // Get the Python formatter and config let PythonFormatter { formatter, config } = get_python_formatter_with_config(py)?; @@ -436,14 +406,12 @@ impl PyDataFrame { /// https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 /// - we have not decided on the table_provider approach yet #[allow(clippy::wrong_self_convention)] - fn into_view(&self) -> PyDataFusionResult { + pub fn into_view(&self) -> PyDataFusionResult { // Call the underlying Rust DataFrame::into_view method. // Note that the Rust method consumes self; here we clone the inner Arc - // so that we don’t invalidate this PyDataFrame. - let table_provider = self.df.as_ref().clone().into_view(); - let table_provider = PyTableProvider::new(table_provider); - - Ok(table_provider.as_table()) + // so that we don't invalidate this PyDataFrame. + let table_provider = self.into_view_provider(); + Ok(PyTableProvider::new(table_provider)) } #[pyo3(signature = (*args))] diff --git a/src/lib.rs b/src/lib.rs index 29d3f41da..19e3bc504 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -52,6 +52,7 @@ pub mod pyarrow_util; mod record_batch; pub mod sql; pub mod store; +pub mod table; pub mod unparser; #[cfg(feature = "substrait")] @@ -88,6 +89,7 @@ fn _internal(py: Python, m: Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; m.add_class::()?; + m.add_class::()?; m.add_class::()?; m.add_class::()?; m.add_class::()?; @@ -97,6 +99,7 @@ fn _internal(py: Python, m: Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; m.add_class::()?; + m.add_class::()?; let catalog = PyModule::new(py, "catalog")?; catalog::init_module(&catalog)?; diff --git a/src/table.rs b/src/table.rs new file mode 100644 index 000000000..c6a9a640f --- /dev/null +++ b/src/table.rs @@ -0,0 +1,114 @@ +// 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. + +use std::ffi::CString; +use std::sync::Arc; + +use datafusion::datasource::TableProvider; +use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; +use pyo3::exceptions::PyDeprecationWarning; +use pyo3::prelude::*; +use pyo3::types::PyCapsule; + +use crate::catalog::PyTable; +use crate::dataframe::PyDataFrame; +use crate::errors::{py_datafusion_err, PyDataFusionResult}; +use crate::utils::{get_tokio_runtime, validate_pycapsule}; + +/// Represents a table provider that can be registered with DataFusion +#[pyclass(name = "TableProvider", module = "datafusion")] +#[derive(Clone)] +pub struct PyTableProvider { + pub(crate) provider: Arc, +} + +impl PyTableProvider { + pub(crate) fn new(provider: Arc) -> Self { + Self { provider } + } + + /// Return a `PyTable` wrapper around this provider. + /// + /// Historically callers chained `as_table().table()` to access the + /// underlying `Arc`. Prefer [`as_arc`] or + /// [`into_inner`] for direct access instead. + pub fn as_table(&self) -> PyTable { + PyTable::new(Arc::clone(&self.provider)) + } + + /// Return a clone of the inner [`TableProvider`]. + pub fn as_arc(&self) -> Arc { + Arc::clone(&self.provider) + } + + /// Consume this wrapper and return the inner [`TableProvider`]. + pub fn into_inner(self) -> Arc { + self.provider + } +} + +#[pymethods] +impl PyTableProvider { + /// Create a `TableProvider` from a PyCapsule containing an FFI pointer + #[staticmethod] + pub fn from_capsule(capsule: Bound<'_, PyAny>) -> PyResult { + let capsule = capsule.downcast::().map_err(py_datafusion_err)?; + validate_pycapsule(capsule, "datafusion_table_provider")?; + + let provider = unsafe { capsule.reference::() }; + let provider: ForeignTableProvider = provider.into(); + + Ok(Self::new(Arc::new(provider))) + } + + /// Create a `TableProvider` from a `DataFrame`. + /// + /// This method simply delegates to `DataFrame.into_view`. + #[staticmethod] + pub fn from_dataframe(df: &PyDataFrame) -> PyDataFusionResult { + let table_provider = df.into_view_provider(); + Ok(Self::new(table_provider)) + } + + /// Create a `TableProvider` from a `DataFrame` by converting it into a view. + /// + /// Deprecated: prefer `DataFrame.into_view` or + /// `TableProvider.from_dataframe` instead. + #[staticmethod] + pub fn from_view(py: Python<'_>, df: &PyDataFrame) -> PyDataFusionResult { + py.import("warnings")?.call_method1( + "warn", + ( + "PyTableProvider.from_view() is deprecated; use DataFrame.into_view() or TableProvider.from_dataframe() instead.", + py.get_type::(), + ), + )?; + Self::from_dataframe(df) + } + + fn __datafusion_table_provider__<'py>( + &self, + py: Python<'py>, + ) -> PyResult> { + let name = CString::new("datafusion_table_provider").unwrap(); + + let runtime = get_tokio_runtime().0.handle().clone(); + let provider = FFI_TableProvider::new(Arc::clone(&self.provider), false, Some(runtime)); + + PyCapsule::new(py, provider, Some(name.clone())) + } +} diff --git a/src/udtf.rs b/src/udtf.rs index db16d6c05..a14b48c36 100644 --- a/src/udtf.rs +++ b/src/udtf.rs @@ -18,9 +18,9 @@ use pyo3::prelude::*; use std::sync::Arc; -use crate::dataframe::PyTableProvider; use crate::errors::{py_datafusion_err, to_datafusion_err}; use crate::expr::PyExpr; +use crate::table::PyTableProvider; use crate::utils::validate_pycapsule; use datafusion::catalog::{TableFunctionImpl, TableProvider}; use datafusion::error::Result as DataFusionResult; @@ -87,7 +87,7 @@ impl PyTableFunction { fn call_python_table_function( func: &Arc, args: &[Expr], -) -> DataFusionResult> { +) -> DataFusionResult> { let args = args .iter() .map(|arg| PyExpr::from(arg.clone())) @@ -107,7 +107,7 @@ fn call_python_table_function( let provider = unsafe { capsule.reference::() }; let provider: ForeignTableProvider = provider.into(); - Ok(Arc::new(provider) as Arc) + Ok(Arc::new(provider) as Arc) } else { Err(PyNotImplementedError::new_err( "__datafusion_table_provider__ does not exist on Table Provider object.", @@ -121,7 +121,14 @@ impl TableFunctionImpl for PyTableFunction { fn call(&self, args: &[Expr]) -> DataFusionResult> { match &self.inner { PyTableFunctionInner::FFIFunction(func) => func.call(args), - PyTableFunctionInner::PythonFunction(obj) => call_python_table_function(obj, args), + PyTableFunctionInner::PythonFunction(obj) => { + let send_result = call_python_table_function(obj, args)?; + // Convert from our Send type to the trait expected type + let raw = Arc::into_raw(send_result); + let wide: *const dyn TableProvider = raw as *const _; + let arc = unsafe { Arc::from_raw(wide) }; + Ok(arc) + } } } } diff --git a/tests/dataframe_into_view.rs b/tests/dataframe_into_view.rs new file mode 100644 index 000000000..39e9453d5 --- /dev/null +++ b/tests/dataframe_into_view.rs @@ -0,0 +1,39 @@ +use std::sync::Arc; + +use datafusion::arrow::array::Int32Array; +use datafusion::arrow::datatypes::{DataType, Field, Schema}; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::datasource::MemTable; +use datafusion::prelude::SessionContext; +use datafusion_python::dataframe::PyDataFrame; + +#[test] +fn dataframe_into_view_returns_table_provider() { + // Create an in-memory table with one Int32 column. + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + ) + .unwrap(); + let table = MemTable::try_new(schema, vec![vec![batch]]).unwrap(); + + // Build a DataFrame from the table and convert it into a view. + let ctx = SessionContext::new(); + let df = ctx.read_table(Arc::new(table)).unwrap(); + let py_df = PyDataFrame::new(df); + let provider = py_df.into_view().unwrap(); + + // Register the view in a new context and ensure it can be queried. + let ctx = SessionContext::new(); + ctx.register_table("view", provider.into_inner()).unwrap(); + + let rt = tokio::runtime::Runtime::new().unwrap(); + let batches = rt.block_on(async { + let df = ctx.sql("SELECT * FROM view").await.unwrap(); + df.collect().await.unwrap() + }); + + assert_eq!(batches.len(), 1); + assert_eq!(batches[0].num_rows(), 3); +} From 3da3f93794c5e9c26b428ec1ebe6fdeae33089a2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 16 Sep 2025 10:18:47 +0800 Subject: [PATCH 02/34] =?UTF-8?q?Refactors=20and=20bug=20fixes=20around=20?= =?UTF-8?q?TableProvider=20registration=20and=20DataFrame=E2=86=92TablePro?= =?UTF-8?q?vider=20conversion,=20plus=20tests=20and=20FFI/pycapsule=20impr?= =?UTF-8?q?ovements.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit -- Registration logic & API * Refactor of table provider registration logic for improved clarity and simpler call sites. * Remove PyTableProvider registration from an internal module (reduces surprising side effects). * Update table registration method to call `register_table` instead of `register_table_provider`. * Extend `register_table` to support `TableProviderExportable` so more provider types can be registered uniformly. * Improve error messages related to registration failures (missing PyCapsule name and DataFrame registration errors). -- DataFrame ↔ TableProvider conversions * Introduce utility functions to simplify table provider conversions and centralize conversion logic. * Rename `into_view_provider` → `to_view_provider` for clearer intent. * Fix `from_dataframe` to return the correct type and update `DataFrame.into_view` to import the correct `TableProvider`. * Remove an obsolete `dataframe_into_view` test case after the refactor. -- FFI / PyCapsule handling * Update `FFI_TableProvider` initialization to accept an optional parameter (improves FFI ergonomics). * Introduce `table_provider_from_pycapsule` utility to standardize pycapsule-based construction. * Improve the error message when a PyCapsule name is missing to help debugging. -- DeltaTable & specific integrations * Update TableProvider registration for `DeltaTable` to use the correct registration method (matches the new API surface). -- Tests, docs & minor fixes * Add tests for registering a `TableProvider` from a `DataFrame` and from a capsule to ensure conversion paths are covered. * Fix a typo in the `register_view` docstring and another typo in the error message for unsupported volatility type. * Simplify version retrieval by removing exception handling around `PackageNotFoundError` (streamlines code path). --- docs/source/user-guide/data-sources.rst | 2 +- docs/source/user-guide/io/table_provider.rst | 19 ++--- .../python/tests/_test_table_function.py | 2 +- .../python/tests/_test_table_provider.py | 6 +- python/datafusion/catalog.py | 18 ++++- python/datafusion/context.py | 23 ++++-- python/datafusion/dataframe.py | 4 +- python/tests/test_context.py | 31 +++++++- src/catalog.rs | 67 +++++------------- src/context.rs | 22 +++--- src/dataframe.rs | 10 +-- src/lib.rs | 1 - src/table.rs | 8 +-- src/udtf.rs | 28 +++----- src/utils.rs | 70 +++++++++++++++++-- tests/dataframe_into_view.rs | 39 ----------- 16 files changed, 186 insertions(+), 164 deletions(-) delete mode 100644 tests/dataframe_into_view.rs diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index 7df0be98d..95e1343b7 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -155,7 +155,7 @@ as Delta Lake. This will require a recent version of from datafusion import TableProvider delta_table = DeltaTable("path_to_table") - provider = TableProvider.from_capsule(delta_table) + provider = TableProvider.from_capsule(delta_table.__datafusion_table_provider__()) ctx.register_table("my_delta_table", provider) df = ctx.table("my_delta_table") df.show() diff --git a/docs/source/user-guide/io/table_provider.rst b/docs/source/user-guide/io/table_provider.rst index 6b81c27c7..720beeccc 100644 --- a/docs/source/user-guide/io/table_provider.rst +++ b/docs/source/user-guide/io/table_provider.rst @@ -39,9 +39,8 @@ A complete example can be found in the `examples folder PyResult> { let name = CString::new("datafusion_table_provider").unwrap(); - let provider = Arc::new(self.clone()) - .map_err(|e| PyRuntimeError::new_err(e.to_string()))?; - let provider = FFI_TableProvider::new(Arc::new(provider), false); + let provider = Arc::new(self.clone()); + let provider = FFI_TableProvider::new(provider, false, None); PyCapsule::new_bound(py, provider, Some(name.clone())) } @@ -50,8 +49,10 @@ A complete example can be found in the `examples folder Table: """Return the table with the given ``name`` from this schema.""" return Table(self._raw_schema.table(name)) - def register_table(self, name, table: Table | TableProvider) -> None: - """Register a table or table provider in this schema.""" + def register_table( + self, name, table: Table | TableProvider | TableProviderExportable + ) -> None: + """Register a table or table provider in this schema. + + Objects implementing ``__datafusion_table_provider__`` are also supported + and treated as :class:`TableProvider` instances. + """ if isinstance(table, Table): return self._raw_schema.register_table(name, table.table) return self._raw_schema.register_table(name, table) @@ -221,11 +228,16 @@ def table(self, name: str) -> Table | None: """Retrieve a specific table from this schema.""" ... - def register_table(self, name: str, table: Table | TableProvider) -> None: # noqa: B027 + def register_table( # noqa: B027 + self, name: str, table: Table | TableProvider | TableProviderExportable + ) -> None: """Add a table to this schema. This method is optional. If your schema provides a fixed list of tables, you do not need to implement this method. + + Objects implementing ``__datafusion_table_provider__`` are also supported + and treated as :class:`TableProvider` instances. """ def deregister_table(self, name, cascade: bool) -> None: # noqa: B027 diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 0a3944a68..fb2ba4589 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -735,7 +735,7 @@ def from_polars(self, data: pl.DataFrame, name: str | None = None) -> DataFrame: # 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: - """Register a :py:class: `~datafusion.detaframe.DataFrame` as a view. + """Register a :py:class:`~datafusion.dataframe.DataFrame` as a view. Args: name (str): The name to register the view under. @@ -744,16 +744,26 @@ def register_view(self, name: str, df: DataFrame) -> None: view = df.into_view() self.ctx.register_table(name, view) - def register_table(self, name: str, table: Table | TableProvider) -> None: + def register_table( + self, name: str, table: Table | TableProvider | TableProviderExportable + ) -> None: """Register a :py:class:`~datafusion.catalog.Table` or ``TableProvider``. The registered table can be referenced from SQL statements executed against this context. + Plain :py:class:`~datafusion.dataframe.DataFrame` objects are not supported; + convert them first with :meth:`datafusion.dataframe.DataFrame.into_view` or + :meth:`datafusion.catalog.TableProvider.from_dataframe`. + + Objects implementing ``__datafusion_table_provider__`` are also supported + and treated as :class:`~datafusion.catalog.TableProvider` instances. + Args: name: Name of the resultant table. - table: DataFusion :class:`Table` or :class:`TableProvider` to add to the - session context. + table: DataFusion :class:`Table`, :class:`TableProvider`, or any object + implementing ``__datafusion_table_provider__`` to add to the session + context. """ if isinstance(table, Table): self.ctx.register_table(name, table.table) @@ -778,11 +788,14 @@ def register_catalog_provider( self.ctx.register_catalog_provider(name, provider) def register_table_provider( - self, name: str, provider: TableProviderExportable | TableProvider + self, name: str, provider: Table | TableProvider | TableProviderExportable ) -> None: """Register a table provider. Deprecated: use :meth:`register_table` instead. + + Objects implementing ``__datafusion_table_provider__`` are also supported + and treated as :class:`~datafusion.catalog.TableProvider` instances. """ warnings.warn( "register_table_provider is deprecated; use register_table", diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 0656384eb..671c3aeb1 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -40,7 +40,6 @@ from datafusion._internal import DataFrame as DataFrameInternal from datafusion._internal import ParquetColumnOptions as ParquetColumnOptionsInternal from datafusion._internal import ParquetWriterOptions as ParquetWriterOptionsInternal -from datafusion._internal import TableProvider as TableProviderInternal from datafusion.expr import Expr, SortExpr, sort_or_default from datafusion.plan import ExecutionPlan, LogicalPlan from datafusion.record_batch import RecordBatchStream @@ -53,6 +52,7 @@ import polars as pl import pyarrow as pa + from datafusion._internal import TableProvider from datafusion._internal import expr as expr_internal from enum import Enum @@ -308,7 +308,7 @@ def __init__(self, df: DataFrameInternal) -> None: """ self.df = df - def into_view(self) -> TableProviderInternal: + def into_view(self) -> TableProvider: """Convert ``DataFrame`` into a ``TableProvider`` view for registration. This is the preferred way to obtain a view for diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 91692bbe7..215b5df2f 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -349,6 +349,30 @@ def test_table_provider_from_capsule(ctx): assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] +def test_table_provider_from_dataframe(ctx): + df = ctx.from_pydict({"a": [1, 2]}).df + provider = TableProvider.from_dataframe(df) + ctx.register_table("from_dataframe_tbl", provider) + result = ctx.sql("SELECT * FROM from_dataframe_tbl").collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + +def test_register_table_capsule_direct(ctx): + df = ctx.from_pydict({"a": [1, 2]}) + provider = df.into_view() + + class CapsuleProvider: + def __init__(self, inner): + self._inner = inner + + def __datafusion_table_provider__(self): + return self._inner.__datafusion_table_provider__() + + ctx.register_table("capsule_direct_tbl", CapsuleProvider(provider)) + result = ctx.sql("SELECT * FROM capsule_direct_tbl").collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + def test_table_provider_from_capsule_invalid(): with pytest.raises(Exception): # noqa: B017 TableProvider.from_capsule(object()) @@ -356,9 +380,14 @@ def test_table_provider_from_capsule_invalid(): def test_register_table_with_dataframe_errors(ctx): df = ctx.from_pydict({"a": [1]}) - with pytest.raises(Exception): # noqa: B017 + with pytest.raises(Exception) as exc_info: # noqa: B017 ctx.register_table("bad", df) + assert ( + str(exc_info.value) + == 'Expected a Table or TableProvider. Convert DataFrames with "DataFrame.into_view()" or "TableProvider.from_dataframe()".' + ) + def test_register_dataset(ctx): # create a RecordBatch and register it as a pyarrow.dataset.Dataset diff --git a/src/catalog.rs b/src/catalog.rs index 0f8d33d6a..9fd43d703 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -18,7 +18,10 @@ use crate::dataset::Dataset; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; use crate::table::PyTableProvider; -use crate::utils::{validate_pycapsule, wait_for_future}; +use crate::utils::{ + table_provider_from_pycapsule, table_provider_send_to_table_provider, table_provider_to_send, + validate_pycapsule, wait_for_future, +}; use async_trait::async_trait; use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; use datafusion::common::DataFusionError; @@ -28,7 +31,6 @@ use datafusion::{ datasource::{TableProvider, TableType}, }; use datafusion_ffi::schema_provider::{FFI_SchemaProvider, ForeignSchemaProvider}; -use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::exceptions::PyKeyError; use pyo3::prelude::*; use pyo3::types::PyCapsule; @@ -197,28 +199,16 @@ impl PySchema { } fn register_table(&self, name: &str, table_provider: Bound<'_, PyAny>) -> PyResult<()> { - let provider = if table_provider.hasattr("__datafusion_table_provider__")? { - let capsule = table_provider - .getattr("__datafusion_table_provider__")? - .call0()?; - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - Arc::new(provider) as Arc + let provider = if let Ok(py_table) = table_provider.extract::() { + py_table.table + } else if let Ok(py_provider) = table_provider.extract::() { + py_provider.into_inner() + } else if let Some(provider) = table_provider_from_pycapsule(&table_provider)? { + provider } else { - match table_provider.extract::() { - Ok(py_table) => py_table.table, - Err(_) => match table_provider.extract::() { - Ok(py_provider) => py_provider.into_inner(), - Err(_) => { - let py = table_provider.py(); - let provider = Dataset::new(&table_provider, py)?; - Arc::new(provider) as Arc - } - }, - } + let py = table_provider.py(); + let provider = Dataset::new(&table_provider, py)?; + Arc::new(provider) as Arc }; let _ = self @@ -308,15 +298,8 @@ impl RustWrappedPySchemaProvider { return Ok(None); } - if py_table.hasattr("__datafusion_table_provider__")? { - let capsule = py_table.getattr("__datafusion_table_provider__")?.call0()?; - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - - Ok(Some(Arc::new(provider) as Arc)) + if let Some(provider) = table_provider_from_pycapsule(&py_table)? { + Ok(Some(provider)) } else { if let Ok(inner_table) = py_table.getattr("table") { if let Ok(inner_table) = inner_table.extract::() { @@ -370,13 +353,7 @@ impl SchemaProvider for RustWrappedPySchemaProvider { ) -> datafusion::common::Result>, DataFusionError> { // Convert from our internal Send type to the trait expected type match self.table_inner(name).map_err(to_datafusion_err)? { - Some(table) => { - // Safe conversion: we're widening the bounds (removing Send) - let raw = Arc::into_raw(table); - let wide: *const dyn TableProvider = raw as *const _; - let arc = unsafe { Arc::from_raw(wide) }; - Ok(Some(arc)) - } + Some(table) => Ok(Some(table_provider_send_to_table_provider(table))), None => Ok(None), } } @@ -387,11 +364,7 @@ impl SchemaProvider for RustWrappedPySchemaProvider { table: Arc, ) -> datafusion::common::Result>> { // Convert from trait type to our internal Send type - let send_table = { - let raw = Arc::into_raw(table); - let send: *const (dyn TableProvider + Send) = raw as *const _; - unsafe { Arc::from_raw(send) } - }; + let send_table = table_provider_to_send(table); let py_table = PyTable::new(send_table); Python::with_gil(|py| { @@ -423,12 +396,8 @@ impl SchemaProvider for RustWrappedPySchemaProvider { // Otherwise, return None. let dataset = match Dataset::new(&table, py) { Ok(dataset) => { - // Convert from our internal Send type to trait expected type let send_table = Arc::new(dataset) as Arc; - let raw = Arc::into_raw(send_table); - let wide: *const dyn TableProvider = raw as *const _; - let arc = unsafe { Arc::from_raw(wide) }; - Some(arc) + Some(table_provider_send_to_table_provider(send_table)) } Err(_) => None, }; diff --git a/src/context.rs b/src/context.rs index 670b079f3..6d7660744 100644 --- a/src/context.rs +++ b/src/context.rs @@ -46,7 +46,10 @@ 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, get_tokio_runtime, table_provider_from_pycapsule, validate_pycapsule, + wait_for_future, +}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::arrow::record_batch::RecordBatch; @@ -72,7 +75,6 @@ use datafusion::prelude::{ AvroReadOptions, CsvReadOptions, DataFrame, NdJsonReadOptions, ParquetReadOptions, }; use datafusion_ffi::catalog_provider::{FFI_CatalogProvider, ForeignCatalogProvider}; -use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::types::{PyCapsule, PyDict, PyList, PyTuple, PyType}; use pyo3::IntoPyObjectExt; use tokio::task::JoinHandle; @@ -608,23 +610,15 @@ impl PySessionContext { name: &str, table_provider: Bound<'_, PyAny>, ) -> PyDataFusionResult<()> { - let provider = if table_provider.hasattr("__datafusion_table_provider__")? { - let capsule = table_provider - .getattr("__datafusion_table_provider__")? - .call0()?; - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - Arc::new(provider) as Arc - } else if let Ok(py_table) = table_provider.extract::() { + let provider = if let Ok(py_table) = table_provider.extract::() { py_table.table() } else if let Ok(py_provider) = table_provider.extract::() { py_provider.into_inner() + } else if let Some(provider) = table_provider_from_pycapsule(&table_provider)? { + provider } else { return Err(crate::errors::PyDataFusionError::Common( - "Expected a Table or TableProvider.".to_string(), + "Expected a Table or TableProvider. Convert DataFrames with \"DataFrame.into_view()\" or \"TableProvider.from_dataframe()\".".to_string(), )); }; diff --git a/src/dataframe.rs b/src/dataframe.rs index 76cb53ec0..9a531e59b 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -268,8 +268,7 @@ impl PyDataFrame { } } - #[allow(clippy::wrong_self_convention)] - pub(crate) fn into_view_provider(&self) -> Arc { + pub(crate) fn to_view_provider(&self) -> Arc { self.df.as_ref().clone().into_view() } @@ -400,17 +399,14 @@ impl PyDataFrame { /// Convert this DataFrame into a Table that can be used in register_table /// By convention, into_... methods consume self and return the new object. - /// Disabling the clippy lint, so we can use &self - /// because we're working with Python bindings - /// where objects are shared + /// Here we intentionally borrow to avoid invalidating the Python wrapper. /// https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 /// - we have not decided on the table_provider approach yet - #[allow(clippy::wrong_self_convention)] pub fn into_view(&self) -> PyDataFusionResult { // Call the underlying Rust DataFrame::into_view method. // Note that the Rust method consumes self; here we clone the inner Arc // so that we don't invalidate this PyDataFrame. - let table_provider = self.into_view_provider(); + let table_provider = self.to_view_provider(); Ok(PyTableProvider::new(table_provider)) } diff --git a/src/lib.rs b/src/lib.rs index 19e3bc504..80e9b5069 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -89,7 +89,6 @@ fn _internal(py: Python, m: Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; m.add_class::()?; - m.add_class::()?; m.add_class::()?; m.add_class::()?; m.add_class::()?; diff --git a/src/table.rs b/src/table.rs index c6a9a640f..4ee58b81a 100644 --- a/src/table.rs +++ b/src/table.rs @@ -79,9 +79,9 @@ impl PyTableProvider { /// /// This method simply delegates to `DataFrame.into_view`. #[staticmethod] - pub fn from_dataframe(df: &PyDataFrame) -> PyDataFusionResult { - let table_provider = df.into_view_provider(); - Ok(Self::new(table_provider)) + pub fn from_dataframe(df: &PyDataFrame) -> Self { + let table_provider = df.to_view_provider(); + Self::new(table_provider) } /// Create a `TableProvider` from a `DataFrame` by converting it into a view. @@ -97,7 +97,7 @@ impl PyTableProvider { py.get_type::(), ), )?; - Self::from_dataframe(df) + Ok(Self::from_dataframe(df)) } fn __datafusion_table_provider__<'py>( diff --git a/src/udtf.rs b/src/udtf.rs index a14b48c36..ea3886084 100644 --- a/src/udtf.rs +++ b/src/udtf.rs @@ -21,11 +21,12 @@ use std::sync::Arc; use crate::errors::{py_datafusion_err, to_datafusion_err}; use crate::expr::PyExpr; use crate::table::PyTableProvider; -use crate::utils::validate_pycapsule; +use crate::utils::{ + table_provider_from_pycapsule, table_provider_send_to_table_provider, validate_pycapsule, +}; use datafusion::catalog::{TableFunctionImpl, TableProvider}; use datafusion::error::Result as DataFusionResult; use datafusion::logical_expr::Expr; -use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use datafusion_ffi::udtf::{FFI_TableFunction, ForeignTableFunction}; use pyo3::exceptions::PyNotImplementedError; use pyo3::types::{PyCapsule, PyTuple}; @@ -99,20 +100,11 @@ fn call_python_table_function( let provider_obj = func.call1(py, py_args)?; let provider = provider_obj.bind(py); - if provider.hasattr("__datafusion_table_provider__")? { - let capsule = provider.getattr("__datafusion_table_provider__")?.call0()?; - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - - Ok(Arc::new(provider) as Arc) - } else { - Err(PyNotImplementedError::new_err( + table_provider_from_pycapsule(provider)?.ok_or_else(|| { + PyNotImplementedError::new_err( "__datafusion_table_provider__ does not exist on Table Provider object.", - )) - } + ) + }) }) .map_err(to_datafusion_err) } @@ -123,11 +115,7 @@ impl TableFunctionImpl for PyTableFunction { PyTableFunctionInner::FFIFunction(func) => func.call(args), PyTableFunctionInner::PythonFunction(obj) => { let send_result = call_python_table_function(obj, args)?; - // Convert from our Send type to the trait expected type - let raw = Arc::into_raw(send_result); - let wide: *const dyn TableProvider = raw as *const _; - let arc = unsafe { Arc::from_raw(wide) }; - Ok(arc) + Ok(table_provider_send_to_table_provider(send_result)) } } } diff --git a/src/utils.rs b/src/utils.rs index 3b30de5de..9bf665de5 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -17,15 +17,21 @@ use crate::{ common::data_type::PyScalarValue, - errors::{PyDataFusionError, PyDataFusionResult}, + errors::{py_datafusion_err, PyDataFusionError, PyDataFusionResult}, TokioRuntime, }; use datafusion::{ - common::ScalarValue, execution::context::SessionContext, logical_expr::Volatility, + common::ScalarValue, datasource::TableProvider, execution::context::SessionContext, + logical_expr::Volatility, }; +use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::prelude::*; use pyo3::{exceptions::PyValueError, types::PyCapsule}; -use std::{future::Future, sync::OnceLock, time::Duration}; +use std::{ + future::Future, + sync::{Arc, OnceLock}, + time::Duration, +}; use tokio::{runtime::Runtime, time::sleep}; /// Utility to get the Tokio Runtime from Python #[inline] @@ -91,7 +97,7 @@ pub(crate) fn parse_volatility(value: &str) -> PyDataFusionResult { "volatile" => Volatility::Volatile, value => { return Err(PyDataFusionError::Common(format!( - "Unsupportad volatility type: `{value}`, supported \ + "Unsupported volatility type: `{value}`, supported \ values are: immutable, stable and volatile." ))) } @@ -101,9 +107,9 @@ pub(crate) fn parse_volatility(value: &str) -> PyDataFusionResult { pub(crate) fn validate_pycapsule(capsule: &Bound, name: &str) -> PyResult<()> { let capsule_name = capsule.name()?; if capsule_name.is_none() { - return Err(PyValueError::new_err( - "Expected schema PyCapsule to have name set.", - )); + return Err(PyValueError::new_err(format!( + "Expected {name} PyCapsule to have name set." + ))); } let capsule_name = capsule_name.unwrap().to_str()?; @@ -116,6 +122,56 @@ pub(crate) fn validate_pycapsule(capsule: &Bound, name: &str) -> PyRe Ok(()) } +/// Convert a [`TableProvider`] wrapped in an [`Arc`] with a `Send` auto trait into one +/// without the marker. +/// +/// # Safety +/// +/// Removing `Send` from a trait object only relaxes the bounds. The underlying vtable is +/// unchanged, so it is safe to reuse the pointer produced by [`Arc::into_raw`]. +pub(crate) fn table_provider_send_to_table_provider( + table: Arc, +) -> Arc { + let raw: *const (dyn TableProvider + Send) = Arc::into_raw(table); + // SAFETY: `Send` is an auto trait with no associated data, so the trait object layout + // is identical and the pointer may be reinterpreted without changing the reference + // count. + unsafe { Arc::from_raw(raw as *const dyn TableProvider) } +} + +/// Convert a [`TableProvider`] wrapped in an [`Arc`] into one that also carries the `Send` +/// auto trait. +/// +/// # Safety +/// +/// DataFusion's `TableProvider` trait requires `Send`, so the underlying provider implements +/// the marker. This allows us to reinterpret the pointer as a `TableProvider + Send` trait +/// object. +pub(crate) fn table_provider_to_send( + table: Arc, +) -> Arc { + let raw: *const dyn TableProvider = Arc::into_raw(table); + // SAFETY: The underlying type implements `Send`, so the pointer can be safely treated as + // a `TableProvider + Send` trait object. + unsafe { Arc::from_raw(raw as *const (dyn TableProvider + Send)) } +} + +pub(crate) fn table_provider_from_pycapsule( + obj: &Bound, +) -> PyResult>> { + if obj.hasattr("__datafusion_table_provider__")? { + let capsule = obj.getattr("__datafusion_table_provider__")?.call0()?; + let capsule = capsule.downcast::().map_err(py_datafusion_err)?; + validate_pycapsule(capsule, "datafusion_table_provider")?; + + let provider = unsafe { capsule.reference::() }; + let provider: ForeignTableProvider = provider.into(); + Ok(Some(Arc::new(provider) as Arc)) + } else { + Ok(None) + } +} + pub(crate) fn py_obj_to_scalar_value(py: Python, obj: PyObject) -> PyResult { // convert Python object to PyScalarValue to ScalarValue diff --git a/tests/dataframe_into_view.rs b/tests/dataframe_into_view.rs deleted file mode 100644 index 39e9453d5..000000000 --- a/tests/dataframe_into_view.rs +++ /dev/null @@ -1,39 +0,0 @@ -use std::sync::Arc; - -use datafusion::arrow::array::Int32Array; -use datafusion::arrow::datatypes::{DataType, Field, Schema}; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::datasource::MemTable; -use datafusion::prelude::SessionContext; -use datafusion_python::dataframe::PyDataFrame; - -#[test] -fn dataframe_into_view_returns_table_provider() { - // Create an in-memory table with one Int32 column. - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], - ) - .unwrap(); - let table = MemTable::try_new(schema, vec![vec![batch]]).unwrap(); - - // Build a DataFrame from the table and convert it into a view. - let ctx = SessionContext::new(); - let df = ctx.read_table(Arc::new(table)).unwrap(); - let py_df = PyDataFrame::new(df); - let provider = py_df.into_view().unwrap(); - - // Register the view in a new context and ensure it can be queried. - let ctx = SessionContext::new(); - ctx.register_table("view", provider.into_inner()).unwrap(); - - let rt = tokio::runtime::Runtime::new().unwrap(); - let batches = rt.block_on(async { - let df = ctx.sql("SELECT * FROM view").await.unwrap(); - df.collect().await.unwrap() - }); - - assert_eq!(batches.len(), 1); - assert_eq!(batches[0].num_rows(), 3); -} From 512442b7defc225df86895f9f8767f91478bf90f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 16 Sep 2025 15:10:51 +0800 Subject: [PATCH 03/34] TableProvider refactor & PyDataFrame integration MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Removed unused helpers (`extract_table_provider`, `_wrap`) and dead code to simplify maintenance. * Consolidated and streamlined table-provider extraction and registration logic; improved error handling and replaced a hardcoded error message with `EXPECTED_PROVIDER_MSG`. * Marked `from_view` as deprecated; updated deprecation message formatting and adjusted the warning `stacklevel` so it points to caller code. * Removed the `Send` marker from TableProvider trait objects to increase type flexibility — review threading assumptions. * Added type hints to `register_schema` and `deregister_table` methods. * Adjusted tests and exceptions (e.g., changed one test to expect `RuntimeError`) and updated test coverage accordingly. * Introduced a refactored `TableProvider` class and enhanced Python integration by adding support for extracting `PyDataFrame` in `PySchema`. Notes: * Consumers should migrate away from `TableProvider::from_view` to the new TableProvider integration. * Audit any code relying on `Send` for trait objects passed across threads. * Update downstream tests and documentation to reflect the changed exception types and deprecation. --- python/datafusion/__init__.py | 4 +- python/datafusion/catalog.py | 10 ++- python/datafusion/dataframe.py | 6 +- python/datafusion/table_provider.py | 102 ++++++++++++++++++++++++++++ python/tests/test_context.py | 12 ++-- src/catalog.rs | 34 ++++------ src/context.rs | 6 +- src/dataframe.rs | 2 +- src/lib.rs | 2 + src/table.rs | 20 +++--- src/udtf.rs | 11 +-- src/utils.rs | 41 ++--------- 12 files changed, 160 insertions(+), 90 deletions(-) create mode 100644 python/datafusion/table_provider.py diff --git a/python/datafusion/__init__.py b/python/datafusion/__init__.py index 9fae68b53..b622e6925 100644 --- a/python/datafusion/__init__.py +++ b/python/datafusion/__init__.py @@ -33,7 +33,7 @@ from . import functions, object_store, substrait, unparser # The following imports are okay to remain as opaque to the user. -from ._internal import Config, TableProvider +from ._internal import Config, EXPECTED_PROVIDER_MSG from .catalog import Catalog, Database, Table from .col import col, column from .common import ( @@ -54,6 +54,7 @@ from .io import read_avro, read_csv, read_json, read_parquet from .plan import ExecutionPlan, LogicalPlan from .record_batch import RecordBatch, RecordBatchStream +from .table_provider import TableProvider from .user_defined import ( Accumulator, AggregateUDF, @@ -76,6 +77,7 @@ "DFSchema", "DataFrame", "Database", + "EXPECTED_PROVIDER_MSG", "ExecutionPlan", "Expr", "LogicalPlan", diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index 875608f18..595bb2e92 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -85,7 +85,11 @@ def database(self, name: str = "public") -> Schema: """Returns the database with the given ``name`` from this catalog.""" return self.schema(name) - def register_schema(self, name, schema) -> Schema | None: + def register_schema( + self, + name: str, + schema: Schema | SchemaProvider | SchemaProviderExportable, + ) -> Schema | None: """Register a schema with this catalog.""" if isinstance(schema, Schema): return self.catalog.register_schema(name, schema._raw_schema) @@ -126,7 +130,7 @@ def table(self, name: str) -> Table: return Table(self._raw_schema.table(name)) def register_table( - self, name, table: Table | TableProvider | TableProviderExportable + self, name: str, table: Table | TableProvider | TableProviderExportable ) -> None: """Register a table or table provider in this schema. @@ -240,7 +244,7 @@ def register_table( # noqa: B027 and treated as :class:`TableProvider` instances. """ - def deregister_table(self, name, cascade: bool) -> None: # noqa: B027 + def deregister_table(self, name: str, cascade: bool) -> None: # noqa: B027 """Remove a table from this schema. This method is optional. If your schema provides a fixed list of tables, you do diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 671c3aeb1..6cd569dca 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -52,8 +52,8 @@ import polars as pl import pyarrow as pa - from datafusion._internal import TableProvider from datafusion._internal import expr as expr_internal + from datafusion.table_provider import TableProvider from enum import Enum @@ -316,7 +316,9 @@ def into_view(self) -> TableProvider: ``TableProvider.from_dataframe`` calls this method under the hood, and the older ``TableProvider.from_view`` helper is deprecated. """ - return self.df.into_view() + from datafusion.table_provider import TableProvider as _TableProvider + + return _TableProvider(self.df.into_view()) def __getitem__(self, key: str | list[str]) -> DataFrame: """Return a new :py:class`DataFrame` with the specified column or columns. diff --git a/python/datafusion/table_provider.py b/python/datafusion/table_provider.py new file mode 100644 index 000000000..e086476ae --- /dev/null +++ b/python/datafusion/table_provider.py @@ -0,0 +1,102 @@ +# 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. +"""Wrapper helpers for :mod:`datafusion._internal.TableProvider`.""" + +from __future__ import annotations + +import warnings +from typing import Any + +import datafusion._internal as df_internal +from datafusion._internal import EXPECTED_PROVIDER_MSG + +_InternalTableProvider = df_internal.TableProvider + + +class TableProvider: + """High level wrapper around :mod:`datafusion._internal.TableProvider`.""" + + __slots__ = ("_table_provider",) + + def __init__(self, table_provider: _InternalTableProvider) -> None: + """Wrap a low level :class:`~datafusion._internal.TableProvider`.""" + if isinstance(table_provider, TableProvider): + table_provider = table_provider._table_provider + + if not isinstance(table_provider, _InternalTableProvider): + raise TypeError(EXPECTED_PROVIDER_MSG) + + self._table_provider = table_provider + + @classmethod + def from_capsule(cls, capsule: Any) -> TableProvider: + """Create a :class:`TableProvider` from a PyCapsule.""" + provider = _InternalTableProvider.from_capsule(capsule) + return cls(provider) + + @classmethod + def from_dataframe(cls, df: Any) -> TableProvider: + """Create a :class:`TableProvider` from a :class:`DataFrame`.""" + from datafusion.dataframe import DataFrame as DataFrameWrapper + + if isinstance(df, DataFrameWrapper): + df = df.df + + provider = _InternalTableProvider.from_dataframe(df) + return cls(provider) + + @classmethod + def from_view(cls, df: Any) -> TableProvider: + """Deprecated. + + Use :meth:`DataFrame.into_view` or :meth:`TableProvider.from_dataframe`. + """ + from datafusion.dataframe import DataFrame as DataFrameWrapper + + if isinstance(df, DataFrameWrapper): + df = df.df + + provider = _InternalTableProvider.from_view(df) + warnings.warn( + "TableProvider.from_view is deprecated; use DataFrame.into_view or " + "TableProvider.from_dataframe instead.", + DeprecationWarning, + stacklevel=2, + ) + return cls(provider) + + # ------------------------------------------------------------------ + # passthrough helpers + # ------------------------------------------------------------------ + def __getattr__(self, name: str) -> Any: + """Delegate attribute lookup to the wrapped provider.""" + return getattr(self._table_provider, name) + + def __dir__(self) -> list[str]: + """Expose delegated attributes via :func:`dir`.""" + return dir(self._table_provider) + super().__dir__() + + def __repr__(self) -> str: # pragma: no cover - simple delegation + """Return a representation of the wrapped provider.""" + return repr(self._table_provider) + + def __datafusion_table_provider__(self) -> Any: + """Expose the wrapped provider for FFI integrations.""" + return self._table_provider.__datafusion_table_provider__() + + +__all__ = ["TableProvider"] diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 215b5df2f..93795efc9 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -23,6 +23,7 @@ import pytest from datafusion import ( DataFrame, + EXPECTED_PROVIDER_MSG, RuntimeEnvBuilder, SessionConfig, SessionContext, @@ -350,7 +351,7 @@ def test_table_provider_from_capsule(ctx): def test_table_provider_from_dataframe(ctx): - df = ctx.from_pydict({"a": [1, 2]}).df + df = ctx.from_pydict({"a": [1, 2]}) provider = TableProvider.from_dataframe(df) ctx.register_table("from_dataframe_tbl", provider) result = ctx.sql("SELECT * FROM from_dataframe_tbl").collect() @@ -374,19 +375,16 @@ def __datafusion_table_provider__(self): def test_table_provider_from_capsule_invalid(): - with pytest.raises(Exception): # noqa: B017 + with pytest.raises(RuntimeError): TableProvider.from_capsule(object()) def test_register_table_with_dataframe_errors(ctx): df = ctx.from_pydict({"a": [1]}) - with pytest.raises(Exception) as exc_info: # noqa: B017 + with pytest.raises(Exception) as exc_info: ctx.register_table("bad", df) - assert ( - str(exc_info.value) - == 'Expected a Table or TableProvider. Convert DataFrames with "DataFrame.into_view()" or "TableProvider.from_dataframe()".' - ) + assert str(exc_info.value) == EXPECTED_PROVIDER_MSG def test_register_dataset(ctx): diff --git a/src/catalog.rs b/src/catalog.rs index 9fd43d703..5b6de0740 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. +use crate::dataframe::PyDataFrame; use crate::dataset::Dataset; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; use crate::table::PyTableProvider; use crate::utils::{ - table_provider_from_pycapsule, table_provider_send_to_table_provider, table_provider_to_send, - validate_pycapsule, wait_for_future, + table_provider_from_pycapsule, validate_pycapsule, wait_for_future, EXPECTED_PROVIDER_MSG, }; use async_trait::async_trait; use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; @@ -54,7 +54,7 @@ pub struct PySchema { #[pyclass(name = "RawTable", module = "datafusion.catalog", subclass)] #[derive(Clone)] pub struct PyTable { - pub table: Arc, + pub table: Arc, } impl From> for PyCatalog { @@ -70,11 +70,11 @@ impl From> for PySchema { } impl PyTable { - pub fn new(table: Arc) -> Self { + pub fn new(table: Arc) -> Self { Self { table } } - pub fn table(&self) -> Arc { + pub fn table(&self) -> Arc { self.table.clone() } } @@ -203,12 +203,14 @@ impl PySchema { py_table.table } else if let Ok(py_provider) = table_provider.extract::() { py_provider.into_inner() + } else if table_provider.extract::().is_ok() { + return Err(PyDataFusionError::Common(EXPECTED_PROVIDER_MSG.to_string()).into()); } else if let Some(provider) = table_provider_from_pycapsule(&table_provider)? { provider } else { let py = table_provider.py(); let provider = Dataset::new(&table_provider, py)?; - Arc::new(provider) as Arc + Arc::new(provider) as Arc }; let _ = self @@ -288,7 +290,7 @@ impl RustWrappedPySchemaProvider { } } - fn table_inner(&self, name: &str) -> PyResult>> { + fn table_inner(&self, name: &str) -> PyResult>> { Python::with_gil(|py| { let provider = self.schema_provider.bind(py); let py_table_method = provider.getattr("table")?; @@ -315,7 +317,7 @@ impl RustWrappedPySchemaProvider { Ok(py_table) => Ok(Some(py_table.table)), Err(_) => { let ds = Dataset::new(&py_table, py).map_err(py_datafusion_err)?; - Ok(Some(Arc::new(ds) as Arc)) + Ok(Some(Arc::new(ds) as Arc)) } } } @@ -351,11 +353,7 @@ impl SchemaProvider for RustWrappedPySchemaProvider { &self, name: &str, ) -> datafusion::common::Result>, DataFusionError> { - // Convert from our internal Send type to the trait expected type - match self.table_inner(name).map_err(to_datafusion_err)? { - Some(table) => Ok(Some(table_provider_send_to_table_provider(table))), - None => Ok(None), - } + self.table_inner(name).map_err(to_datafusion_err) } fn register_table( @@ -363,10 +361,7 @@ impl SchemaProvider for RustWrappedPySchemaProvider { name: String, table: Arc, ) -> datafusion::common::Result>> { - // Convert from trait type to our internal Send type - let send_table = table_provider_to_send(table); - - let py_table = PyTable::new(send_table); + let py_table = PyTable::new(table); Python::with_gil(|py| { let provider = self.schema_provider.bind(py); let _ = provider @@ -395,10 +390,7 @@ impl SchemaProvider for RustWrappedPySchemaProvider { // If we can turn this table provider into a `Dataset`, return it. // Otherwise, return None. let dataset = match Dataset::new(&table, py) { - Ok(dataset) => { - let send_table = Arc::new(dataset) as Arc; - Some(table_provider_send_to_table_provider(send_table)) - } + Ok(dataset) => Some(Arc::new(dataset) as Arc), Err(_) => None, }; diff --git a/src/context.rs b/src/context.rs index 6d7660744..8845f3c39 100644 --- a/src/context.rs +++ b/src/context.rs @@ -48,7 +48,7 @@ use crate::udtf::PyTableFunction; use crate::udwf::PyWindowUDF; use crate::utils::{ get_global_ctx, get_tokio_runtime, table_provider_from_pycapsule, validate_pycapsule, - wait_for_future, + wait_for_future, EXPECTED_PROVIDER_MSG, }; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; @@ -618,7 +618,7 @@ impl PySessionContext { provider } else { return Err(crate::errors::PyDataFusionError::Common( - "Expected a Table or TableProvider. Convert DataFrames with \"DataFrame.into_view()\" or \"TableProvider.from_dataframe()\".".to_string(), + EXPECTED_PROVIDER_MSG.to_string(), )); }; @@ -852,7 +852,7 @@ impl PySessionContext { dataset: &Bound<'_, PyAny>, py: Python, ) -> PyDataFusionResult<()> { - let table: Arc = Arc::new(Dataset::new(dataset, py)?); + let table: Arc = Arc::new(Dataset::new(dataset, py)?); self.ctx.register_table(name, table)?; diff --git a/src/dataframe.rs b/src/dataframe.rs index 9a531e59b..6d9f728ed 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -268,7 +268,7 @@ impl PyDataFrame { } } - pub(crate) fn to_view_provider(&self) -> Arc { + pub(crate) fn to_view_provider(&self) -> Arc { self.df.as_ref().clone().into_view() } diff --git a/src/lib.rs b/src/lib.rs index 80e9b5069..d45992db2 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -81,6 +81,8 @@ fn _internal(py: Python, m: Bound<'_, PyModule>) -> PyResult<()> { // Initialize logging pyo3_log::init(); + m.add("EXPECTED_PROVIDER_MSG", crate::utils::EXPECTED_PROVIDER_MSG)?; + // Register the python classes m.add_class::()?; m.add_class::()?; diff --git a/src/table.rs b/src/table.rs index 4ee58b81a..06c6666df 100644 --- a/src/table.rs +++ b/src/table.rs @@ -22,7 +22,7 @@ use datafusion::datasource::TableProvider; use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::exceptions::PyDeprecationWarning; use pyo3::prelude::*; -use pyo3::types::PyCapsule; +use pyo3::types::{PyCapsule, PyDict}; use crate::catalog::PyTable; use crate::dataframe::PyDataFrame; @@ -33,30 +33,30 @@ use crate::utils::{get_tokio_runtime, validate_pycapsule}; #[pyclass(name = "TableProvider", module = "datafusion")] #[derive(Clone)] pub struct PyTableProvider { - pub(crate) provider: Arc, + pub(crate) provider: Arc, } impl PyTableProvider { - pub(crate) fn new(provider: Arc) -> Self { + pub(crate) fn new(provider: Arc) -> Self { Self { provider } } /// Return a `PyTable` wrapper around this provider. /// /// Historically callers chained `as_table().table()` to access the - /// underlying `Arc`. Prefer [`as_arc`] or + /// underlying [`Arc`]. Prefer [`as_arc`] or /// [`into_inner`] for direct access instead. pub fn as_table(&self) -> PyTable { PyTable::new(Arc::clone(&self.provider)) } /// Return a clone of the inner [`TableProvider`]. - pub fn as_arc(&self) -> Arc { + pub fn as_arc(&self) -> Arc { Arc::clone(&self.provider) } /// Consume this wrapper and return the inner [`TableProvider`]. - pub fn into_inner(self) -> Arc { + pub fn into_inner(self) -> Arc { self.provider } } @@ -90,12 +90,15 @@ impl PyTableProvider { /// `TableProvider.from_dataframe` instead. #[staticmethod] pub fn from_view(py: Python<'_>, df: &PyDataFrame) -> PyDataFusionResult { - py.import("warnings")?.call_method1( + let kwargs = PyDict::new(py); + kwargs.set_item("stacklevel", 3)?; + py.import("warnings")?.call_method( "warn", ( "PyTableProvider.from_view() is deprecated; use DataFrame.into_view() or TableProvider.from_dataframe() instead.", py.get_type::(), ), + Some(&kwargs), )?; Ok(Self::from_dataframe(df)) } @@ -107,7 +110,8 @@ impl PyTableProvider { let name = CString::new("datafusion_table_provider").unwrap(); let runtime = get_tokio_runtime().0.handle().clone(); - let provider = FFI_TableProvider::new(Arc::clone(&self.provider), false, Some(runtime)); + let provider: Arc = self.provider.clone(); + let provider = FFI_TableProvider::new(provider, false, Some(runtime)); PyCapsule::new(py, provider, Some(name.clone())) } diff --git a/src/udtf.rs b/src/udtf.rs index ea3886084..311d67e24 100644 --- a/src/udtf.rs +++ b/src/udtf.rs @@ -21,9 +21,7 @@ use std::sync::Arc; use crate::errors::{py_datafusion_err, to_datafusion_err}; use crate::expr::PyExpr; use crate::table::PyTableProvider; -use crate::utils::{ - table_provider_from_pycapsule, table_provider_send_to_table_provider, validate_pycapsule, -}; +use crate::utils::{table_provider_from_pycapsule, validate_pycapsule}; use datafusion::catalog::{TableFunctionImpl, TableProvider}; use datafusion::error::Result as DataFusionResult; use datafusion::logical_expr::Expr; @@ -88,7 +86,7 @@ impl PyTableFunction { fn call_python_table_function( func: &Arc, args: &[Expr], -) -> DataFusionResult> { +) -> DataFusionResult> { let args = args .iter() .map(|arg| PyExpr::from(arg.clone())) @@ -113,10 +111,7 @@ impl TableFunctionImpl for PyTableFunction { fn call(&self, args: &[Expr]) -> DataFusionResult> { match &self.inner { PyTableFunctionInner::FFIFunction(func) => func.call(args), - PyTableFunctionInner::PythonFunction(obj) => { - let send_result = call_python_table_function(obj, args)?; - Ok(table_provider_send_to_table_provider(send_result)) - } + PyTableFunctionInner::PythonFunction(obj) => call_python_table_function(obj, args), } } } diff --git a/src/utils.rs b/src/utils.rs index 9bf665de5..3f888abc0 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -33,6 +33,9 @@ use std::{ time::Duration, }; use tokio::{runtime::Runtime, time::sleep}; + +pub(crate) const EXPECTED_PROVIDER_MSG: &str = + "Expected a Table or TableProvider. Convert DataFrames with \"DataFrame.into_view()\" or \"TableProvider.from_dataframe()\"."; /// Utility to get the Tokio Runtime from Python #[inline] pub(crate) fn get_tokio_runtime() -> &'static TokioRuntime { @@ -122,43 +125,9 @@ pub(crate) fn validate_pycapsule(capsule: &Bound, name: &str) -> PyRe Ok(()) } -/// Convert a [`TableProvider`] wrapped in an [`Arc`] with a `Send` auto trait into one -/// without the marker. -/// -/// # Safety -/// -/// Removing `Send` from a trait object only relaxes the bounds. The underlying vtable is -/// unchanged, so it is safe to reuse the pointer produced by [`Arc::into_raw`]. -pub(crate) fn table_provider_send_to_table_provider( - table: Arc, -) -> Arc { - let raw: *const (dyn TableProvider + Send) = Arc::into_raw(table); - // SAFETY: `Send` is an auto trait with no associated data, so the trait object layout - // is identical and the pointer may be reinterpreted without changing the reference - // count. - unsafe { Arc::from_raw(raw as *const dyn TableProvider) } -} - -/// Convert a [`TableProvider`] wrapped in an [`Arc`] into one that also carries the `Send` -/// auto trait. -/// -/// # Safety -/// -/// DataFusion's `TableProvider` trait requires `Send`, so the underlying provider implements -/// the marker. This allows us to reinterpret the pointer as a `TableProvider + Send` trait -/// object. -pub(crate) fn table_provider_to_send( - table: Arc, -) -> Arc { - let raw: *const dyn TableProvider = Arc::into_raw(table); - // SAFETY: The underlying type implements `Send`, so the pointer can be safely treated as - // a `TableProvider + Send` trait object. - unsafe { Arc::from_raw(raw as *const (dyn TableProvider + Send)) } -} - pub(crate) fn table_provider_from_pycapsule( obj: &Bound, -) -> PyResult>> { +) -> PyResult>> { if obj.hasattr("__datafusion_table_provider__")? { let capsule = obj.getattr("__datafusion_table_provider__")?.call0()?; let capsule = capsule.downcast::().map_err(py_datafusion_err)?; @@ -166,7 +135,7 @@ pub(crate) fn table_provider_from_pycapsule( let provider = unsafe { capsule.reference::() }; let provider: ForeignTableProvider = provider.into(); - Ok(Some(Arc::new(provider) as Arc)) + Ok(Some(Arc::new(provider) as Arc)) } else { Ok(None) } From a8275dc0f614a6f17a901c93f799550bde2fe0ea Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 16 Sep 2025 18:19:59 +0800 Subject: [PATCH 04/34] Normalize & simplify TableProvider/DataFrame registration; add utilities, docs, and robustness fixes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Normalized table-provider handling and simplified registration flow across the codebase; multiple commits centralize provider coercion and normalization. * Introduced utility helpers (`coerce_table_provider`, `extract_table_provider`, `_normalize_table_provider`) to centralize extraction, error handling, and improve clarity. * Simplified `from_dataframe` / `into_view` behavior: clearer implementations, direct returns of DataFrame views where appropriate, and added internal tests for DataFrame flows. * Fixed DataFrame registration semantics: enforce `TypeError` for invalid registrations; added handling for `DataFrameWrapper` by converting it to a view. * Added tests, including a schema registration test using a PyArrow dataset and internal DataFrame tests to cover new flows. * Documentation improvements: expanded `from_dataframe` docstrings with parameter details, added usage examples for `into_view`, and documented deprecations (e.g., `register_table_provider` → `register_table`). * Warning and UX fixes: synchronized deprecation `stacklevel` so warnings point to caller code; improved `__dir__` to return sorted, unique attributes. * Cleanup: removed unused imports (including an unused error import from `utils.rs`) and other dead code to reduce noise. --- dev/changelog/49.0.0.md | 4 + docs/source/user-guide/data-sources.rst | 9 ++- docs/source/user-guide/io/table_provider.rst | 7 ++ python/datafusion/catalog.py | 6 +- python/datafusion/context.py | 18 ++--- python/datafusion/dataframe.py | 14 +++- python/datafusion/table_provider.py | 29 +++++-- python/datafusion/utils.py | 80 ++++++++++++++++++++ python/tests/test_catalog.py | 34 ++++++++- python/tests/test_context.py | 56 +++++++++++++- src/catalog.rs | 17 +---- src/context.rs | 16 +--- src/table.rs | 3 +- src/utils.rs | 37 +++++++-- 14 files changed, 268 insertions(+), 62 deletions(-) create mode 100644 python/datafusion/utils.py diff --git a/dev/changelog/49.0.0.md b/dev/changelog/49.0.0.md index 008bd43bc..15b3c2382 100644 --- a/dev/changelog/49.0.0.md +++ b/dev/changelog/49.0.0.md @@ -25,6 +25,10 @@ This release consists of 16 commits from 7 contributors. See credits at the end - fix(build): Include build.rs in published crates [#1199](https://github.com/apache/datafusion-python/pull/1199) (colinmarc) +**Deprecations:** + +- Document that `SessionContext.register_table_provider` is deprecated in favor of `SessionContext.register_table`. + **Other:** - 48.0.0 Release [#1175](https://github.com/apache/datafusion-python/pull/1175) (timsaucer) diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index 95e1343b7..62beec4f1 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -160,7 +160,14 @@ as Delta Lake. This will require a recent version of df = ctx.table("my_delta_table") df.show() -On older versions of ``deltalake`` (prior to 0.22) you can use the +.. note:: + + :py:meth:`~datafusion.context.SessionContext.register_table_provider` is + deprecated. Use + :py:meth:`~datafusion.context.SessionContext.register_table` with a + :py:class:`~datafusion.TableProvider` instead. + +On older versions of ``deltalake`` (prior to 0.22) you can use the `Arrow DataSet `_ interface to import to DataFusion, but this does not support features such as filter push down which can lead to a significant performance difference. diff --git a/docs/source/user-guide/io/table_provider.rst b/docs/source/user-guide/io/table_provider.rst index 720beeccc..0e7746cca 100644 --- a/docs/source/user-guide/io/table_provider.rst +++ b/docs/source/user-guide/io/table_provider.rst @@ -54,6 +54,13 @@ Call the provider's ``__datafusion_table_provider__()`` method to obtain the cap before constructing a ``TableProvider``. The ``TableProvider.from_view()`` helper is deprecated; instead use ``TableProvider.from_dataframe()`` or ``DataFrame.into_view()``. +.. note:: + + :py:meth:`~datafusion.context.SessionContext.register_table_provider` is + deprecated. Use + :py:meth:`~datafusion.context.SessionContext.register_table` with the + resulting :py:class:`~datafusion.TableProvider` instead. + .. code-block:: python from datafusion import SessionContext, TableProvider diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index 595bb2e92..8faf75a0a 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -23,6 +23,7 @@ from typing import TYPE_CHECKING, Protocol import datafusion._internal as df_internal +from datafusion.utils import _normalize_table_provider if TYPE_CHECKING: import pyarrow as pa @@ -137,9 +138,8 @@ def register_table( Objects implementing ``__datafusion_table_provider__`` are also supported and treated as :class:`TableProvider` instances. """ - if isinstance(table, Table): - return self._raw_schema.register_table(name, table.table) - return self._raw_schema.register_table(name, table) + provider = _normalize_table_provider(table) + return self._raw_schema.register_table(name, provider) def deregister_table(self, name: str) -> None: """Deregister a table provider from this schema.""" diff --git a/python/datafusion/context.py b/python/datafusion/context.py index fb2ba4589..f6b65d860 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -34,6 +34,7 @@ from datafusion.expr import Expr, SortExpr, sort_list_to_raw_sort_list from datafusion.record_batch import RecordBatchStream from datafusion.user_defined import AggregateUDF, ScalarUDF, TableFunction, WindowUDF +from datafusion.utils import _normalize_table_provider from ._internal import RuntimeEnvBuilder as RuntimeEnvBuilderInternal from ._internal import SessionConfig as SessionConfigInternal @@ -735,7 +736,7 @@ def from_polars(self, data: pl.DataFrame, name: str | None = None) -> DataFrame: # 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: - """Register a :py:class:`~datafusion.dataframe.DataFrame` as a view. + """Register a :py:class: `~datafusion.detaframe.DataFrame` as a view. Args: name (str): The name to register the view under. @@ -747,17 +748,18 @@ def register_view(self, name: str, df: DataFrame) -> None: def register_table( self, name: str, table: Table | TableProvider | TableProviderExportable ) -> None: - """Register a :py:class:`~datafusion.catalog.Table` or ``TableProvider``. + """Register a :py:class:`~datafusion.catalog.Table` or + :py:class:`~datafusion.TableProvider`. The registered table can be referenced from SQL statements executed against this context. Plain :py:class:`~datafusion.dataframe.DataFrame` objects are not supported; convert them first with :meth:`datafusion.dataframe.DataFrame.into_view` or - :meth:`datafusion.catalog.TableProvider.from_dataframe`. + :meth:`datafusion.TableProvider.from_dataframe`. Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as :class:`~datafusion.catalog.TableProvider` instances. + and treated as :py:class:`~datafusion.TableProvider` instances. Args: name: Name of the resultant table. @@ -765,10 +767,8 @@ def register_table( implementing ``__datafusion_table_provider__`` to add to the session context. """ - if isinstance(table, Table): - self.ctx.register_table(name, table.table) - else: - self.ctx.register_table(name, table) + provider = _normalize_table_provider(table) + self.ctx.register_table(name, provider) def deregister_table(self, name: str) -> None: """Remove a table from the session.""" @@ -795,7 +795,7 @@ def register_table_provider( Deprecated: use :meth:`register_table` instead. Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as :class:`~datafusion.catalog.TableProvider` instances. + and treated as :py:class:`~datafusion.TableProvider` instances. """ warnings.warn( "register_table_provider is deprecated; use register_table", diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 6cd569dca..d160276e1 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -313,8 +313,20 @@ def into_view(self) -> TableProvider: This is the preferred way to obtain a view for :py:meth:`~datafusion.context.SessionContext.register_table`. - ``TableProvider.from_dataframe`` calls this method under the hood, + ``datafusion.TableProvider.from_dataframe`` calls this method under the hood, and the older ``TableProvider.from_view`` helper is deprecated. + + The ``DataFrame`` remains valid after conversion, so it can still be used for + additional queries alongside the returned view. + + Examples: + >>> from datafusion import SessionContext + >>> ctx = SessionContext() + >>> df = ctx.sql("SELECT 1 AS value") + >>> provider = df.into_view() + >>> ctx.register_table("values_view", provider) + >>> df.collect() # The DataFrame is still usable + >>> ctx.sql("SELECT value FROM values_view").collect() """ from datafusion.table_provider import TableProvider as _TableProvider diff --git a/python/datafusion/table_provider.py b/python/datafusion/table_provider.py index e086476ae..e379e2c5d 100644 --- a/python/datafusion/table_provider.py +++ b/python/datafusion/table_provider.py @@ -26,6 +26,9 @@ _InternalTableProvider = df_internal.TableProvider +# Keep in sync with ``datafusion._internal.TableProvider.from_view``. +_FROM_VIEW_WARN_STACKLEVEL = 2 + class TableProvider: """High level wrapper around :mod:`datafusion._internal.TableProvider`.""" @@ -50,14 +53,26 @@ def from_capsule(cls, capsule: Any) -> TableProvider: @classmethod def from_dataframe(cls, df: Any) -> TableProvider: - """Create a :class:`TableProvider` from a :class:`DataFrame`.""" + """Create a :class:`TableProvider` from tabular data. + + Parameters + ---------- + df: + Either a :class:`~datafusion.dataframe.DataFrame` wrapper or the + corresponding :class:`~datafusion._internal.DataFrame`. When + working with third-party DataFrame libraries, convert them via + :meth:`~datafusion.SessionContext.from_arrow` before calling + :meth:`~datafusion.dataframe.DataFrame.into_view` or this + constructor. + """ from datafusion.dataframe import DataFrame as DataFrameWrapper if isinstance(df, DataFrameWrapper): - df = df.df + dataframe = df + else: + dataframe = DataFrameWrapper(df) - provider = _InternalTableProvider.from_dataframe(df) - return cls(provider) + return dataframe.into_view() @classmethod def from_view(cls, df: Any) -> TableProvider: @@ -74,8 +89,8 @@ def from_view(cls, df: Any) -> TableProvider: warnings.warn( "TableProvider.from_view is deprecated; use DataFrame.into_view or " "TableProvider.from_dataframe instead.", - DeprecationWarning, - stacklevel=2, + category=DeprecationWarning, + stacklevel=_FROM_VIEW_WARN_STACKLEVEL, ) return cls(provider) @@ -88,7 +103,7 @@ def __getattr__(self, name: str) -> Any: def __dir__(self) -> list[str]: """Expose delegated attributes via :func:`dir`.""" - return dir(self._table_provider) + super().__dir__() + return sorted(set(super().__dir__()) | set(dir(self._table_provider))) def __repr__(self) -> str: # pragma: no cover - simple delegation """Return a representation of the wrapped provider.""" diff --git a/python/datafusion/utils.py b/python/datafusion/utils.py new file mode 100644 index 000000000..59664236d --- /dev/null +++ b/python/datafusion/utils.py @@ -0,0 +1,80 @@ +# 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. +"""Miscellaneous helper utilities for DataFusion's Python bindings.""" + +from __future__ import annotations + +from importlib import import_module, util +from typing import TYPE_CHECKING, Any + +from datafusion._internal import EXPECTED_PROVIDER_MSG + +_PYARROW_DATASET_TYPES: tuple[type[Any], ...] +_dataset_spec = util.find_spec("pyarrow.dataset") +if _dataset_spec is None: # pragma: no cover - optional dependency at runtime + _PYARROW_DATASET_TYPES = () +else: # pragma: no cover - exercised in environments with pyarrow installed + _dataset_module = import_module("pyarrow.dataset") + dataset_base = getattr(_dataset_module, "Dataset", None) + dataset_types: set[type[Any]] = set() + if isinstance(dataset_base, type): + dataset_types.add(dataset_base) + for value in vars(_dataset_module).values(): + if isinstance(value, type) and issubclass(value, dataset_base): + dataset_types.add(value) + _PYARROW_DATASET_TYPES = tuple(dataset_types) + +if TYPE_CHECKING: # pragma: no cover - imported for typing only + from datafusion import TableProvider + from datafusion.catalog import Table + from datafusion.context import TableProviderExportable + + +def _normalize_table_provider( + table: Table | TableProvider | TableProviderExportable, +) -> Any: + """Return the underlying provider for supported table inputs. + + Args: + table: A :class:`~datafusion.catalog.Table`, + :class:`~datafusion.table_provider.TableProvider`, or object exporting a + DataFusion table provider via ``__datafusion_table_provider__``. + + Returns: + The object expected by the Rust bindings for table registration. + + Raises: + TypeError: If ``table`` is not a supported table provider input. + """ + + from datafusion.catalog import Table as _Table + from datafusion.table_provider import TableProvider as _TableProvider + + if isinstance(table, _Table): + return table.table + + if isinstance(table, _TableProvider): + return table._table_provider + + if _PYARROW_DATASET_TYPES and isinstance(table, _PYARROW_DATASET_TYPES): + return table + + provider_factory = getattr(table, "__datafusion_table_provider__", None) + if callable(provider_factory): + return table + + raise TypeError(EXPECTED_PROVIDER_MSG) diff --git a/python/tests/test_catalog.py b/python/tests/test_catalog.py index 1f9ecbfc3..fd91d6677 100644 --- a/python/tests/test_catalog.py +++ b/python/tests/test_catalog.py @@ -20,7 +20,7 @@ import pyarrow as pa import pyarrow.dataset as ds import pytest -from datafusion import SessionContext, Table +from datafusion import EXPECTED_PROVIDER_MSG, SessionContext, Table # Note we take in `database` as a variable even though we don't use @@ -164,6 +164,38 @@ def test_python_table_provider(ctx: SessionContext): assert schema.table_names() == {"table4"} +def test_schema_register_table_with_pyarrow_dataset(ctx: SessionContext): + schema = ctx.catalog().schema() + batch = pa.RecordBatch.from_arrays( + [pa.array([1, 2, 3]), pa.array([4, 5, 6])], + names=["a", "b"], + ) + dataset = ds.dataset([batch]) + table_name = "pa_dataset" + + try: + schema.register_table(table_name, dataset) + assert table_name in schema.table_names() + + result = ctx.sql(f"SELECT a, b FROM {table_name}").collect() + + assert len(result) == 1 + assert result[0].column(0) == pa.array([1, 2, 3]) + assert result[0].column(1) == pa.array([4, 5, 6]) + finally: + schema.deregister_table(table_name) + + +def test_schema_register_table_with_dataframe_errors(ctx: SessionContext): + schema = ctx.catalog().schema() + df = ctx.from_pydict({"a": [1]}) + + with pytest.raises(Exception) as exc_info: + schema.register_table("bad", df) + + assert str(exc_info.value) == EXPECTED_PROVIDER_MSG + + def test_in_end_to_end_python_providers(ctx: SessionContext): """Test registering all python providers and running a query against them.""" diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 93795efc9..3e4d84302 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -16,14 +16,16 @@ # under the License. import datetime as dt import gzip +import inspect import pathlib +import warnings import pyarrow as pa import pyarrow.dataset as ds import pytest from datafusion import ( - DataFrame, EXPECTED_PROVIDER_MSG, + DataFrame, RuntimeEnvBuilder, SessionConfig, SessionContext, @@ -353,11 +355,61 @@ def test_table_provider_from_capsule(ctx): def test_table_provider_from_dataframe(ctx): df = ctx.from_pydict({"a": [1, 2]}) provider = TableProvider.from_dataframe(df) + assert isinstance(provider, TableProvider) ctx.register_table("from_dataframe_tbl", provider) result = ctx.sql("SELECT * FROM from_dataframe_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] +def test_table_provider_from_dataframe_internal(ctx): + df = ctx.from_pydict({"a": [1, 2]}) + provider = TableProvider.from_dataframe(df.df) + assert isinstance(provider, TableProvider) + ctx.register_table("from_internal_dataframe_tbl", provider) + result = ctx.sql("SELECT * FROM from_internal_dataframe_tbl").collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + +def test_table_provider_from_view_warning_origin(ctx): + from datafusion.table_provider import TableProvider as WrapperTableProvider + + wrapper_df = ctx.from_pydict({"a": [1]}) + test_path = pathlib.Path(__file__).resolve() + + with warnings.catch_warnings(record=True) as caught: + warnings.simplefilter("always") + call_lineno = inspect.currentframe().f_lineno + 1 + WrapperTableProvider.from_view(wrapper_df) + + assert len(caught) >= 1 + + rust_warning = next( + ( + warning + for warning in caught + if "PyTableProvider.from_view()" in str(warning.message) + ), + None, + ) + assert rust_warning is not None + assert issubclass(rust_warning.category, DeprecationWarning) + assert pathlib.Path(rust_warning.filename).resolve() == test_path + assert rust_warning.lineno == call_lineno + + py_warning = next( + ( + warning + for warning in caught + if "TableProvider.from_view is deprecated" in str(warning.message) + ), + None, + ) + assert py_warning is not None + assert issubclass(py_warning.category, DeprecationWarning) + assert pathlib.Path(py_warning.filename).resolve() == test_path + assert py_warning.lineno == call_lineno + + def test_register_table_capsule_direct(ctx): df = ctx.from_pydict({"a": [1, 2]}) provider = df.into_view() @@ -381,7 +433,7 @@ def test_table_provider_from_capsule_invalid(): def test_register_table_with_dataframe_errors(ctx): df = ctx.from_pydict({"a": [1]}) - with pytest.raises(Exception) as exc_info: + with pytest.raises(TypeError) as exc_info: ctx.register_table("bad", df) assert str(exc_info.value) == EXPECTED_PROVIDER_MSG diff --git a/src/catalog.rs b/src/catalog.rs index 5b6de0740..03e6408cd 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -15,12 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::dataframe::PyDataFrame; use crate::dataset::Dataset; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; use crate::table::PyTableProvider; use crate::utils::{ - table_provider_from_pycapsule, validate_pycapsule, wait_for_future, EXPECTED_PROVIDER_MSG, + coerce_table_provider, table_provider_from_pycapsule, validate_pycapsule, wait_for_future, }; use async_trait::async_trait; use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; @@ -199,19 +198,7 @@ impl PySchema { } fn register_table(&self, name: &str, table_provider: Bound<'_, PyAny>) -> PyResult<()> { - let provider = if let Ok(py_table) = table_provider.extract::() { - py_table.table - } else if let Ok(py_provider) = table_provider.extract::() { - py_provider.into_inner() - } else if table_provider.extract::().is_ok() { - return Err(PyDataFusionError::Common(EXPECTED_PROVIDER_MSG.to_string()).into()); - } else if let Some(provider) = table_provider_from_pycapsule(&table_provider)? { - provider - } else { - let py = table_provider.py(); - let provider = Dataset::new(&table_provider, py)?; - Arc::new(provider) as Arc - }; + let provider = coerce_table_provider(&table_provider).map_err(PyErr::from)?; let _ = self .schema diff --git a/src/context.rs b/src/context.rs index 8845f3c39..d2c9b1c98 100644 --- a/src/context.rs +++ b/src/context.rs @@ -41,14 +41,12 @@ use crate::record_batch::PyRecordBatchStream; use crate::sql::exceptions::py_value_err; use crate::sql::logical::PyLogicalPlan; use crate::store::StorageContexts; -use crate::table::PyTableProvider; 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, table_provider_from_pycapsule, validate_pycapsule, - wait_for_future, EXPECTED_PROVIDER_MSG, + coerce_table_provider, get_global_ctx, get_tokio_runtime, validate_pycapsule, wait_for_future, }; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; @@ -610,17 +608,7 @@ impl PySessionContext { name: &str, table_provider: Bound<'_, PyAny>, ) -> PyDataFusionResult<()> { - let provider = if let Ok(py_table) = table_provider.extract::() { - py_table.table() - } else if let Ok(py_provider) = table_provider.extract::() { - py_provider.into_inner() - } else if let Some(provider) = table_provider_from_pycapsule(&table_provider)? { - provider - } else { - return Err(crate::errors::PyDataFusionError::Common( - EXPECTED_PROVIDER_MSG.to_string(), - )); - }; + let provider = coerce_table_provider(&table_provider)?; self.ctx.register_table(name, provider)?; Ok(()) diff --git a/src/table.rs b/src/table.rs index 06c6666df..85145e564 100644 --- a/src/table.rs +++ b/src/table.rs @@ -91,7 +91,8 @@ impl PyTableProvider { #[staticmethod] pub fn from_view(py: Python<'_>, df: &PyDataFrame) -> PyDataFusionResult { let kwargs = PyDict::new(py); - kwargs.set_item("stacklevel", 3)?; + // Keep stack level consistent with python/datafusion/table_provider.py + kwargs.set_item("stacklevel", 2)?; py.import("warnings")?.call_method( "warn", ( diff --git a/src/utils.rs b/src/utils.rs index 3f888abc0..04f569146 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -16,15 +16,18 @@ // under the License. use crate::{ + catalog::PyTable, common::data_type::PyScalarValue, - errors::{py_datafusion_err, PyDataFusionError, PyDataFusionResult}, + dataframe::PyDataFrame, + dataset::Dataset, + errors::{PyDataFusionError, PyDataFusionResult}, + table::PyTableProvider, TokioRuntime, }; use datafusion::{ common::ScalarValue, datasource::TableProvider, execution::context::SessionContext, logical_expr::Volatility, }; -use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::prelude::*; use pyo3::{exceptions::PyValueError, types::PyCapsule}; use std::{ @@ -130,17 +133,35 @@ pub(crate) fn table_provider_from_pycapsule( ) -> PyResult>> { if obj.hasattr("__datafusion_table_provider__")? { let capsule = obj.getattr("__datafusion_table_provider__")?.call0()?; - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - Ok(Some(Arc::new(provider) as Arc)) + let provider = PyTableProvider::from_capsule(capsule)?; + Ok(Some(provider.into_inner())) } else { Ok(None) } } +pub(crate) fn coerce_table_provider( + obj: &Bound, +) -> PyDataFusionResult> { + if let Ok(py_table) = obj.extract::() { + Ok(py_table.table()) + } else if let Ok(py_provider) = obj.extract::() { + Ok(py_provider.into_inner()) + } else if obj.is_instance_of::() + || obj + .getattr("df") + .is_ok_and(|inner| inner.is_instance_of::()) + { + Err(PyDataFusionError::Common(EXPECTED_PROVIDER_MSG.to_string())) + } else if let Some(provider) = table_provider_from_pycapsule(obj)? { + Ok(provider) + } else { + let py = obj.py(); + let provider = Dataset::new(obj, py)?; + Ok(Arc::new(provider) as Arc) + } +} + pub(crate) fn py_obj_to_scalar_value(py: Python, obj: PyObject) -> PyResult { // convert Python object to PyScalarValue to ScalarValue From 00bd4451b3c501e30e73f644eab688a058aec935 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 18 Sep 2025 16:43:54 +0800 Subject: [PATCH 05/34] refactor: update documentation for DataFrame to Table Provider conversion --- src/dataframe.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index 6d9f728ed..d3d908ab1 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -397,11 +397,12 @@ impl PyDataFrame { PyArrowType(self.df.schema().into()) } - /// Convert this DataFrame into a Table that can be used in register_table + /// Convert this DataFrame into a Table Provider that can be used in register_table /// By convention, into_... methods consume self and return the new object. - /// Here we intentionally borrow to avoid invalidating the Python wrapper. - /// https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116 - /// - we have not decided on the table_provider approach yet + /// Disabling the clippy lint, so we can use &self + /// because we're working with Python bindings + /// where objects are shared + #[allow(clippy::wrong_self_convention)] pub fn into_view(&self) -> PyDataFusionResult { // Call the underlying Rust DataFrame::into_view method. // Note that the Rust method consumes self; here we clone the inner Arc From 6869919a0faefda36a47371cd7ddbd913f20f44b Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 18 Sep 2025 16:47:55 +0800 Subject: [PATCH 06/34] refactor: replace to_view_provider with inner_df for DataFrame access --- src/dataframe.rs | 8 ++++---- src/table.rs | 6 +++++- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/dataframe.rs b/src/dataframe.rs index d3d908ab1..4f3de9263 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -31,7 +31,6 @@ use datafusion::arrow::util::pretty; use datafusion::common::UnnestOptions; use datafusion::config::{CsvOptions, ParquetColumnOptions, ParquetOptions, TableParquetOptions}; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; -use datafusion::datasource::TableProvider; use datafusion::error::DataFusionError; use datafusion::execution::SendableRecordBatchStream; use datafusion::parquet::basic::{BrotliLevel, Compression, GzipLevel, ZstdLevel}; @@ -268,8 +267,9 @@ impl PyDataFrame { } } - pub(crate) fn to_view_provider(&self) -> Arc { - self.df.as_ref().clone().into_view() + /// Return a clone of the inner Arc for crate-local callers. + pub(crate) fn inner_df(&self) -> Arc { + Arc::clone(&self.df) } fn prepare_repr_string(&mut self, py: Python, as_html: bool) -> PyDataFusionResult { @@ -407,7 +407,7 @@ impl PyDataFrame { // Call the underlying Rust DataFrame::into_view method. // Note that the Rust method consumes self; here we clone the inner Arc // so that we don't invalidate this PyDataFrame. - let table_provider = self.to_view_provider(); + let table_provider = self.df.as_ref().clone().into_view(); Ok(PyTableProvider::new(table_provider)) } diff --git a/src/table.rs b/src/table.rs index 85145e564..1e99b360b 100644 --- a/src/table.rs +++ b/src/table.rs @@ -80,7 +80,11 @@ impl PyTableProvider { /// This method simply delegates to `DataFrame.into_view`. #[staticmethod] pub fn from_dataframe(df: &PyDataFrame) -> Self { - let table_provider = df.to_view_provider(); + // Clone the inner DataFrame and convert it into a view TableProvider. + // `into_view` consumes a DataFrame, so clone the underlying DataFrame + // (this mirrors the previous implementation which used + // `self.df.as_ref().clone().into_view()`). + let table_provider = df.inner_df().as_ref().clone().into_view(); Self::new(table_provider) } From 6e46d43c0446914d83a4ffb82a6456bbe379adb0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 18 Sep 2025 16:53:05 +0800 Subject: [PATCH 07/34] refactor: streamline TableProvider creation from DataFrame by consolidating method calls --- src/table.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/table.rs b/src/table.rs index 1e99b360b..0df5070e2 100644 --- a/src/table.rs +++ b/src/table.rs @@ -82,10 +82,7 @@ impl PyTableProvider { pub fn from_dataframe(df: &PyDataFrame) -> Self { // Clone the inner DataFrame and convert it into a view TableProvider. // `into_view` consumes a DataFrame, so clone the underlying DataFrame - // (this mirrors the previous implementation which used - // `self.df.as_ref().clone().into_view()`). - let table_provider = df.inner_df().as_ref().clone().into_view(); - Self::new(table_provider) + Self::new(df.inner_df().as_ref().clone().into_view()) } /// Create a `TableProvider` from a `DataFrame` by converting it into a view. From 1872a7f9bdab2d924ff41bdbee2b616adea36d15 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 18 Sep 2025 17:27:21 +0800 Subject: [PATCH 08/34] fix ruff errors --- python/datafusion/__init__.py | 17 ++++++++--------- python/datafusion/context.py | 18 ++++++++++++------ python/datafusion/expr.py | 11 ++++++----- python/datafusion/io.py | 2 +- python/datafusion/table_provider.py | 5 +---- python/datafusion/utils.py | 1 - 6 files changed, 28 insertions(+), 26 deletions(-) diff --git a/python/datafusion/__init__.py b/python/datafusion/__init__.py index b622e6925..7cffd2113 100644 --- a/python/datafusion/__init__.py +++ b/python/datafusion/__init__.py @@ -21,6 +21,9 @@ See https://datafusion.apache.org/python for more information. """ +# isort: skip_file # Prevent import-sorting linter errors (I001) +# ruff: noqa: I001 + from __future__ import annotations from typing import Any @@ -28,17 +31,16 @@ try: import importlib.metadata as importlib_metadata except ImportError: - import importlib_metadata + import importlib_metadata # type: ignore[import] +# Public submodules from . import functions, object_store, substrait, unparser # The following imports are okay to remain as opaque to the user. from ._internal import Config, EXPECTED_PROVIDER_MSG from .catalog import Catalog, Database, Table from .col import col, column -from .common import ( - DFSchema, -) +from .common import DFSchema from .context import ( RuntimeEnvBuilder, SessionConfig, @@ -47,10 +49,7 @@ ) from .dataframe import DataFrame, ParquetColumnOptions, ParquetWriterOptions from .dataframe_formatter import configure_formatter -from .expr import ( - Expr, - WindowFrame, -) +from .expr import Expr, WindowFrame from .io import read_avro, read_csv, read_json, read_parquet from .plan import ExecutionPlan, LogicalPlan from .record_batch import RecordBatch, RecordBatchStream @@ -70,6 +69,7 @@ __version__ = importlib_metadata.version(__name__) __all__ = [ + "EXPECTED_PROVIDER_MSG", "Accumulator", "AggregateUDF", "Catalog", @@ -77,7 +77,6 @@ "DFSchema", "DataFrame", "Database", - "EXPECTED_PROVIDER_MSG", "ExecutionPlan", "Expr", "LogicalPlan", diff --git a/python/datafusion/context.py b/python/datafusion/context.py index bc7284bd1..480bb24e7 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -29,11 +29,10 @@ import pyarrow as pa -from datafusion.catalog import Catalog, CatalogProvider, Table +from datafusion.catalog import Catalog from datafusion.dataframe import DataFrame -from datafusion.expr import SortKey, sort_list_to_raw_sort_list +from datafusion.expr import sort_list_to_raw_sort_list from datafusion.record_batch import RecordBatchStream -from datafusion.user_defined import AggregateUDF, ScalarUDF, TableFunction, WindowUDF from datafusion.utils import _normalize_table_provider from ._internal import RuntimeEnvBuilder as RuntimeEnvBuilderInternal @@ -50,7 +49,15 @@ import polars as pl # type: ignore[import] from datafusion import TableProvider + from datafusion.catalog import CatalogProvider, Table + from datafusion.expr import SortKey from datafusion.plan import ExecutionPlan, LogicalPlan + from datafusion.user_defined import ( + AggregateUDF, + ScalarUDF, + TableFunction, + WindowUDF, + ) class ArrowStreamExportable(Protocol): @@ -735,7 +742,7 @@ def from_polars(self, data: pl.DataFrame, name: str | None = None) -> DataFrame: # 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: - """Register a :py:class: `~datafusion.detaframe.DataFrame` as a view. + """Register a :py:class:`~datafusion.dataframe.DataFrame` as a view. Args: name (str): The name to register the view under. @@ -747,8 +754,7 @@ def register_view(self, name: str, df: DataFrame) -> None: def register_table( self, name: str, table: Table | TableProvider | TableProviderExportable ) -> None: - """Register a :py:class:`~datafusion.catalog.Table` or - :py:class:`~datafusion.TableProvider`. + """Register a Table or TableProvider. The registered table can be referenced from SQL statements executed against this context. diff --git a/python/datafusion/expr.py b/python/datafusion/expr.py index 5d1180bd1..82e30a78c 100644 --- a/python/datafusion/expr.py +++ b/python/datafusion/expr.py @@ -25,14 +25,12 @@ import typing as _typing from typing import TYPE_CHECKING, Any, ClassVar, Iterable, Optional, Sequence -import pyarrow as pa - try: from warnings import deprecated # Python 3.13+ except ImportError: from typing_extensions import deprecated # Python 3.12 -from datafusion.common import NullTreatment +import pyarrow as pa from ._internal import expr as expr_internal from ._internal import functions as functions_internal @@ -40,8 +38,11 @@ if TYPE_CHECKING: from collections.abc import Sequence - # Type-only imports - from datafusion.common import DataTypeMap, RexType + from datafusion.common import ( # type: ignore[import] + DataTypeMap, + NullTreatment, + RexType, + ) from datafusion.plan import LogicalPlan diff --git a/python/datafusion/io.py b/python/datafusion/io.py index 551e20a6f..67dbc730f 100644 --- a/python/datafusion/io.py +++ b/python/datafusion/io.py @@ -22,13 +22,13 @@ from typing import TYPE_CHECKING from datafusion.context import SessionContext -from datafusion.dataframe import DataFrame if TYPE_CHECKING: import pathlib import pyarrow as pa + from datafusion.dataframe import DataFrame from datafusion.expr import Expr diff --git a/python/datafusion/table_provider.py b/python/datafusion/table_provider.py index e379e2c5d..617e394df 100644 --- a/python/datafusion/table_provider.py +++ b/python/datafusion/table_provider.py @@ -67,10 +67,7 @@ def from_dataframe(cls, df: Any) -> TableProvider: """ from datafusion.dataframe import DataFrame as DataFrameWrapper - if isinstance(df, DataFrameWrapper): - dataframe = df - else: - dataframe = DataFrameWrapper(df) + dataframe = df if isinstance(df, DataFrameWrapper) else DataFrameWrapper(df) return dataframe.into_view() diff --git a/python/datafusion/utils.py b/python/datafusion/utils.py index 59664236d..bae2783d5 100644 --- a/python/datafusion/utils.py +++ b/python/datafusion/utils.py @@ -60,7 +60,6 @@ def _normalize_table_provider( Raises: TypeError: If ``table`` is not a supported table provider input. """ - from datafusion.catalog import Table as _Table from datafusion.table_provider import TableProvider as _TableProvider From 5948fb4dca471bc0300458283b45ba82b07f8bd7 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 18 Sep 2025 18:37:06 +0800 Subject: [PATCH 09/34] refactor: enhance autoapi_skip_member_fn to skip private variables and avoid documentation duplication --- docs/source/conf.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/source/conf.py b/docs/source/conf.py index 28db17d35..b3b0c848e 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -83,6 +83,9 @@ def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa # Duplicate modules (skip module-level docs to avoid duplication) ("module", "datafusion.col"), ("module", "datafusion.udf"), + # Private variables causing duplicate documentation + ("data", "datafusion.utils._PYARROW_DATASET_TYPES"), + ("variable", "datafusion.utils._PYARROW_DATASET_TYPES"), # Deprecated ("class", "datafusion.substrait.serde"), ("class", "datafusion.substrait.plan"), @@ -94,6 +97,10 @@ def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa if (what, name) in skip_contents: skip = True + # Skip private members that start with underscore to avoid duplication + if name.split(".")[-1].startswith("_") and what in ("data", "variable"): + skip = True + return skip From b9851d82a4f54a32d2943a41ecf631cfd401f2b0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 09:55:06 +0800 Subject: [PATCH 10/34] revert main 49.0.0 md --- dev/changelog/49.0.0.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dev/changelog/49.0.0.md b/dev/changelog/49.0.0.md index 15b3c2382..008bd43bc 100644 --- a/dev/changelog/49.0.0.md +++ b/dev/changelog/49.0.0.md @@ -25,10 +25,6 @@ This release consists of 16 commits from 7 contributors. See credits at the end - fix(build): Include build.rs in published crates [#1199](https://github.com/apache/datafusion-python/pull/1199) (colinmarc) -**Deprecations:** - -- Document that `SessionContext.register_table_provider` is deprecated in favor of `SessionContext.register_table`. - **Other:** - 48.0.0 Release [#1175](https://github.com/apache/datafusion-python/pull/1175) (timsaucer) From 586c2cfe73cf33815b0f4a04d612b64a51b640eb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 10:05:16 +0800 Subject: [PATCH 11/34] refactor: add comment in autoapi_skip_member_fn --- docs/source/conf.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/docs/source/conf.py b/docs/source/conf.py index b3b0c848e..18d5f1232 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -94,10 +94,25 @@ def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa ("method", "datafusion.context.SessionContext.tables"), ("method", "datafusion.dataframe.DataFrame.unnest_column"), ] + # Explicitly skip certain members listed above. These are either + # re-exports, duplicate module-level documentation, deprecated + # API surfaces, or private variables that would otherwise appear + # in the generated docs and cause confusing duplication. + # Keeping this explicit list avoids surprising entries in the + # AutoAPI output and gives us a single place to opt-out items + # when we intentionally hide them from the docs. if (what, name) in skip_contents: skip = True - # Skip private members that start with underscore to avoid duplication + # Skip private module-level names (those whose final component + # starts with an underscore) when AutoAPI is rendering data or + # variable entries. Many internal module-level constants are + # implementation details (for example private pyarrow dataset type + # mappings) that would otherwise be emitted as top-level "data" + # or "variable" docs. Filtering them here avoids noisy, + # duplicate, or implementation-specific entries in the public + # documentation while still allowing public members and types to + # be documented normally. if name.split(".")[-1].startswith("_") and what in ("data", "variable"): skip = True From d4ff136ca3c1424fe32fecd8282abe026e466394 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 10:14:07 +0800 Subject: [PATCH 12/34] refactor: remove isort and ruff comments to clean up import section --- python/datafusion/__init__.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/python/datafusion/__init__.py b/python/datafusion/__init__.py index 7cffd2113..8133cb5c2 100644 --- a/python/datafusion/__init__.py +++ b/python/datafusion/__init__.py @@ -21,9 +21,6 @@ See https://datafusion.apache.org/python for more information. """ -# isort: skip_file # Prevent import-sorting linter errors (I001) -# ruff: noqa: I001 - from __future__ import annotations from typing import Any @@ -37,7 +34,7 @@ from . import functions, object_store, substrait, unparser # The following imports are okay to remain as opaque to the user. -from ._internal import Config, EXPECTED_PROVIDER_MSG +from ._internal import EXPECTED_PROVIDER_MSG, Config from .catalog import Catalog, Database, Table from .col import col, column from .common import DFSchema From 29203c659d0e4fb84e2dbfe53a9ee516164c2753 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 10:23:38 +0800 Subject: [PATCH 13/34] docs: enhance docstring for DataFrame.into_view method to clarify usage and advantages --- python/datafusion/dataframe.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 07d899b00..8969dbb26 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -319,7 +319,17 @@ def into_view(self) -> TableProvider: """Convert ``DataFrame`` into a ``TableProvider`` view for registration. This is the preferred way to obtain a view for - :py:meth:`~datafusion.context.SessionContext.register_table`. + :py:meth:`~datafusion.context.SessionContext.register_table` for several reasons: + + 1. **Direct API**: Most efficient path - directly calls the underlying Rust + ``DataFrame.into_view()`` method without intermediate delegations. + 2. **Clear semantics**: The ``into_`` prefix follows Rust conventions, + indicating conversion from one type to another. + 3. **Canonical method**: Other approaches like ``TableProvider.from_dataframe`` + delegate to this method internally, making this the single source of truth. + 4. **Deprecated alternatives**: The older ``TableProvider.from_view`` helper + is deprecated and issues warnings when used. + ``datafusion.TableProvider.from_dataframe`` calls this method under the hood, and the older ``TableProvider.from_view`` helper is deprecated. From ae8c1dd844a0ea314d9f822436d2d920af170d2d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 10:26:12 +0800 Subject: [PATCH 14/34] docs: update example in DataFrame.into_view docstring for clarity --- 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 8969dbb26..4cf8b0c1e 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -340,8 +340,8 @@ def into_view(self) -> TableProvider: >>> from datafusion import SessionContext >>> ctx = SessionContext() >>> df = ctx.sql("SELECT 1 AS value") - >>> provider = df.into_view() - >>> ctx.register_table("values_view", provider) + >>> view = df.into_view() + >>> ctx.register_table("values_view", view) >>> df.collect() # The DataFrame is still usable >>> ctx.sql("SELECT value FROM values_view").collect() """ From 0c5eb17fc4f1097e01fcdc6d7f4403a1e8e129bc Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 11:19:02 +0800 Subject: [PATCH 15/34] docs: update example for registering Delta Lake tables to simplify usage --- docs/source/user-guide/data-sources.rst | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index a3a5ec0a6..5c5f02da4 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -152,14 +152,16 @@ as Delta Lake. This will require a recent version of .. code-block:: python from deltalake import DeltaTable - from datafusion import TableProvider delta_table = DeltaTable("path_to_table") - provider = TableProvider.from_capsule(delta_table.__datafusion_table_provider__()) - ctx.register_table("my_delta_table", provider) + ctx.register_table("my_delta_table", delta_table) df = ctx.table("my_delta_table") df.show() +Objects that implement ``__datafusion_table_provider__`` are supported directly by +:py:meth:`~datafusion.context.SessionContext.register_table`, making it easy to +work with custom table providers from Python libraries such as Delta Lake. + .. note:: :py:meth:`~datafusion.context.SessionContext.register_table_provider` is From f9a3a2297fce99945603963d61b31fe7b655bbeb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 11:19:30 +0800 Subject: [PATCH 16/34] docs: update table provider documentation for clarity and deprecate old methods --- docs/source/user-guide/io/table_provider.rst | 41 +++++++++++--------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/docs/source/user-guide/io/table_provider.rst b/docs/source/user-guide/io/table_provider.rst index 0e7746cca..ab08f42e2 100644 --- a/docs/source/user-guide/io/table_provider.rst +++ b/docs/source/user-guide/io/table_provider.rst @@ -46,40 +46,45 @@ A complete example can be found in the `examples folder Date: Mon, 22 Sep 2025 12:26:18 +0800 Subject: [PATCH 17/34] docs: update documentation to reflect removal of TableProvider and usage of Table instead --- docs/source/user-guide/data-sources.rst | 6 +- docs/source/user-guide/io/table_provider.rst | 43 ++++--- python/datafusion/__init__.py | 2 - python/datafusion/catalog.py | 113 +++++++++++++++--- python/datafusion/context.py | 18 ++- python/datafusion/dataframe.py | 18 +-- python/datafusion/table_provider.py | 114 ------------------- python/datafusion/utils.py | 13 +-- python/tests/test_context.py | 81 ++++--------- src/table.rs | 4 +- src/utils.rs | 2 +- 11 files changed, 163 insertions(+), 251 deletions(-) delete mode 100644 python/datafusion/table_provider.py diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index 5c5f02da4..bedbabffb 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -152,9 +152,11 @@ as Delta Lake. This will require a recent version of .. code-block:: python from deltalake import DeltaTable + from datafusion import Table delta_table = DeltaTable("path_to_table") - ctx.register_table("my_delta_table", delta_table) + table = Table.from_capsule(delta_table.__datafusion_table_provider__()) + ctx.register_table("my_delta_table", table) df = ctx.table("my_delta_table") df.show() @@ -167,7 +169,7 @@ work with custom table providers from Python libraries such as Delta Lake. :py:meth:`~datafusion.context.SessionContext.register_table_provider` is deprecated. Use :py:meth:`~datafusion.context.SessionContext.register_table` with a - :py:class:`~datafusion.TableProvider` instead. + :py:class:`~datafusion.Table` instead. On older versions of ``deltalake`` (prior to 0.22) you can use the `Arrow DataSet `_ diff --git a/docs/source/user-guide/io/table_provider.rst b/docs/source/user-guide/io/table_provider.rst index ab08f42e2..0dfc07c3b 100644 --- a/docs/source/user-guide/io/table_provider.rst +++ b/docs/source/user-guide/io/table_provider.rst @@ -46,45 +46,40 @@ A complete example can be found in the `examples folder Table: return Table(self._raw_schema.table(name)) def register_table( - self, name: str, table: Table | TableProvider | TableProviderExportable + self, name: str, table: Table | TableProviderExportable | Any ) -> None: """Register a table or table provider in this schema. Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as :class:`TableProvider` instances. + and treated as table provider instances. """ provider = _normalize_table_provider(table) return self._raw_schema.register_table(name, provider) @@ -151,31 +153,108 @@ class Database(Schema): """See `Schema`.""" +_InternalRawTable = df_internal.catalog.RawTable +_InternalTableProvider = df_internal.TableProvider + +# Keep in sync with ``datafusion._internal.TableProvider.from_view``. +_FROM_VIEW_WARN_STACKLEVEL = 2 + + class Table: - """DataFusion table.""" + """DataFusion table or table provider wrapper.""" - def __init__(self, table: df_internal.catalog.RawTable) -> None: - """This constructor is not typically called by the end user.""" - self.table = table + __slots__ = ("_table",) + + def __init__( + self, + table: _InternalRawTable | _InternalTableProvider | Table, + ) -> None: + """Wrap a low level table or table provider.""" + + if isinstance(table, Table): + table = table.table + + if not isinstance(table, (_InternalRawTable, _InternalTableProvider)): + raise TypeError(EXPECTED_PROVIDER_MSG) + + self._table = table + + def __getattribute__(self, name: str) -> Any: + """Restrict provider-specific helpers to compatible tables.""" + + if name == "__datafusion_table_provider__": + table = object.__getattribute__(self, "_table") + if not hasattr(table, "__datafusion_table_provider__"): + raise AttributeError(name) + return object.__getattribute__(self, name) def __repr__(self) -> str: """Print a string representation of the table.""" - return self.table.__repr__() + return repr(self._table) - @staticmethod - def from_dataset(dataset: pa.dataset.Dataset) -> Table: - """Turn a pyarrow Dataset into a Table.""" - return Table(df_internal.catalog.RawTable.from_dataset(dataset)) + @property + def table(self) -> _InternalRawTable | _InternalTableProvider: + """Return the wrapped low level table object.""" + return self._table + + @classmethod + def from_dataset(cls, dataset: pa.dataset.Dataset) -> Table: + """Turn a :mod:`pyarrow.dataset` ``Dataset`` into a :class:`Table`.""" + + return cls(_InternalRawTable.from_dataset(dataset)) + + @classmethod + def from_capsule(cls, capsule: Any) -> Table: + """Create a :class:`Table` from a PyCapsule exported provider.""" + + provider = _InternalTableProvider.from_capsule(capsule) + return cls(provider) + + @classmethod + def from_dataframe(cls, df: Any) -> Table: + """Create a :class:`Table` from tabular data.""" + + from datafusion.dataframe import DataFrame as DataFrameWrapper + + dataframe = df if isinstance(df, DataFrameWrapper) else DataFrameWrapper(df) + return dataframe.into_view() + + @classmethod + def from_view(cls, df: Any) -> Table: + """Deprecated helper for constructing tables from views.""" + + from datafusion.dataframe import DataFrame as DataFrameWrapper + + if isinstance(df, DataFrameWrapper): + df = df.df + + provider = _InternalTableProvider.from_view(df) + warnings.warn( + "Table.from_view is deprecated; use DataFrame.into_view or " + "Table.from_dataframe instead.", + category=DeprecationWarning, + stacklevel=_FROM_VIEW_WARN_STACKLEVEL, + ) + return cls(provider) @property def schema(self) -> pa.Schema: """Returns the schema associated with this table.""" - return self.table.schema + return self._table.schema @property def kind(self) -> str: """Returns the kind of table.""" - return self.table.kind + return self._table.kind + + def __datafusion_table_provider__(self) -> Any: + """Expose the wrapped provider for FFI integrations.""" + + exporter = getattr(self._table, "__datafusion_table_provider__", None) + if exporter is None: + msg = "Underlying object does not export __datafusion_table_provider__()" + raise AttributeError(msg) + return exporter() class CatalogProvider(ABC): @@ -233,7 +312,7 @@ def table(self, name: str) -> Table | None: ... def register_table( # noqa: B027 - self, name: str, table: Table | TableProvider | TableProviderExportable + self, name: str, table: Table | TableProviderExportable | Any ) -> None: """Add a table to this schema. @@ -241,7 +320,7 @@ def register_table( # noqa: B027 not need to implement this method. Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as :class:`TableProvider` instances. + and treated as table provider instances. """ def deregister_table(self, name: str, cascade: bool) -> None: # noqa: B027 diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 480bb24e7..4f1c18663 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -48,7 +48,6 @@ import pandas as pd import polars as pl # type: ignore[import] - from datafusion import TableProvider from datafusion.catalog import CatalogProvider, Table from datafusion.expr import SortKey from datafusion.plan import ExecutionPlan, LogicalPlan @@ -752,25 +751,24 @@ def register_view(self, name: str, df: DataFrame) -> None: self.ctx.register_table(name, view) def register_table( - self, name: str, table: Table | TableProvider | TableProviderExportable + self, name: str, table: Table | TableProviderExportable | Any ) -> None: - """Register a Table or TableProvider. + """Register a :py:class:`~datafusion.Table` with this context. The registered table can be referenced from SQL statements executed against this context. Plain :py:class:`~datafusion.dataframe.DataFrame` objects are not supported; convert them first with :meth:`datafusion.dataframe.DataFrame.into_view` or - :meth:`datafusion.TableProvider.from_dataframe`. + :meth:`datafusion.Table.from_dataframe`. Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as :py:class:`~datafusion.TableProvider` instances. + and treated as table provider instances. Args: name: Name of the resultant table. - table: DataFusion :class:`Table`, :class:`TableProvider`, or any object - implementing ``__datafusion_table_provider__`` to add to the session - context. + table: DataFusion :class:`Table` or any object implementing + ``__datafusion_table_provider__`` to add to the session context. """ provider = _normalize_table_provider(table) self.ctx.register_table(name, provider) @@ -793,14 +791,14 @@ def register_catalog_provider( self.ctx.register_catalog_provider(name, provider) def register_table_provider( - self, name: str, provider: Table | TableProvider | TableProviderExportable + self, name: str, provider: Table | TableProviderExportable | Any ) -> None: """Register a table provider. Deprecated: use :meth:`register_table` instead. Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as :py:class:`~datafusion.TableProvider` instances. + and treated as table provider instances. """ warnings.warn( "register_table_provider is deprecated; use register_table", diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 4cf8b0c1e..4abe108d0 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -60,7 +60,7 @@ import polars as pl import pyarrow as pa - from datafusion.table_provider import TableProvider + from datafusion.catalog import Table from enum import Enum @@ -315,8 +315,8 @@ def __init__(self, df: DataFrameInternal) -> None: """ self.df = df - def into_view(self) -> TableProvider: - """Convert ``DataFrame`` into a ``TableProvider`` view for registration. + def into_view(self) -> Table: + """Convert ``DataFrame`` into a :class:`~datafusion.Table` for registration. This is the preferred way to obtain a view for :py:meth:`~datafusion.context.SessionContext.register_table` for several reasons: @@ -325,13 +325,13 @@ def into_view(self) -> TableProvider: ``DataFrame.into_view()`` method without intermediate delegations. 2. **Clear semantics**: The ``into_`` prefix follows Rust conventions, indicating conversion from one type to another. - 3. **Canonical method**: Other approaches like ``TableProvider.from_dataframe`` + 3. **Canonical method**: Other approaches like ``Table.from_dataframe`` delegate to this method internally, making this the single source of truth. - 4. **Deprecated alternatives**: The older ``TableProvider.from_view`` helper + 4. **Deprecated alternatives**: The older ``Table.from_view`` helper is deprecated and issues warnings when used. - ``datafusion.TableProvider.from_dataframe`` calls this method under the hood, - and the older ``TableProvider.from_view`` helper is deprecated. + ``datafusion.Table.from_dataframe`` calls this method under the hood, and the + older ``Table.from_view`` helper is deprecated. The ``DataFrame`` remains valid after conversion, so it can still be used for additional queries alongside the returned view. @@ -345,9 +345,9 @@ def into_view(self) -> TableProvider: >>> df.collect() # The DataFrame is still usable >>> ctx.sql("SELECT value FROM values_view").collect() """ - from datafusion.table_provider import TableProvider as _TableProvider + from datafusion.catalog import Table as _Table - return _TableProvider(self.df.into_view()) + return _Table(self.df.into_view()) def __getitem__(self, key: str | list[str]) -> DataFrame: """Return a new :py:class`DataFrame` with the specified column or columns. diff --git a/python/datafusion/table_provider.py b/python/datafusion/table_provider.py deleted file mode 100644 index 617e394df..000000000 --- a/python/datafusion/table_provider.py +++ /dev/null @@ -1,114 +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. -"""Wrapper helpers for :mod:`datafusion._internal.TableProvider`.""" - -from __future__ import annotations - -import warnings -from typing import Any - -import datafusion._internal as df_internal -from datafusion._internal import EXPECTED_PROVIDER_MSG - -_InternalTableProvider = df_internal.TableProvider - -# Keep in sync with ``datafusion._internal.TableProvider.from_view``. -_FROM_VIEW_WARN_STACKLEVEL = 2 - - -class TableProvider: - """High level wrapper around :mod:`datafusion._internal.TableProvider`.""" - - __slots__ = ("_table_provider",) - - def __init__(self, table_provider: _InternalTableProvider) -> None: - """Wrap a low level :class:`~datafusion._internal.TableProvider`.""" - if isinstance(table_provider, TableProvider): - table_provider = table_provider._table_provider - - if not isinstance(table_provider, _InternalTableProvider): - raise TypeError(EXPECTED_PROVIDER_MSG) - - self._table_provider = table_provider - - @classmethod - def from_capsule(cls, capsule: Any) -> TableProvider: - """Create a :class:`TableProvider` from a PyCapsule.""" - provider = _InternalTableProvider.from_capsule(capsule) - return cls(provider) - - @classmethod - def from_dataframe(cls, df: Any) -> TableProvider: - """Create a :class:`TableProvider` from tabular data. - - Parameters - ---------- - df: - Either a :class:`~datafusion.dataframe.DataFrame` wrapper or the - corresponding :class:`~datafusion._internal.DataFrame`. When - working with third-party DataFrame libraries, convert them via - :meth:`~datafusion.SessionContext.from_arrow` before calling - :meth:`~datafusion.dataframe.DataFrame.into_view` or this - constructor. - """ - from datafusion.dataframe import DataFrame as DataFrameWrapper - - dataframe = df if isinstance(df, DataFrameWrapper) else DataFrameWrapper(df) - - return dataframe.into_view() - - @classmethod - def from_view(cls, df: Any) -> TableProvider: - """Deprecated. - - Use :meth:`DataFrame.into_view` or :meth:`TableProvider.from_dataframe`. - """ - from datafusion.dataframe import DataFrame as DataFrameWrapper - - if isinstance(df, DataFrameWrapper): - df = df.df - - provider = _InternalTableProvider.from_view(df) - warnings.warn( - "TableProvider.from_view is deprecated; use DataFrame.into_view or " - "TableProvider.from_dataframe instead.", - category=DeprecationWarning, - stacklevel=_FROM_VIEW_WARN_STACKLEVEL, - ) - return cls(provider) - - # ------------------------------------------------------------------ - # passthrough helpers - # ------------------------------------------------------------------ - def __getattr__(self, name: str) -> Any: - """Delegate attribute lookup to the wrapped provider.""" - return getattr(self._table_provider, name) - - def __dir__(self) -> list[str]: - """Expose delegated attributes via :func:`dir`.""" - return sorted(set(super().__dir__()) | set(dir(self._table_provider))) - - def __repr__(self) -> str: # pragma: no cover - simple delegation - """Return a representation of the wrapped provider.""" - return repr(self._table_provider) - - def __datafusion_table_provider__(self) -> Any: - """Expose the wrapped provider for FFI integrations.""" - return self._table_provider.__datafusion_table_provider__() - - -__all__ = ["TableProvider"] diff --git a/python/datafusion/utils.py b/python/datafusion/utils.py index bae2783d5..eb3e3d626 100644 --- a/python/datafusion/utils.py +++ b/python/datafusion/utils.py @@ -39,20 +39,19 @@ _PYARROW_DATASET_TYPES = tuple(dataset_types) if TYPE_CHECKING: # pragma: no cover - imported for typing only - from datafusion import TableProvider from datafusion.catalog import Table from datafusion.context import TableProviderExportable def _normalize_table_provider( - table: Table | TableProvider | TableProviderExportable, + table: Table | TableProviderExportable | Any, ) -> Any: """Return the underlying provider for supported table inputs. Args: - table: A :class:`~datafusion.catalog.Table`, - :class:`~datafusion.table_provider.TableProvider`, or object exporting a - DataFusion table provider via ``__datafusion_table_provider__``. + table: A :class:`~datafusion.Table`, object exporting a DataFusion table + provider via ``__datafusion_table_provider__``, or compatible + :mod:`pyarrow.dataset` implementation. Returns: The object expected by the Rust bindings for table registration. @@ -61,14 +60,10 @@ def _normalize_table_provider( TypeError: If ``table`` is not a supported table provider input. """ from datafusion.catalog import Table as _Table - from datafusion.table_provider import TableProvider as _TableProvider if isinstance(table, _Table): return table.table - if isinstance(table, _TableProvider): - return table._table_provider - if _PYARROW_DATASET_TYPES and isinstance(table, _PYARROW_DATASET_TYPES): return table diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 3e4d84302..694ae0c18 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -16,9 +16,7 @@ # under the License. import datetime as dt import gzip -import inspect import pathlib -import warnings import pyarrow as pa import pyarrow.dataset as ds @@ -30,7 +28,7 @@ SessionConfig, SessionContext, SQLOptions, - TableProvider, + Table, column, literal, ) @@ -336,80 +334,41 @@ def test_deregister_table(ctx, database): def test_register_table_from_dataframe_into_view(ctx): df = ctx.from_pydict({"a": [1, 2]}) - provider = df.into_view() - ctx.register_table("view_tbl", provider) + table = df.into_view() + assert isinstance(table, Table) + ctx.register_table("view_tbl", table) result = ctx.sql("SELECT * FROM view_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] -def test_table_provider_from_capsule(ctx): +def test_table_from_capsule(ctx): df = ctx.from_pydict({"a": [1, 2]}) - provider = df.into_view() - capsule = provider.__datafusion_table_provider__() - provider2 = TableProvider.from_capsule(capsule) - ctx.register_table("capsule_tbl", provider2) + table = df.into_view() + capsule = table.__datafusion_table_provider__() + table2 = Table.from_capsule(capsule) + assert isinstance(table2, Table) + ctx.register_table("capsule_tbl", table2) result = ctx.sql("SELECT * FROM capsule_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] -def test_table_provider_from_dataframe(ctx): +def test_table_from_dataframe(ctx): df = ctx.from_pydict({"a": [1, 2]}) - provider = TableProvider.from_dataframe(df) - assert isinstance(provider, TableProvider) - ctx.register_table("from_dataframe_tbl", provider) + table = Table.from_dataframe(df) + assert isinstance(table, Table) + ctx.register_table("from_dataframe_tbl", table) result = ctx.sql("SELECT * FROM from_dataframe_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] -def test_table_provider_from_dataframe_internal(ctx): +def test_table_from_dataframe_internal(ctx): df = ctx.from_pydict({"a": [1, 2]}) - provider = TableProvider.from_dataframe(df.df) - assert isinstance(provider, TableProvider) - ctx.register_table("from_internal_dataframe_tbl", provider) + table = Table.from_dataframe(df.df) + assert isinstance(table, Table) + ctx.register_table("from_internal_dataframe_tbl", table) result = ctx.sql("SELECT * FROM from_internal_dataframe_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] - -def test_table_provider_from_view_warning_origin(ctx): - from datafusion.table_provider import TableProvider as WrapperTableProvider - - wrapper_df = ctx.from_pydict({"a": [1]}) - test_path = pathlib.Path(__file__).resolve() - - with warnings.catch_warnings(record=True) as caught: - warnings.simplefilter("always") - call_lineno = inspect.currentframe().f_lineno + 1 - WrapperTableProvider.from_view(wrapper_df) - - assert len(caught) >= 1 - - rust_warning = next( - ( - warning - for warning in caught - if "PyTableProvider.from_view()" in str(warning.message) - ), - None, - ) - assert rust_warning is not None - assert issubclass(rust_warning.category, DeprecationWarning) - assert pathlib.Path(rust_warning.filename).resolve() == test_path - assert rust_warning.lineno == call_lineno - - py_warning = next( - ( - warning - for warning in caught - if "TableProvider.from_view is deprecated" in str(warning.message) - ), - None, - ) - assert py_warning is not None - assert issubclass(py_warning.category, DeprecationWarning) - assert pathlib.Path(py_warning.filename).resolve() == test_path - assert py_warning.lineno == call_lineno - - def test_register_table_capsule_direct(ctx): df = ctx.from_pydict({"a": [1, 2]}) provider = df.into_view() @@ -426,9 +385,9 @@ def __datafusion_table_provider__(self): assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] -def test_table_provider_from_capsule_invalid(): +def test_table_from_capsule_invalid(): with pytest.raises(RuntimeError): - TableProvider.from_capsule(object()) + Table.from_capsule(object()) def test_register_table_with_dataframe_errors(ctx): diff --git a/src/table.rs b/src/table.rs index 0df5070e2..29476e473 100644 --- a/src/table.rs +++ b/src/table.rs @@ -88,7 +88,7 @@ impl PyTableProvider { /// Create a `TableProvider` from a `DataFrame` by converting it into a view. /// /// Deprecated: prefer `DataFrame.into_view` or - /// `TableProvider.from_dataframe` instead. + /// `Table.from_dataframe` instead. #[staticmethod] pub fn from_view(py: Python<'_>, df: &PyDataFrame) -> PyDataFusionResult { let kwargs = PyDict::new(py); @@ -97,7 +97,7 @@ impl PyTableProvider { py.import("warnings")?.call_method( "warn", ( - "PyTableProvider.from_view() is deprecated; use DataFrame.into_view() or TableProvider.from_dataframe() instead.", + "PyTableProvider.from_view() is deprecated; use DataFrame.into_view() or Table.from_dataframe() instead.", py.get_type::(), ), Some(&kwargs), diff --git a/src/utils.rs b/src/utils.rs index 04f569146..6d5755f88 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -38,7 +38,7 @@ use std::{ use tokio::{runtime::Runtime, time::sleep}; pub(crate) const EXPECTED_PROVIDER_MSG: &str = - "Expected a Table or TableProvider. Convert DataFrames with \"DataFrame.into_view()\" or \"TableProvider.from_dataframe()\"."; + "Expected a Table. Convert DataFrames with \"DataFrame.into_view()\" or \"Table.from_dataframe()\"."; /// Utility to get the Tokio Runtime from Python #[inline] pub(crate) fn get_tokio_runtime() -> &'static TokioRuntime { From afc9b4ec38b651091214b3ef389d8f0117819d94 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 16:15:36 +0800 Subject: [PATCH 18/34] remove TableProvider in Python, update missing_exports function, doc --- python/tests/test_wrapper_coverage.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/python/tests/test_wrapper_coverage.py b/python/tests/test_wrapper_coverage.py index f484cb282..53a987423 100644 --- a/python/tests/test_wrapper_coverage.py +++ b/python/tests/test_wrapper_coverage.py @@ -21,6 +21,9 @@ import datafusion.substrait import pytest + +IGNORED_EXPORTS = {"TableProvider"} + # EnumType introduced in 3.11. 3.10 and prior it was called EnumMeta. try: from enum import EnumType @@ -36,6 +39,7 @@ def missing_exports(internal_obj, wrapped_obj) -> None: # noqa: C901 - Raw* classes: Internal implementation details that shouldn't be exposed - _global_ctx: Internal implementation detail - __self__, __class__, __repr__: Python special attributes + - TableProvider: Superseded by the public ``Table`` API in Python """ # Special case enums - EnumType overrides a some of the internal functions, # so check all of the values exist and move on @@ -50,6 +54,10 @@ def missing_exports(internal_obj, wrapped_obj) -> None: # noqa: C901 for internal_attr_name in dir(internal_obj): wrapped_attr_name = internal_attr_name.removeprefix("Raw") + + if wrapped_attr_name in IGNORED_EXPORTS: + continue + assert wrapped_attr_name in dir(wrapped_obj) internal_attr = getattr(internal_obj, internal_attr_name) @@ -71,6 +79,8 @@ def missing_exports(internal_obj, wrapped_obj) -> None: # noqa: C901 # We have cases like __all__ that are a list and we want to be certain that # every value in the list in the internal object is also in the wrapper list for val in internal_attr: + if isinstance(val, str) and val in IGNORED_EXPORTS: + continue if isinstance(val, str) and val.startswith("Raw"): assert val[3:] in wrapped_attr else: From 918b1cec316f4d73728ac95d9443aa06d34b4bf0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 18:14:13 +0800 Subject: [PATCH 19/34] Fix Ruff errors --- python/datafusion/catalog.py | 10 +------ python/datafusion/dataframe.py | 7 +++-- python/tests/test_context.py | 1 + python/tests/test_wrapper_coverage.py | 41 ++++++++++++++++----------- 4 files changed, 31 insertions(+), 28 deletions(-) diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index b2ac37746..bd3300dab 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -19,11 +19,10 @@ from __future__ import annotations +import warnings from abc import ABC, abstractmethod from typing import TYPE_CHECKING, Any, Protocol -import warnings - import datafusion._internal as df_internal from datafusion._internal import EXPECTED_PROVIDER_MSG from datafusion.utils import _normalize_table_provider @@ -170,7 +169,6 @@ def __init__( table: _InternalRawTable | _InternalTableProvider | Table, ) -> None: """Wrap a low level table or table provider.""" - if isinstance(table, Table): table = table.table @@ -181,7 +179,6 @@ def __init__( def __getattribute__(self, name: str) -> Any: """Restrict provider-specific helpers to compatible tables.""" - if name == "__datafusion_table_provider__": table = object.__getattribute__(self, "_table") if not hasattr(table, "__datafusion_table_provider__"): @@ -200,20 +197,17 @@ def table(self) -> _InternalRawTable | _InternalTableProvider: @classmethod def from_dataset(cls, dataset: pa.dataset.Dataset) -> Table: """Turn a :mod:`pyarrow.dataset` ``Dataset`` into a :class:`Table`.""" - return cls(_InternalRawTable.from_dataset(dataset)) @classmethod def from_capsule(cls, capsule: Any) -> Table: """Create a :class:`Table` from a PyCapsule exported provider.""" - provider = _InternalTableProvider.from_capsule(capsule) return cls(provider) @classmethod def from_dataframe(cls, df: Any) -> Table: """Create a :class:`Table` from tabular data.""" - from datafusion.dataframe import DataFrame as DataFrameWrapper dataframe = df if isinstance(df, DataFrameWrapper) else DataFrameWrapper(df) @@ -222,7 +216,6 @@ def from_dataframe(cls, df: Any) -> Table: @classmethod def from_view(cls, df: Any) -> Table: """Deprecated helper for constructing tables from views.""" - from datafusion.dataframe import DataFrame as DataFrameWrapper if isinstance(df, DataFrameWrapper): @@ -249,7 +242,6 @@ def kind(self) -> str: def __datafusion_table_provider__(self) -> Any: """Expose the wrapped provider for FFI integrations.""" - exporter = getattr(self._table, "__datafusion_table_provider__", None) if exporter is None: msg = "Underlying object does not export __datafusion_table_provider__()" diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 4abe108d0..7834ceffd 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -319,7 +319,8 @@ def into_view(self) -> Table: """Convert ``DataFrame`` into a :class:`~datafusion.Table` for registration. This is the preferred way to obtain a view for - :py:meth:`~datafusion.context.SessionContext.register_table` for several reasons: + :py:meth:`~datafusion.context.SessionContext.register_table` for several + reasons: 1. **Direct API**: Most efficient path - directly calls the underlying Rust ``DataFrame.into_view()`` method without intermediate delegations. @@ -330,8 +331,8 @@ def into_view(self) -> Table: 4. **Deprecated alternatives**: The older ``Table.from_view`` helper is deprecated and issues warnings when used. - ``datafusion.Table.from_dataframe`` calls this method under the hood, and the - older ``Table.from_view`` helper is deprecated. + ``datafusion.Table.from_dataframe`` calls this method under the hood, + and the older ``Table.from_view`` helper is deprecated. The ``DataFrame`` remains valid after conversion, so it can still be used for additional queries alongside the returned view. diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 694ae0c18..243178797 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -369,6 +369,7 @@ def test_table_from_dataframe_internal(ctx): result = ctx.sql("SELECT * FROM from_internal_dataframe_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + def test_register_table_capsule_direct(ctx): df = ctx.from_pydict({"a": [1, 2]}) provider = df.into_view() diff --git a/python/tests/test_wrapper_coverage.py b/python/tests/test_wrapper_coverage.py index 53a987423..5df454d1d 100644 --- a/python/tests/test_wrapper_coverage.py +++ b/python/tests/test_wrapper_coverage.py @@ -21,7 +21,6 @@ import datafusion.substrait import pytest - IGNORED_EXPORTS = {"TableProvider"} # EnumType introduced in 3.11. 3.10 and prior it was called EnumMeta. @@ -31,7 +30,29 @@ from enum import EnumMeta as EnumType -def missing_exports(internal_obj, wrapped_obj) -> None: # noqa: C901 +def _check_enum_exports(internal_obj, wrapped_obj) -> None: + """Check that all enum values are present in wrapped object.""" + expected_values = [v for v in dir(internal_obj) if not v.startswith("__")] + for value in expected_values: + assert value in dir(wrapped_obj) + + +def _check_list_attribute(internal_attr, wrapped_attr) -> None: + """Check that list attributes match between internal and wrapped objects.""" + assert isinstance(wrapped_attr, list) + + # We have cases like __all__ that are a list and we want to be certain that + # every value in the list in the internal object is also in the wrapper list + for val in internal_attr: + if isinstance(val, str) and val in IGNORED_EXPORTS: + continue + if isinstance(val, str) and val.startswith("Raw"): + assert val[3:] in wrapped_attr + else: + assert val in wrapped_attr + + +def missing_exports(internal_obj, wrapped_obj) -> None: """ Identify if any of the rust exposted structs or functions do not have wrappers. @@ -44,9 +65,7 @@ def missing_exports(internal_obj, wrapped_obj) -> None: # noqa: C901 # Special case enums - EnumType overrides a some of the internal functions, # so check all of the values exist and move on if isinstance(wrapped_obj, EnumType): - expected_values = [v for v in dir(internal_obj) if not v.startswith("__")] - for value in expected_values: - assert value in dir(wrapped_obj) + _check_enum_exports(internal_obj, wrapped_obj) return if "__repr__" in internal_obj.__dict__ and "__repr__" not in wrapped_obj.__dict__: @@ -74,17 +93,7 @@ def missing_exports(internal_obj, wrapped_obj) -> None: # noqa: C901 continue if isinstance(internal_attr, list): - assert isinstance(wrapped_attr, list) - - # We have cases like __all__ that are a list and we want to be certain that - # every value in the list in the internal object is also in the wrapper list - for val in internal_attr: - if isinstance(val, str) and val in IGNORED_EXPORTS: - continue - if isinstance(val, str) and val.startswith("Raw"): - assert val[3:] in wrapped_attr - else: - assert val in wrapped_attr + _check_list_attribute(internal_attr, wrapped_attr) elif hasattr(internal_attr, "__dict__"): # Check all submodules recursively missing_exports(internal_attr, wrapped_attr) From 93f0a31cb70d1d3d501fad75c7372d1a999dbb65 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 19:10:19 +0800 Subject: [PATCH 20/34] Refactor test_table_loading to use Table instead of TableProvider --- .../python/tests/_test_table_provider.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/examples/datafusion-ffi-example/python/tests/_test_table_provider.py b/examples/datafusion-ffi-example/python/tests/_test_table_provider.py index 7d495eb10..01a961e0e 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_table_provider.py +++ b/examples/datafusion-ffi-example/python/tests/_test_table_provider.py @@ -18,16 +18,14 @@ from __future__ import annotations import pyarrow as pa -from datafusion import SessionContext, TableProvider +from datafusion import SessionContext, Table from datafusion_ffi_example import MyTableProvider def test_table_loading(): ctx = SessionContext() table = MyTableProvider(3, 2, 4) - ctx.register_table( - "t", TableProvider.from_capsule(table.__datafusion_table_provider__()) - ) + ctx.register_table("t", Table.from_capsule(table.__datafusion_table_provider__())) result = ctx.table("t").collect() assert len(result) == 4 From 7bc303d6e8988eeb0a3c9c2d0d29c56b3c89c6ef Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 20:54:13 +0800 Subject: [PATCH 21/34] Refactor aggregate tests to simplify result assertions and improve readability --- .../python/tests/_test_aggregate_udf.py | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py index 7ea6b295c..e8668873b 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py +++ b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py @@ -45,16 +45,13 @@ def test_ffi_aggregate_register(): result = ctx.sql("select my_custom_sum(a) from test_table group by b").collect() - assert len(result) == 2 + assert result assert result[0].num_columns == 1 - result = [r.column(0) for r in result] - expected = [ - pa.array([3], type=pa.int64()), - pa.array([3], type=pa.int64()), - ] + aggregates = pa.concat_arrays([batch.column(0) for batch in result]) - assert result == expected + assert len(aggregates) == 2 + assert aggregates.to_pylist() == [3, 3] def test_ffi_aggregate_call_directly(): @@ -65,13 +62,10 @@ def test_ffi_aggregate_call_directly(): ctx.table("test_table").aggregate([col("b")], [my_udaf(col("a"))]).collect() ) - assert len(result) == 2 + assert result assert result[0].num_columns == 2 - result = [r.column(1) for r in result] - expected = [ - pa.array([3], type=pa.int64()), - pa.array([3], type=pa.int64()), - ] + aggregates = pa.concat_arrays([batch.column(1) for batch in result]) - assert result == expected + assert len(aggregates) == 2 + assert aggregates.to_pylist() == [3, 3] From 442961403084b6d244be9df66c76f8fe59aa2fdf Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 22 Sep 2025 20:59:22 +0800 Subject: [PATCH 22/34] Add comments to clarify table normalization in aggregate tests --- .../python/tests/_test_aggregate_udf.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py index e8668873b..b2b0480db 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py +++ b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py @@ -48,6 +48,9 @@ def test_ffi_aggregate_register(): assert result assert result[0].num_columns == 1 + # Normalizing table registration in _normalize_table_provider feeds the Rust layer + # an actual TableProvider, so collect() emits the grouped rows in a single record batch + # instead of two separate batches. aggregates = pa.concat_arrays([batch.column(0) for batch in result]) assert len(aggregates) == 2 @@ -57,11 +60,14 @@ def test_ffi_aggregate_register(): def test_ffi_aggregate_call_directly(): ctx = setup_context_with_table() my_udaf = udaf(MySumUDF()) - + result = ( ctx.table("test_table").aggregate([col("b")], [my_udaf(col("a"))]).collect() ) + # Normalizing table registration in _normalize_table_provider feeds the Rust layer + # an actual TableProvider, so collect() emits the grouped rows in a single record batch + # instead of two separate batches. assert result assert result[0].num_columns == 2 From 38bb25a9cc278ea5fdededaf62498c9a42dd8551 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 10:20:36 -0400 Subject: [PATCH 23/34] Initial implementation of unified table suggestion --- .../python/tests/_test_aggregate_udf.py | 2 +- python/datafusion/__init__.py | 3 +- python/datafusion/catalog.py | 91 ++---------- python/datafusion/context.py | 6 +- python/datafusion/utils.py | 39 +---- python/tests/test_catalog.py | 12 +- python/tests/test_context.py | 45 +----- src/catalog.rs | 94 ++---------- src/context.rs | 13 +- src/dataframe.rs | 6 +- src/lib.rs | 3 - src/table.rs | 140 +++++++----------- src/udtf.rs | 6 +- src/utils.rs | 39 ++--- 14 files changed, 108 insertions(+), 391 deletions(-) diff --git a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py index b2b0480db..66e6889a8 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py +++ b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py @@ -60,7 +60,7 @@ def test_ffi_aggregate_register(): def test_ffi_aggregate_call_directly(): ctx = setup_context_with_table() my_udaf = udaf(MySumUDF()) - + result = ( ctx.table("test_table").aggregate([col("b")], [my_udaf(col("a"))]).collect() ) diff --git a/python/datafusion/__init__.py b/python/datafusion/__init__.py index 66565a4db..9ebd58ea6 100644 --- a/python/datafusion/__init__.py +++ b/python/datafusion/__init__.py @@ -34,7 +34,7 @@ from . import functions, object_store, substrait, unparser # The following imports are okay to remain as opaque to the user. -from ._internal import EXPECTED_PROVIDER_MSG, Config +from ._internal import Config from .catalog import Catalog, Database, Table from .col import col, column from .common import DFSchema @@ -65,7 +65,6 @@ __version__ = importlib_metadata.version(__name__) __all__ = [ - "EXPECTED_PROVIDER_MSG", "Accumulator", "AggregateUDF", "Catalog", diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index bd3300dab..add7d8e28 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -19,17 +19,15 @@ from __future__ import annotations -import warnings from abc import ABC, abstractmethod from typing import TYPE_CHECKING, Any, Protocol import datafusion._internal as df_internal -from datafusion._internal import EXPECTED_PROVIDER_MSG -from datafusion.utils import _normalize_table_provider if TYPE_CHECKING: import pyarrow as pa + from datafusion import DataFrame from datafusion.context import TableProviderExportable try: @@ -139,8 +137,7 @@ def register_table( Objects implementing ``__datafusion_table_provider__`` are also supported and treated as table provider instances. """ - provider = _normalize_table_provider(table) - return self._raw_schema.register_table(name, provider) + return self._raw_schema.register_table(name, table) def deregister_table(self, name: str) -> None: """Deregister a table provider from this schema.""" @@ -152,101 +149,37 @@ class Database(Schema): """See `Schema`.""" -_InternalRawTable = df_internal.catalog.RawTable -_InternalTableProvider = df_internal.TableProvider - -# Keep in sync with ``datafusion._internal.TableProvider.from_view``. -_FROM_VIEW_WARN_STACKLEVEL = 2 - - class Table: """DataFusion table or table provider wrapper.""" - __slots__ = ("_table",) + __slots__ = ("_inner",) def __init__( self, - table: _InternalRawTable | _InternalTableProvider | Table, + table: DataFrame | TableProviderExportable | pa.dataset.Dataset, ) -> None: """Wrap a low level table or table provider.""" - if isinstance(table, Table): - table = table.table - - if not isinstance(table, (_InternalRawTable, _InternalTableProvider)): - raise TypeError(EXPECTED_PROVIDER_MSG) - - self._table = table - - def __getattribute__(self, name: str) -> Any: - """Restrict provider-specific helpers to compatible tables.""" - if name == "__datafusion_table_provider__": - table = object.__getattribute__(self, "_table") - if not hasattr(table, "__datafusion_table_provider__"): - raise AttributeError(name) - return object.__getattribute__(self, name) + self._inner = df_internal.catalog.RawTable(table) def __repr__(self) -> str: """Print a string representation of the table.""" - return repr(self._table) - - @property - def table(self) -> _InternalRawTable | _InternalTableProvider: - """Return the wrapped low level table object.""" - return self._table + return repr(self._inner) - @classmethod - def from_dataset(cls, dataset: pa.dataset.Dataset) -> Table: + @deprecated("Use Table() constructor instead.") + @staticmethod + def from_dataset(dataset: pa.dataset.Dataset) -> Table: """Turn a :mod:`pyarrow.dataset` ``Dataset`` into a :class:`Table`.""" - return cls(_InternalRawTable.from_dataset(dataset)) - - @classmethod - def from_capsule(cls, capsule: Any) -> Table: - """Create a :class:`Table` from a PyCapsule exported provider.""" - provider = _InternalTableProvider.from_capsule(capsule) - return cls(provider) - - @classmethod - def from_dataframe(cls, df: Any) -> Table: - """Create a :class:`Table` from tabular data.""" - from datafusion.dataframe import DataFrame as DataFrameWrapper - - dataframe = df if isinstance(df, DataFrameWrapper) else DataFrameWrapper(df) - return dataframe.into_view() - - @classmethod - def from_view(cls, df: Any) -> Table: - """Deprecated helper for constructing tables from views.""" - from datafusion.dataframe import DataFrame as DataFrameWrapper - - if isinstance(df, DataFrameWrapper): - df = df.df - - provider = _InternalTableProvider.from_view(df) - warnings.warn( - "Table.from_view is deprecated; use DataFrame.into_view or " - "Table.from_dataframe instead.", - category=DeprecationWarning, - stacklevel=_FROM_VIEW_WARN_STACKLEVEL, - ) - return cls(provider) + return Table(dataset) @property def schema(self) -> pa.Schema: """Returns the schema associated with this table.""" - return self._table.schema + return self._inner.schema @property def kind(self) -> str: """Returns the kind of table.""" - return self._table.kind - - def __datafusion_table_provider__(self) -> Any: - """Expose the wrapped provider for FFI integrations.""" - exporter = getattr(self._table, "__datafusion_table_provider__", None) - if exporter is None: - msg = "Underlying object does not export __datafusion_table_provider__()" - raise AttributeError(msg) - return exporter() + return self._inner.kind class CatalogProvider(ABC): diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 4f1c18663..8e68e2ddd 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -33,7 +33,6 @@ from datafusion.dataframe import DataFrame from datafusion.expr import sort_list_to_raw_sort_list from datafusion.record_batch import RecordBatchStream -from datafusion.utils import _normalize_table_provider from ._internal import RuntimeEnvBuilder as RuntimeEnvBuilderInternal from ._internal import SessionConfig as SessionConfigInternal @@ -770,8 +769,7 @@ def register_table( table: DataFusion :class:`Table` or any object implementing ``__datafusion_table_provider__`` to add to the session context. """ - provider = _normalize_table_provider(table) - self.ctx.register_table(name, provider) + self.ctx.register_table(name, table) def deregister_table(self, name: str) -> None: """Remove a table from the session.""" @@ -1197,7 +1195,7 @@ def read_table(self, table: Table) -> DataFrame: :py:class:`~datafusion.catalog.ListingTable`, create a :py:class:`~datafusion.dataframe.DataFrame`. """ - return DataFrame(self.ctx.read_table(table.table)) + return DataFrame(self.ctx.read_table(table._inner)) def execute(self, plan: ExecutionPlan, partitions: int) -> RecordBatchStream: """Execute the ``plan`` and return the results.""" diff --git a/python/datafusion/utils.py b/python/datafusion/utils.py index eb3e3d626..71e5ffe4f 100644 --- a/python/datafusion/utils.py +++ b/python/datafusion/utils.py @@ -19,9 +19,7 @@ from __future__ import annotations from importlib import import_module, util -from typing import TYPE_CHECKING, Any - -from datafusion._internal import EXPECTED_PROVIDER_MSG +from typing import Any _PYARROW_DATASET_TYPES: tuple[type[Any], ...] _dataset_spec = util.find_spec("pyarrow.dataset") @@ -37,38 +35,3 @@ if isinstance(value, type) and issubclass(value, dataset_base): dataset_types.add(value) _PYARROW_DATASET_TYPES = tuple(dataset_types) - -if TYPE_CHECKING: # pragma: no cover - imported for typing only - from datafusion.catalog import Table - from datafusion.context import TableProviderExportable - - -def _normalize_table_provider( - table: Table | TableProviderExportable | Any, -) -> Any: - """Return the underlying provider for supported table inputs. - - Args: - table: A :class:`~datafusion.Table`, object exporting a DataFusion table - provider via ``__datafusion_table_provider__``, or compatible - :mod:`pyarrow.dataset` implementation. - - Returns: - The object expected by the Rust bindings for table registration. - - Raises: - TypeError: If ``table`` is not a supported table provider input. - """ - from datafusion.catalog import Table as _Table - - if isinstance(table, _Table): - return table.table - - if _PYARROW_DATASET_TYPES and isinstance(table, _PYARROW_DATASET_TYPES): - return table - - provider_factory = getattr(table, "__datafusion_table_provider__", None) - if callable(provider_factory): - return table - - raise TypeError(EXPECTED_PROVIDER_MSG) diff --git a/python/tests/test_catalog.py b/python/tests/test_catalog.py index fd91d6677..98ecf8de2 100644 --- a/python/tests/test_catalog.py +++ b/python/tests/test_catalog.py @@ -20,7 +20,7 @@ import pyarrow as pa import pyarrow.dataset as ds import pytest -from datafusion import EXPECTED_PROVIDER_MSG, SessionContext, Table +from datafusion import SessionContext, Table # Note we take in `database` as a variable even though we don't use @@ -186,16 +186,6 @@ def test_schema_register_table_with_pyarrow_dataset(ctx: SessionContext): schema.deregister_table(table_name) -def test_schema_register_table_with_dataframe_errors(ctx: SessionContext): - schema = ctx.catalog().schema() - df = ctx.from_pydict({"a": [1]}) - - with pytest.raises(Exception) as exc_info: - schema.register_table("bad", df) - - assert str(exc_info.value) == EXPECTED_PROVIDER_MSG - - def test_in_end_to_end_python_providers(ctx: SessionContext): """Test registering all python providers and running a query against them.""" diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 243178797..e04cc1054 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -22,7 +22,6 @@ import pyarrow.dataset as ds import pytest from datafusion import ( - EXPECTED_PROVIDER_MSG, DataFrame, RuntimeEnvBuilder, SessionConfig, @@ -341,20 +340,9 @@ def test_register_table_from_dataframe_into_view(ctx): assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] -def test_table_from_capsule(ctx): - df = ctx.from_pydict({"a": [1, 2]}) - table = df.into_view() - capsule = table.__datafusion_table_provider__() - table2 = Table.from_capsule(capsule) - assert isinstance(table2, Table) - ctx.register_table("capsule_tbl", table2) - result = ctx.sql("SELECT * FROM capsule_tbl").collect() - assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] - - def test_table_from_dataframe(ctx): df = ctx.from_pydict({"a": [1, 2]}) - table = Table.from_dataframe(df) + table = Table(df) assert isinstance(table, Table) ctx.register_table("from_dataframe_tbl", table) result = ctx.sql("SELECT * FROM from_dataframe_tbl").collect() @@ -363,42 +351,13 @@ def test_table_from_dataframe(ctx): def test_table_from_dataframe_internal(ctx): df = ctx.from_pydict({"a": [1, 2]}) - table = Table.from_dataframe(df.df) + table = Table(df.df) assert isinstance(table, Table) ctx.register_table("from_internal_dataframe_tbl", table) result = ctx.sql("SELECT * FROM from_internal_dataframe_tbl").collect() assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] -def test_register_table_capsule_direct(ctx): - df = ctx.from_pydict({"a": [1, 2]}) - provider = df.into_view() - - class CapsuleProvider: - def __init__(self, inner): - self._inner = inner - - def __datafusion_table_provider__(self): - return self._inner.__datafusion_table_provider__() - - ctx.register_table("capsule_direct_tbl", CapsuleProvider(provider)) - result = ctx.sql("SELECT * FROM capsule_direct_tbl").collect() - assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] - - -def test_table_from_capsule_invalid(): - with pytest.raises(RuntimeError): - Table.from_capsule(object()) - - -def test_register_table_with_dataframe_errors(ctx): - df = ctx.from_pydict({"a": [1]}) - with pytest.raises(TypeError) as exc_info: - ctx.register_table("bad", df) - - assert str(exc_info.value) == EXPECTED_PROVIDER_MSG - - def test_register_dataset(ctx): # create a RecordBatch and register it as a pyarrow.dataset.Dataset batch = pa.RecordBatch.from_arrays( diff --git a/src/catalog.rs b/src/catalog.rs index 03e6408cd..02ebfb93c 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -17,17 +17,14 @@ use crate::dataset::Dataset; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; -use crate::table::PyTableProvider; -use crate::utils::{ - coerce_table_provider, table_provider_from_pycapsule, validate_pycapsule, wait_for_future, -}; +use crate::table::PyTable; +use crate::utils::{validate_pycapsule, wait_for_future}; use async_trait::async_trait; use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; use datafusion::common::DataFusionError; use datafusion::{ - arrow::pyarrow::ToPyArrow, catalog::{CatalogProvider, SchemaProvider}, - datasource::{TableProvider, TableType}, + datasource::TableProvider, }; use datafusion_ffi::schema_provider::{FFI_SchemaProvider, ForeignSchemaProvider}; use pyo3::exceptions::PyKeyError; @@ -50,12 +47,6 @@ pub struct PySchema { pub schema: Arc, } -#[pyclass(name = "RawTable", module = "datafusion.catalog", subclass)] -#[derive(Clone)] -pub struct PyTable { - pub table: Arc, -} - impl From> for PyCatalog { fn from(catalog: Arc) -> Self { Self { catalog } @@ -68,16 +59,6 @@ impl From> for PySchema { } } -impl PyTable { - pub fn new(table: Arc) -> Self { - Self { table } - } - - pub fn table(&self) -> Arc { - self.table.clone() - } -} - #[pymethods] impl PyCatalog { #[new] @@ -183,7 +164,7 @@ impl PySchema { fn table(&self, name: &str, py: Python) -> PyDataFusionResult { if let Some(table) = wait_for_future(py, self.schema.table(name))?? { - Ok(PyTable::new(table)) + Ok(PyTable::from(table)) } else { Err(PyDataFusionError::Common(format!( "Table not found: {name}" @@ -197,12 +178,12 @@ impl PySchema { Ok(format!("Schema(table_names=[{}])", names.join(";"))) } - fn register_table(&self, name: &str, table_provider: Bound<'_, PyAny>) -> PyResult<()> { - let provider = coerce_table_provider(&table_provider).map_err(PyErr::from)?; + fn register_table(&self, name: &str, table_provider: &Bound<'_, PyAny>) -> PyResult<()> { + let table = PyTable::new(table_provider)?; let _ = self .schema - .register_table(name.to_string(), provider) + .register_table(name.to_string(), table.table) .map_err(py_datafusion_err)?; Ok(()) @@ -218,43 +199,6 @@ impl PySchema { } } -#[pymethods] -impl PyTable { - /// Get a reference to the schema for this table - #[getter] - fn schema(&self, py: Python) -> PyResult { - self.table.schema().to_pyarrow(py) - } - - #[staticmethod] - fn from_dataset(py: Python<'_>, dataset: &Bound<'_, PyAny>) -> PyResult { - let ds = Arc::new(Dataset::new(dataset, py).map_err(py_datafusion_err)?) - as Arc; - - Ok(Self::new(ds)) - } - - /// Get the type of this table for metadata/catalog purposes. - #[getter] - fn kind(&self) -> &str { - match self.table.table_type() { - TableType::Base => "physical", - TableType::View => "view", - TableType::Temporary => "temporary", - } - } - - fn __repr__(&self) -> PyResult { - let kind = self.kind(); - Ok(format!("Table(kind={kind})")) - } - - // fn scan - // fn statistics - // fn has_exact_statistics - // fn supports_filter_pushdown -} - #[derive(Debug)] pub(crate) struct RustWrappedPySchemaProvider { schema_provider: PyObject, @@ -287,27 +231,9 @@ impl RustWrappedPySchemaProvider { return Ok(None); } - if let Some(provider) = table_provider_from_pycapsule(&py_table)? { - Ok(Some(provider)) - } else { - if let Ok(inner_table) = py_table.getattr("table") { - if let Ok(inner_table) = inner_table.extract::() { - return Ok(Some(inner_table.table)); - } - } - - if let Ok(py_provider) = py_table.extract::() { - return Ok(Some(py_provider.into_inner())); - } + let table = PyTable::new(&py_table)?; - match py_table.extract::() { - Ok(py_table) => Ok(Some(py_table.table)), - Err(_) => { - let ds = Dataset::new(&py_table, py).map_err(py_datafusion_err)?; - Ok(Some(Arc::new(ds) as Arc)) - } - } - } + Ok(Some(table.table)) }) } } @@ -348,7 +274,7 @@ impl SchemaProvider for RustWrappedPySchemaProvider { name: String, table: Arc, ) -> datafusion::common::Result>> { - let py_table = PyTable::new(table); + let py_table = PyTable::from(table); Python::with_gil(|py| { let provider = self.schema_provider.bind(py); let _ = provider diff --git a/src/context.rs b/src/context.rs index d2c9b1c98..d2a01af91 100644 --- a/src/context.rs +++ b/src/context.rs @@ -31,7 +31,7 @@ use uuid::Uuid; use pyo3::exceptions::{PyKeyError, PyValueError}; use pyo3::prelude::*; -use crate::catalog::{PyCatalog, PyTable, RustWrappedPyCatalogProvider}; +use crate::catalog::{PyCatalog, RustWrappedPyCatalogProvider}; use crate::dataframe::PyDataFrame; use crate::dataset::Dataset; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionResult}; @@ -41,13 +41,12 @@ use crate::record_batch::PyRecordBatchStream; use crate::sql::exceptions::py_value_err; use crate::sql::logical::PyLogicalPlan; use crate::store::StorageContexts; +use crate::table::PyTable; use crate::udaf::PyAggregateUDF; use crate::udf::PyScalarUDF; use crate::udtf::PyTableFunction; use crate::udwf::PyWindowUDF; -use crate::utils::{ - coerce_table_provider, get_global_ctx, get_tokio_runtime, validate_pycapsule, wait_for_future, -}; +use crate::utils::{get_global_ctx, get_tokio_runtime, validate_pycapsule, wait_for_future}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::arrow::record_batch::RecordBatch; @@ -606,11 +605,11 @@ impl PySessionContext { pub fn register_table( &mut self, name: &str, - table_provider: Bound<'_, PyAny>, + table: Bound<'_, PyAny>, ) -> PyDataFusionResult<()> { - let provider = coerce_table_provider(&table_provider)?; + let table = PyTable::new(&table)?; - self.ctx.register_table(name, provider)?; + self.ctx.register_table(name, table.table)?; Ok(()) } diff --git a/src/dataframe.rs b/src/dataframe.rs index 17900afda..6aba49479 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -47,7 +47,7 @@ use crate::expr::sort_expr::to_sort_expressions; use crate::physical_plan::PyExecutionPlan; use crate::record_batch::PyRecordBatchStream; use crate::sql::logical::PyLogicalPlan; -pub use crate::table::PyTableProvider; +use crate::table::PyTable; use crate::utils::{ get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, validate_pycapsule, wait_for_future, }; @@ -402,12 +402,12 @@ impl PyDataFrame { /// because we're working with Python bindings /// where objects are shared #[allow(clippy::wrong_self_convention)] - pub fn into_view(&self) -> PyDataFusionResult { + pub fn into_view(&self) -> PyDataFusionResult { // Call the underlying Rust DataFrame::into_view method. // Note that the Rust method consumes self; here we clone the inner Arc // so that we don't invalidate this PyDataFrame. let table_provider = self.df.as_ref().clone().into_view(); - Ok(PyTableProvider::new(table_provider)) + Ok(PyTable::from(table_provider)) } #[pyo3(signature = (*args))] diff --git a/src/lib.rs b/src/lib.rs index d45992db2..0361c7315 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -81,8 +81,6 @@ fn _internal(py: Python, m: Bound<'_, PyModule>) -> PyResult<()> { // Initialize logging pyo3_log::init(); - m.add("EXPECTED_PROVIDER_MSG", crate::utils::EXPECTED_PROVIDER_MSG)?; - // Register the python classes m.add_class::()?; m.add_class::()?; @@ -100,7 +98,6 @@ fn _internal(py: Python, m: Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; m.add_class::()?; - m.add_class::()?; let catalog = PyModule::new(py, "catalog")?; catalog::init_module(&catalog)?; diff --git a/src/table.rs b/src/table.rs index 29476e473..db2eea0e1 100644 --- a/src/table.rs +++ b/src/table.rs @@ -15,106 +15,80 @@ // specific language governing permissions and limitations // under the License. -use std::ffi::CString; -use std::sync::Arc; - -use datafusion::datasource::TableProvider; -use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; -use pyo3::exceptions::PyDeprecationWarning; +use arrow::pyarrow::ToPyArrow; +use datafusion::datasource::{TableProvider, TableType}; use pyo3::prelude::*; -use pyo3::types::{PyCapsule, PyDict}; +use std::sync::Arc; -use crate::catalog::PyTable; use crate::dataframe::PyDataFrame; -use crate::errors::{py_datafusion_err, PyDataFusionResult}; -use crate::utils::{get_tokio_runtime, validate_pycapsule}; +use crate::dataset::Dataset; +use crate::utils::table_provider_from_pycapsule; -/// Represents a table provider that can be registered with DataFusion -#[pyclass(name = "TableProvider", module = "datafusion")] +#[pyclass(name = "RawTable", module = "datafusion.catalog", subclass)] #[derive(Clone)] -pub struct PyTableProvider { - pub(crate) provider: Arc, +pub struct PyTable { + pub table: Arc, } -impl PyTableProvider { - pub(crate) fn new(provider: Arc) -> Self { - Self { provider } - } - - /// Return a `PyTable` wrapper around this provider. - /// - /// Historically callers chained `as_table().table()` to access the - /// underlying [`Arc`]. Prefer [`as_arc`] or - /// [`into_inner`] for direct access instead. - pub fn as_table(&self) -> PyTable { - PyTable::new(Arc::clone(&self.provider)) - } - - /// Return a clone of the inner [`TableProvider`]. - pub fn as_arc(&self) -> Arc { - Arc::clone(&self.provider) - } - - /// Consume this wrapper and return the inner [`TableProvider`]. - pub fn into_inner(self) -> Arc { - self.provider +impl PyTable { + pub fn table(&self) -> Arc { + self.table.clone() } } #[pymethods] -impl PyTableProvider { - /// Create a `TableProvider` from a PyCapsule containing an FFI pointer - #[staticmethod] - pub fn from_capsule(capsule: Bound<'_, PyAny>) -> PyResult { - let capsule = capsule.downcast::().map_err(py_datafusion_err)?; - validate_pycapsule(capsule, "datafusion_table_provider")?; - - let provider = unsafe { capsule.reference::() }; - let provider: ForeignTableProvider = provider.into(); - - Ok(Self::new(Arc::new(provider))) +impl PyTable { + #[new] + pub fn new(obj: &Bound<'_, PyAny>) -> PyResult { + if let Ok(py_table) = obj.extract::() { + Ok(py_table) + } else if let Ok(py_table) = obj + .getattr("_inner") + .and_then(|inner| inner.extract::()) + { + Ok(py_table) + } else if let Ok(py_df) = obj.extract::() { + let provider = py_df.inner_df().as_ref().clone().into_view(); + Ok(PyTable::from(provider)) + } else if let Ok(py_df) = obj + .getattr("df") + .and_then(|inner| inner.extract::()) + { + let provider = py_df.inner_df().as_ref().clone().into_view(); + Ok(PyTable::from(provider)) + } else if let Some(provider) = table_provider_from_pycapsule(obj)? { + Ok(PyTable::from(provider)) + } else { + let py = obj.py(); + let provider = Arc::new(Dataset::new(obj, py)?) as Arc; + Ok(PyTable::from(provider)) + } } - /// Create a `TableProvider` from a `DataFrame`. - /// - /// This method simply delegates to `DataFrame.into_view`. - #[staticmethod] - pub fn from_dataframe(df: &PyDataFrame) -> Self { - // Clone the inner DataFrame and convert it into a view TableProvider. - // `into_view` consumes a DataFrame, so clone the underlying DataFrame - Self::new(df.inner_df().as_ref().clone().into_view()) + /// Get a reference to the schema for this table + #[getter] + fn schema(&self, py: Python) -> PyResult { + self.table.schema().to_pyarrow(py) } - /// Create a `TableProvider` from a `DataFrame` by converting it into a view. - /// - /// Deprecated: prefer `DataFrame.into_view` or - /// `Table.from_dataframe` instead. - #[staticmethod] - pub fn from_view(py: Python<'_>, df: &PyDataFrame) -> PyDataFusionResult { - let kwargs = PyDict::new(py); - // Keep stack level consistent with python/datafusion/table_provider.py - kwargs.set_item("stacklevel", 2)?; - py.import("warnings")?.call_method( - "warn", - ( - "PyTableProvider.from_view() is deprecated; use DataFrame.into_view() or Table.from_dataframe() instead.", - py.get_type::(), - ), - Some(&kwargs), - )?; - Ok(Self::from_dataframe(df)) + /// Get the type of this table for metadata/catalog purposes. + #[getter] + fn kind(&self) -> &str { + match self.table.table_type() { + TableType::Base => "physical", + TableType::View => "view", + TableType::Temporary => "temporary", + } } - fn __datafusion_table_provider__<'py>( - &self, - py: Python<'py>, - ) -> PyResult> { - let name = CString::new("datafusion_table_provider").unwrap(); - - let runtime = get_tokio_runtime().0.handle().clone(); - let provider: Arc = self.provider.clone(); - let provider = FFI_TableProvider::new(provider, false, Some(runtime)); + fn __repr__(&self) -> PyResult { + let kind = self.kind(); + Ok(format!("Table(kind={kind})")) + } +} - PyCapsule::new(py, provider, Some(name.clone())) +impl From> for PyTable { + fn from(table: Arc) -> Self { + Self { table } } } diff --git a/src/udtf.rs b/src/udtf.rs index 311d67e24..7a1b0aaf0 100644 --- a/src/udtf.rs +++ b/src/udtf.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use crate::errors::{py_datafusion_err, to_datafusion_err}; use crate::expr::PyExpr; -use crate::table::PyTableProvider; +use crate::table::PyTable; use crate::utils::{table_provider_from_pycapsule, validate_pycapsule}; use datafusion::catalog::{TableFunctionImpl, TableProvider}; use datafusion::error::Result as DataFusionResult; @@ -70,11 +70,11 @@ impl PyTableFunction { } #[pyo3(signature = (*args))] - pub fn __call__(&self, args: Vec) -> PyResult { + pub fn __call__(&self, args: Vec) -> PyResult { let args: Vec = args.iter().map(|e| e.expr.clone()).collect(); let table_provider = self.call(&args).map_err(py_datafusion_err)?; - Ok(PyTableProvider::new(table_provider)) + Ok(PyTable::from(table_provider)) } fn __repr__(&self) -> PyResult { diff --git a/src/utils.rs b/src/utils.rs index 6d5755f88..0fcfadcea 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -15,19 +15,17 @@ // specific language governing permissions and limitations // under the License. +use crate::errors::py_datafusion_err; use crate::{ - catalog::PyTable, common::data_type::PyScalarValue, - dataframe::PyDataFrame, - dataset::Dataset, errors::{PyDataFusionError, PyDataFusionResult}, - table::PyTableProvider, TokioRuntime, }; use datafusion::{ common::ScalarValue, datasource::TableProvider, execution::context::SessionContext, logical_expr::Volatility, }; +use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; use pyo3::prelude::*; use pyo3::{exceptions::PyValueError, types::PyCapsule}; use std::{ @@ -37,8 +35,6 @@ use std::{ }; use tokio::{runtime::Runtime, time::sleep}; -pub(crate) const EXPECTED_PROVIDER_MSG: &str = - "Expected a Table. Convert DataFrames with \"DataFrame.into_view()\" or \"Table.from_dataframe()\"."; /// Utility to get the Tokio Runtime from Python #[inline] pub(crate) fn get_tokio_runtime() -> &'static TokioRuntime { @@ -133,32 +129,15 @@ pub(crate) fn table_provider_from_pycapsule( ) -> PyResult>> { if obj.hasattr("__datafusion_table_provider__")? { let capsule = obj.getattr("__datafusion_table_provider__")?.call0()?; - let provider = PyTableProvider::from_capsule(capsule)?; - Ok(Some(provider.into_inner())) - } else { - Ok(None) - } -} + let capsule = capsule.downcast::().map_err(py_datafusion_err)?; + validate_pycapsule(capsule, "datafusion_table_provider")?; -pub(crate) fn coerce_table_provider( - obj: &Bound, -) -> PyDataFusionResult> { - if let Ok(py_table) = obj.extract::() { - Ok(py_table.table()) - } else if let Ok(py_provider) = obj.extract::() { - Ok(py_provider.into_inner()) - } else if obj.is_instance_of::() - || obj - .getattr("df") - .is_ok_and(|inner| inner.is_instance_of::()) - { - Err(PyDataFusionError::Common(EXPECTED_PROVIDER_MSG.to_string())) - } else if let Some(provider) = table_provider_from_pycapsule(obj)? { - Ok(provider) + let provider = unsafe { capsule.reference::() }; + let provider: ForeignTableProvider = provider.into(); + + Ok(Some(Arc::new(provider))) } else { - let py = obj.py(); - let provider = Dataset::new(obj, py)?; - Ok(Arc::new(provider) as Arc) + Ok(None) } } From 49abaebe00506dd411786314d77169bd51e19f44 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 12:17:54 -0400 Subject: [PATCH 24/34] update unit tests --- python/tests/test_catalog.py | 2 +- python/tests/test_context.py | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/python/tests/test_catalog.py b/python/tests/test_catalog.py index 98ecf8de2..f0c492ce7 100644 --- a/python/tests/test_catalog.py +++ b/python/tests/test_catalog.py @@ -53,7 +53,7 @@ def create_dataset() -> Table: names=["a", "b"], ) dataset = ds.dataset([batch]) - return Table.from_dataset(dataset) + return Table(dataset) class CustomSchemaProvider(dfn.catalog.SchemaProvider): diff --git a/python/tests/test_context.py b/python/tests/test_context.py index e04cc1054..50076c9b6 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -331,6 +331,13 @@ def test_deregister_table(ctx, database): assert public.names() == {"csv1", "csv2"} +def test_register_table_from_dataframe(ctx): + df = ctx.from_pydict({"a": [1, 2]}) + ctx.register_table("df_tbl", df) + result = ctx.sql("SELECT * FROM df_tbl").collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + def test_register_table_from_dataframe_into_view(ctx): df = ctx.from_pydict({"a": [1, 2]}) table = df.into_view() From cb7a7553e0b0cac67dc349ece44be0b062916762 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 12:51:54 -0400 Subject: [PATCH 25/34] Change documentation to be more user oriented --- docs/source/conf.py | 15 -------- docs/source/contributor-guide/ffi.rst | 2 +- docs/source/user-guide/data-sources.rst | 14 +------- docs/source/user-guide/io/table_provider.rst | 29 ++------------- python/datafusion/utils.py | 37 -------------------- 5 files changed, 5 insertions(+), 92 deletions(-) delete mode 100644 python/datafusion/utils.py diff --git a/docs/source/conf.py b/docs/source/conf.py index 18d5f1232..01813b032 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -83,9 +83,6 @@ def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa # Duplicate modules (skip module-level docs to avoid duplication) ("module", "datafusion.col"), ("module", "datafusion.udf"), - # Private variables causing duplicate documentation - ("data", "datafusion.utils._PYARROW_DATASET_TYPES"), - ("variable", "datafusion.utils._PYARROW_DATASET_TYPES"), # Deprecated ("class", "datafusion.substrait.serde"), ("class", "datafusion.substrait.plan"), @@ -104,18 +101,6 @@ def autoapi_skip_member_fn(app, what, name, obj, skip, options) -> bool: # noqa if (what, name) in skip_contents: skip = True - # Skip private module-level names (those whose final component - # starts with an underscore) when AutoAPI is rendering data or - # variable entries. Many internal module-level constants are - # implementation details (for example private pyarrow dataset type - # mappings) that would otherwise be emitted as top-level "data" - # or "variable" docs. Filtering them here avoids noisy, - # duplicate, or implementation-specific entries in the public - # documentation while still allowing public members and types to - # be documented normally. - if name.split(".")[-1].startswith("_") and what in ("data", "variable"): - skip = True - return skip diff --git a/docs/source/contributor-guide/ffi.rst b/docs/source/contributor-guide/ffi.rst index e8abde00d..72fba8e34 100644 --- a/docs/source/contributor-guide/ffi.rst +++ b/docs/source/contributor-guide/ffi.rst @@ -34,7 +34,7 @@ as performant as possible and to utilize the features of DataFusion, you may dec your source in Rust and then expose it through `PyO3 `_ as a Python library. At first glance, it may appear the best way to do this is to add the ``datafusion-python`` -crate as a dependency, produce a DataFusion table in Rust, and then register it with the +crate as a dependency, provide a ``PyTable``, and then to register it with the ``SessionContext``. Unfortunately, this will not work. When you produce your code as a Python library and it needs to interact with the DataFusion diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index bedbabffb..19d5f96d2 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -155,22 +155,10 @@ as Delta Lake. This will require a recent version of from datafusion import Table delta_table = DeltaTable("path_to_table") - table = Table.from_capsule(delta_table.__datafusion_table_provider__()) - ctx.register_table("my_delta_table", table) + ctx.register_table("my_delta_table", delta_table) df = ctx.table("my_delta_table") df.show() -Objects that implement ``__datafusion_table_provider__`` are supported directly by -:py:meth:`~datafusion.context.SessionContext.register_table`, making it easy to -work with custom table providers from Python libraries such as Delta Lake. - -.. note:: - - :py:meth:`~datafusion.context.SessionContext.register_table_provider` is - deprecated. Use - :py:meth:`~datafusion.context.SessionContext.register_table` with a - :py:class:`~datafusion.Table` instead. - On older versions of ``deltalake`` (prior to 0.22) you can use the `Arrow DataSet `_ interface to import to DataFusion, but this does not support features such as filter push down diff --git a/docs/source/user-guide/io/table_provider.rst b/docs/source/user-guide/io/table_provider.rst index 0dfc07c3b..29e5d9880 100644 --- a/docs/source/user-guide/io/table_provider.rst +++ b/docs/source/user-guide/io/table_provider.rst @@ -37,7 +37,7 @@ A complete example can be found in the `examples folder , ) -> PyResult> { - let name = CString::new("datafusion_table_provider").unwrap(); + let name = cr"datafusion_table_provider".into(); let provider = Arc::new(self.clone()); let provider = FFI_TableProvider::new(provider, false, None); @@ -48,18 +48,7 @@ A complete example can be found in the `examples folder Date: Wed, 1 Oct 2025 12:58:35 -0400 Subject: [PATCH 26/34] Update ffi examples --- .../python/tests/_test_aggregate_udf.py | 28 +++++++++---------- .../python/tests/_test_catalog_provider.py | 6 ++-- .../python/tests/_test_table_provider.py | 4 +-- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py index 66e6889a8..7ea6b295c 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py +++ b/examples/datafusion-ffi-example/python/tests/_test_aggregate_udf.py @@ -45,16 +45,16 @@ def test_ffi_aggregate_register(): result = ctx.sql("select my_custom_sum(a) from test_table group by b").collect() - assert result + assert len(result) == 2 assert result[0].num_columns == 1 - # Normalizing table registration in _normalize_table_provider feeds the Rust layer - # an actual TableProvider, so collect() emits the grouped rows in a single record batch - # instead of two separate batches. - aggregates = pa.concat_arrays([batch.column(0) for batch in result]) + result = [r.column(0) for r in result] + expected = [ + pa.array([3], type=pa.int64()), + pa.array([3], type=pa.int64()), + ] - assert len(aggregates) == 2 - assert aggregates.to_pylist() == [3, 3] + assert result == expected def test_ffi_aggregate_call_directly(): @@ -65,13 +65,13 @@ def test_ffi_aggregate_call_directly(): ctx.table("test_table").aggregate([col("b")], [my_udaf(col("a"))]).collect() ) - # Normalizing table registration in _normalize_table_provider feeds the Rust layer - # an actual TableProvider, so collect() emits the grouped rows in a single record batch - # instead of two separate batches. - assert result + assert len(result) == 2 assert result[0].num_columns == 2 - aggregates = pa.concat_arrays([batch.column(1) for batch in result]) + result = [r.column(1) for r in result] + expected = [ + pa.array([3], type=pa.int64()), + pa.array([3], type=pa.int64()), + ] - assert len(aggregates) == 2 - assert aggregates.to_pylist() == [3, 3] + assert result == expected diff --git a/examples/datafusion-ffi-example/python/tests/_test_catalog_provider.py b/examples/datafusion-ffi-example/python/tests/_test_catalog_provider.py index 72aadf64c..1bf1bf136 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_catalog_provider.py +++ b/examples/datafusion-ffi-example/python/tests/_test_catalog_provider.py @@ -36,9 +36,9 @@ def test_catalog_provider(): my_catalog_schemas = my_catalog.names() assert expected_schema_name in my_catalog_schemas - my_database = my_catalog.database(expected_schema_name) - assert expected_table_name in my_database.names() - my_table = my_database.table(expected_table_name) + my_schema = my_catalog.schema(expected_schema_name) + assert expected_table_name in my_schema.names() + my_table = my_schema.table(expected_table_name) assert expected_table_columns == my_table.schema.names result = ctx.table( diff --git a/examples/datafusion-ffi-example/python/tests/_test_table_provider.py b/examples/datafusion-ffi-example/python/tests/_test_table_provider.py index 01a961e0e..91a77e61b 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_table_provider.py +++ b/examples/datafusion-ffi-example/python/tests/_test_table_provider.py @@ -18,14 +18,14 @@ from __future__ import annotations import pyarrow as pa -from datafusion import SessionContext, Table +from datafusion import SessionContext from datafusion_ffi_example import MyTableProvider def test_table_loading(): ctx = SessionContext() table = MyTableProvider(3, 2, 4) - ctx.register_table("t", Table.from_capsule(table.__datafusion_table_provider__())) + ctx.register_table("t", table) result = ctx.table("t").collect() assert len(result) == 4 From c70968ff22e09a30f628c626fd7e57f6a67c1cf7 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 13:12:05 -0400 Subject: [PATCH 27/34] Update documentation --- python/datafusion/catalog.py | 13 +++++-------- python/datafusion/context.py | 4 +++- src/table.rs | 12 ++++++++++++ 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index add7d8e28..c4a7d8045 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -130,13 +130,11 @@ def table(self, name: str) -> Table: return Table(self._raw_schema.table(name)) def register_table( - self, name: str, table: Table | TableProviderExportable | Any + self, + name: str, + table: Table | TableProviderExportable | DataFrame | pa.dataset.Dataset, ) -> None: - """Register a table or table provider in this schema. - - Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as table provider instances. - """ + """Register a table in this schema.""" return self._raw_schema.register_table(name, table) def deregister_table(self, name: str) -> None: @@ -155,8 +153,7 @@ class Table: __slots__ = ("_inner",) def __init__( - self, - table: DataFrame | TableProviderExportable | pa.dataset.Dataset, + self, table: Table | TableProviderExportable | DataFrame | pa.dataset.Dataset ) -> None: """Wrap a low level table or table provider.""" self._inner = df_internal.catalog.RawTable(table) diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 8e68e2ddd..540099255 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -750,7 +750,9 @@ def register_view(self, name: str, df: DataFrame) -> None: self.ctx.register_table(name, view) def register_table( - self, name: str, table: Table | TableProviderExportable | Any + self, + name: str, + table: Table | TableProviderExportable | DataFrame | pa.dataset.Dataset, ) -> None: """Register a :py:class:`~datafusion.Table` with this context. diff --git a/src/table.rs b/src/table.rs index db2eea0e1..812581edb 100644 --- a/src/table.rs +++ b/src/table.rs @@ -24,6 +24,9 @@ use crate::dataframe::PyDataFrame; use crate::dataset::Dataset; use crate::utils::table_provider_from_pycapsule; +/// This struct is used as a common method for all TableProviders, +/// whether they refer to an FFI provider, an internally known +/// implementation, a dataset, or a dataframe view. #[pyclass(name = "RawTable", module = "datafusion.catalog", subclass)] #[derive(Clone)] pub struct PyTable { @@ -38,6 +41,15 @@ impl PyTable { #[pymethods] impl PyTable { + /// Instantiate from any Python object that supports any of the table + /// types. We do not know a priori when using this method if the object + /// will be passed a wrapped or raw class. Here we handle all of the + /// following object types: + /// + /// - PyTable (essentially a clone operation), but either raw or wrapped + /// - DataFrame, either raw or wrapped + /// - FFI Table Providers via PyCapsule + /// - PyArrow Dataset objects #[new] pub fn new(obj: &Bound<'_, PyAny>) -> PyResult { if let Ok(py_table) = obj.extract::() { From 90b3cb675309a0e4e129dd480ea6db44dd0ac3c5 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 13:23:19 -0400 Subject: [PATCH 28/34] More documentation --- docs/source/user-guide/data-sources.rst | 1 - python/datafusion/catalog.py | 14 ++++++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/docs/source/user-guide/data-sources.rst b/docs/source/user-guide/data-sources.rst index 19d5f96d2..26f1303c4 100644 --- a/docs/source/user-guide/data-sources.rst +++ b/docs/source/user-guide/data-sources.rst @@ -152,7 +152,6 @@ as Delta Lake. This will require a recent version of .. code-block:: python from deltalake import DeltaTable - from datafusion import Table delta_table = DeltaTable("path_to_table") ctx.register_table("my_delta_table", delta_table) diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index c4a7d8045..f2216843e 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -148,14 +148,24 @@ class Database(Schema): class Table: - """DataFusion table or table provider wrapper.""" + """A DataFusion table. + + Internally we currently support the following types of tables: + + - Tables created using built-in DataFusion methods, such as + reading from CSV or Parquet + - pyarrow datasets + - DataFusion DataFrames, which will be converted into a view + - Externally provided tables implemented with the FFI PyCapsule + interface (advanced) + """ __slots__ = ("_inner",) def __init__( self, table: Table | TableProviderExportable | DataFrame | pa.dataset.Dataset ) -> None: - """Wrap a low level table or table provider.""" + """Constructor.""" self._inner = df_internal.catalog.RawTable(table) def __repr__(self) -> str: From 25d414116986831edaf82daef47be05d3eed36de Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 13:27:14 -0400 Subject: [PATCH 29/34] Make documentation more user facing --- python/datafusion/catalog.py | 3 --- python/datafusion/context.py | 20 ++++---------------- 2 files changed, 4 insertions(+), 19 deletions(-) diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index f2216843e..ba95a8716 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -250,9 +250,6 @@ def register_table( # noqa: B027 This method is optional. If your schema provides a fixed list of tables, you do not need to implement this method. - - Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as table provider instances. """ def deregister_table(self, name: str, cascade: bool) -> None: # noqa: B027 diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 540099255..101be44e8 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -759,13 +759,6 @@ def register_table( The registered table can be referenced from SQL statements executed against this context. - Plain :py:class:`~datafusion.dataframe.DataFrame` objects are not supported; - convert them first with :meth:`datafusion.dataframe.DataFrame.into_view` or - :meth:`datafusion.Table.from_dataframe`. - - Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as table provider instances. - Args: name: Name of the resultant table. table: DataFusion :class:`Table` or any object implementing @@ -790,21 +783,16 @@ def register_catalog_provider( else: self.ctx.register_catalog_provider(name, provider) + @deprecated("Use register_table() instead.") def register_table_provider( - self, name: str, provider: Table | TableProviderExportable | Any + self, + name: str, + provider: Table | TableProviderExportable | DataFrame | pa.dataset.Dataset, ) -> None: """Register a table provider. Deprecated: use :meth:`register_table` instead. - - Objects implementing ``__datafusion_table_provider__`` are also supported - and treated as table provider instances. """ - warnings.warn( - "register_table_provider is deprecated; use register_table", - DeprecationWarning, - stacklevel=2, - ) self.register_table(name, provider) def register_udtf(self, func: TableFunction) -> None: From 29b634e584e8fd96793d7b27da19315badd941df Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 13:38:45 -0400 Subject: [PATCH 30/34] More documentation updates --- python/datafusion/dataframe.py | 21 +-------------------- python/tests/test_wrapper_coverage.py | 7 ------- 2 files changed, 1 insertion(+), 27 deletions(-) diff --git a/python/datafusion/dataframe.py b/python/datafusion/dataframe.py index 7834ceffd..615ed8ae1 100644 --- a/python/datafusion/dataframe.py +++ b/python/datafusion/dataframe.py @@ -316,26 +316,7 @@ def __init__(self, df: DataFrameInternal) -> None: self.df = df def into_view(self) -> Table: - """Convert ``DataFrame`` into a :class:`~datafusion.Table` for registration. - - This is the preferred way to obtain a view for - :py:meth:`~datafusion.context.SessionContext.register_table` for several - reasons: - - 1. **Direct API**: Most efficient path - directly calls the underlying Rust - ``DataFrame.into_view()`` method without intermediate delegations. - 2. **Clear semantics**: The ``into_`` prefix follows Rust conventions, - indicating conversion from one type to another. - 3. **Canonical method**: Other approaches like ``Table.from_dataframe`` - delegate to this method internally, making this the single source of truth. - 4. **Deprecated alternatives**: The older ``Table.from_view`` helper - is deprecated and issues warnings when used. - - ``datafusion.Table.from_dataframe`` calls this method under the hood, - and the older ``Table.from_view`` helper is deprecated. - - The ``DataFrame`` remains valid after conversion, so it can still be used for - additional queries alongside the returned view. + """Convert ``DataFrame`` into a :class:`~datafusion.Table`. Examples: >>> from datafusion import SessionContext diff --git a/python/tests/test_wrapper_coverage.py b/python/tests/test_wrapper_coverage.py index 5df454d1d..cc0e029af 100644 --- a/python/tests/test_wrapper_coverage.py +++ b/python/tests/test_wrapper_coverage.py @@ -21,8 +21,6 @@ import datafusion.substrait import pytest -IGNORED_EXPORTS = {"TableProvider"} - # EnumType introduced in 3.11. 3.10 and prior it was called EnumMeta. try: from enum import EnumType @@ -44,8 +42,6 @@ def _check_list_attribute(internal_attr, wrapped_attr) -> None: # We have cases like __all__ that are a list and we want to be certain that # every value in the list in the internal object is also in the wrapper list for val in internal_attr: - if isinstance(val, str) and val in IGNORED_EXPORTS: - continue if isinstance(val, str) and val.startswith("Raw"): assert val[3:] in wrapped_attr else: @@ -74,9 +70,6 @@ def missing_exports(internal_obj, wrapped_obj) -> None: for internal_attr_name in dir(internal_obj): wrapped_attr_name = internal_attr_name.removeprefix("Raw") - if wrapped_attr_name in IGNORED_EXPORTS: - continue - assert wrapped_attr_name in dir(wrapped_obj) internal_attr = getattr(internal_obj, internal_attr_name) From 20099d2174e8da816f29446d2e9319b2cafced31 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 13:44:26 -0400 Subject: [PATCH 31/34] remove cruft --- python/datafusion/context.py | 3 +-- python/tests/test_wrapper_coverage.py | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 101be44e8..86b9d90e2 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -761,8 +761,7 @@ def register_table( Args: name: Name of the resultant table. - table: DataFusion :class:`Table` or any object implementing - ``__datafusion_table_provider__`` to add to the session context. + table: Any object that can be converted into a :class:`Table`. """ self.ctx.register_table(name, table) diff --git a/python/tests/test_wrapper_coverage.py b/python/tests/test_wrapper_coverage.py index cc0e029af..cf6719ecf 100644 --- a/python/tests/test_wrapper_coverage.py +++ b/python/tests/test_wrapper_coverage.py @@ -56,7 +56,6 @@ def missing_exports(internal_obj, wrapped_obj) -> None: - Raw* classes: Internal implementation details that shouldn't be exposed - _global_ctx: Internal implementation detail - __self__, __class__, __repr__: Python special attributes - - TableProvider: Superseded by the public ``Table`` API in Python """ # Special case enums - EnumType overrides a some of the internal functions, # so check all of the values exist and move on From 9964b7fe61a5487a8b2e377be8f563d2ee47351d Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 1 Oct 2025 14:01:57 -0400 Subject: [PATCH 32/34] fix ordering --- python/datafusion/catalog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/datafusion/catalog.py b/python/datafusion/catalog.py index ba95a8716..da54d233d 100644 --- a/python/datafusion/catalog.py +++ b/python/datafusion/catalog.py @@ -172,8 +172,8 @@ def __repr__(self) -> str: """Print a string representation of the table.""" return repr(self._inner) - @deprecated("Use Table() constructor instead.") @staticmethod + @deprecated("Use Table() constructor instead.") def from_dataset(dataset: pa.dataset.Dataset) -> Table: """Turn a :mod:`pyarrow.dataset` ``Dataset`` into a :class:`Table`.""" return Table(dataset) From 81b46cb96bf6c4ff1dfe74a7c09dbe71170c2ec1 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Thu, 2 Oct 2025 07:37:34 -0400 Subject: [PATCH 33/34] give read_table the same treatment --- .../python/tests/_test_table_provider.py | 4 ++++ python/datafusion/context.py | 13 +++++------- python/tests/test_context.py | 21 ++++++++++++++++++- src/context.rs | 3 ++- 4 files changed, 31 insertions(+), 10 deletions(-) diff --git a/examples/datafusion-ffi-example/python/tests/_test_table_provider.py b/examples/datafusion-ffi-example/python/tests/_test_table_provider.py index 91a77e61b..48feaff64 100644 --- a/examples/datafusion-ffi-example/python/tests/_test_table_provider.py +++ b/examples/datafusion-ffi-example/python/tests/_test_table_provider.py @@ -40,3 +40,7 @@ def test_table_loading(): ] assert result == expected + + result = ctx.read_table(table).collect() + result = [r.column(0) for r in result] + assert result == expected diff --git a/python/datafusion/context.py b/python/datafusion/context.py index 86b9d90e2..0aa2f27c4 100644 --- a/python/datafusion/context.py +++ b/python/datafusion/context.py @@ -1177,14 +1177,11 @@ def read_avro( self.ctx.read_avro(str(path), schema, file_partition_cols, file_extension) ) - def read_table(self, table: Table) -> DataFrame: - """Creates a :py:class:`~datafusion.dataframe.DataFrame` from a table. - - For a :py:class:`~datafusion.catalog.Table` such as a - :py:class:`~datafusion.catalog.ListingTable`, create a - :py:class:`~datafusion.dataframe.DataFrame`. - """ - return DataFrame(self.ctx.read_table(table._inner)) + def read_table( + self, table: Table | TableProviderExportable | DataFrame | pa.dataset.Dataset + ) -> DataFrame: + """Creates a :py:class:`~datafusion.dataframe.DataFrame` from a table.""" + return DataFrame(self.ctx.read_table(table)) def execute(self, plan: ExecutionPlan, partitions: int) -> RecordBatchStream: """Execute the ``plan`` and return the results.""" diff --git a/python/tests/test_context.py b/python/tests/test_context.py index 50076c9b6..94d1e6a39 100644 --- a/python/tests/test_context.py +++ b/python/tests/test_context.py @@ -312,7 +312,7 @@ def test_register_table(ctx, database): assert public.names() == {"csv", "csv1", "csv2", "csv3"} -def test_read_table(ctx, database): +def test_read_table_from_catalog(ctx, database): default = ctx.catalog() public = default.schema("public") assert public.names() == {"csv", "csv1", "csv2"} @@ -322,6 +322,25 @@ def test_read_table(ctx, database): table_df.show() +def test_read_table_from_df(ctx): + df = ctx.from_pydict({"a": [1, 2]}) + result = ctx.read_table(df).collect() + assert [b.to_pydict() for b in result] == [{"a": [1, 2]}] + + +def test_read_table_from_dataset(ctx): + batch = pa.RecordBatch.from_arrays( + [pa.array([1, 2, 3]), pa.array([4, 5, 6])], + names=["a", "b"], + ) + dataset = ds.dataset([batch]) + + result = ctx.read_table(dataset).collect() + + assert result[0].column(0) == pa.array([1, 2, 3]) + assert result[0].column(1) == pa.array([4, 5, 6]) + + def test_deregister_table(ctx, database): default = ctx.catalog() public = default.schema("public") diff --git a/src/context.rs b/src/context.rs index 23cf7e548..4a358e4bc 100644 --- a/src/context.rs +++ b/src/context.rs @@ -1076,7 +1076,8 @@ impl PySessionContext { Ok(PyDataFrame::new(df)) } - pub fn read_table(&self, table: &PyTable) -> PyDataFusionResult { + pub fn read_table(&self, table: Bound<'_, PyAny>) -> PyDataFusionResult { + let table = PyTable::new(&table)?; let df = self.ctx.read_table(table.table())?; Ok(PyDataFrame::new(df)) } From 36084a0c36c559a57cd7f9ffaedfce6f078d700c Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Fri, 3 Oct 2025 16:08:24 -0400 Subject: [PATCH 34/34] Reuse Table constructor to idenfity non-ffi tables when using udtf --- python/tests/test_catalog.py | 18 +++++++++++++++++- src/udtf.rs | 9 ++------- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/python/tests/test_catalog.py b/python/tests/test_catalog.py index f0c492ce7..08f494dee 100644 --- a/python/tests/test_catalog.py +++ b/python/tests/test_catalog.py @@ -20,7 +20,7 @@ import pyarrow as pa import pyarrow.dataset as ds import pytest -from datafusion import SessionContext, Table +from datafusion import SessionContext, Table, udtf # Note we take in `database` as a variable even though we don't use @@ -232,3 +232,19 @@ def test_in_end_to_end_python_providers(ctx: SessionContext): assert len(batches) == 1 assert batches[0].column(0) == pa.array([1, 2, 3]) assert batches[0].column(1) == pa.array([4, 5, 6]) + + +def test_register_python_function_as_udtf(ctx: SessionContext): + basic_table = Table(ctx.sql("SELECT 3 AS value")) + + @udtf("my_table_function") + def my_table_function_udtf() -> Table: + return basic_table + + ctx.register_udtf(my_table_function_udtf) + + result = ctx.sql("SELECT * FROM my_table_function()").collect() + assert len(result) == 1 + assert len(result[0]) == 1 + assert len(result[0][0]) == 1 + assert result[0][0][0].as_py() == 3 diff --git a/src/udtf.rs b/src/udtf.rs index 7a1b0aaf0..70a285c62 100644 --- a/src/udtf.rs +++ b/src/udtf.rs @@ -21,12 +21,11 @@ use std::sync::Arc; use crate::errors::{py_datafusion_err, to_datafusion_err}; use crate::expr::PyExpr; use crate::table::PyTable; -use crate::utils::{table_provider_from_pycapsule, validate_pycapsule}; +use crate::utils::validate_pycapsule; use datafusion::catalog::{TableFunctionImpl, TableProvider}; use datafusion::error::Result as DataFusionResult; use datafusion::logical_expr::Expr; use datafusion_ffi::udtf::{FFI_TableFunction, ForeignTableFunction}; -use pyo3::exceptions::PyNotImplementedError; use pyo3::types::{PyCapsule, PyTuple}; /// Represents a user defined table function @@ -98,11 +97,7 @@ fn call_python_table_function( let provider_obj = func.call1(py, py_args)?; let provider = provider_obj.bind(py); - table_provider_from_pycapsule(provider)?.ok_or_else(|| { - PyNotImplementedError::new_err( - "__datafusion_table_provider__ does not exist on Table Provider object.", - ) - }) + Ok::, PyErr>(PyTable::new(provider)?.table) }) .map_err(to_datafusion_err) }