@@ -36,20 +36,19 @@ use datafusion::error::DataFusionError;
3636use datafusion:: execution:: SendableRecordBatchStream ;
3737use datafusion:: parquet:: basic:: { BrotliLevel , Compression , GzipLevel , ZstdLevel } ;
3838use datafusion:: prelude:: * ;
39- use datafusion_ffi:: table_provider:: FFI_TableProvider ;
4039use futures:: { StreamExt , TryStreamExt } ;
4140use pyo3:: exceptions:: PyValueError ;
4241use pyo3:: prelude:: * ;
4342use pyo3:: pybacked:: PyBackedStr ;
4443use pyo3:: types:: { PyCapsule , PyList , PyTuple , PyTupleMethods } ;
4544use tokio:: task:: JoinHandle ;
4645
47- use crate :: catalog:: PyTable ;
4846use crate :: errors:: { py_datafusion_err, to_datafusion_err, PyDataFusionError } ;
4947use crate :: expr:: sort_expr:: to_sort_expressions;
5048use crate :: physical_plan:: PyExecutionPlan ;
5149use crate :: record_batch:: PyRecordBatchStream ;
5250use crate :: sql:: logical:: PyLogicalPlan ;
51+ use crate :: table:: PyTableProvider ;
5352use crate :: utils:: {
5453 get_tokio_runtime, is_ipython_env, py_obj_to_scalar_value, validate_pycapsule, wait_for_future,
5554} ;
@@ -58,40 +57,6 @@ use crate::{
5857 expr:: { sort_expr:: PySortExpr , PyExpr } ,
5958} ;
6059
61- // https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116
62- // - we have not decided on the table_provider approach yet
63- // this is an interim implementation
64- #[ pyclass( name = "TableProvider" , module = "datafusion" ) ]
65- pub struct PyTableProvider {
66- provider : Arc < dyn TableProvider + Send > ,
67- }
68-
69- impl PyTableProvider {
70- pub fn new ( provider : Arc < dyn TableProvider > ) -> Self {
71- Self { provider }
72- }
73-
74- pub fn as_table ( & self ) -> PyTable {
75- let table_provider: Arc < dyn TableProvider > = self . provider . clone ( ) ;
76- PyTable :: new ( table_provider)
77- }
78- }
79-
80- #[ pymethods]
81- impl PyTableProvider {
82- fn __datafusion_table_provider__ < ' py > (
83- & self ,
84- py : Python < ' py > ,
85- ) -> PyResult < Bound < ' py , PyCapsule > > {
86- let name = CString :: new ( "datafusion_table_provider" ) . unwrap ( ) ;
87-
88- let runtime = get_tokio_runtime ( ) . 0 . handle ( ) . clone ( ) ;
89- let provider = FFI_TableProvider :: new ( Arc :: clone ( & self . provider ) , false , Some ( runtime) ) ;
90-
91- PyCapsule :: new ( py, provider, Some ( name. clone ( ) ) )
92- }
93- }
94-
9560/// Configuration for DataFrame display formatting
9661#[ derive( Debug , Clone ) ]
9762pub struct FormatterConfig {
@@ -303,6 +268,10 @@ impl PyDataFrame {
303268 }
304269 }
305270
271+ pub ( crate ) fn into_view_provider ( & self ) -> Arc < dyn TableProvider > {
272+ self . df . as_ref ( ) . clone ( ) . into_view ( )
273+ }
274+
306275 fn prepare_repr_string ( & mut self , py : Python , as_html : bool ) -> PyDataFusionResult < String > {
307276 // Get the Python formatter and config
308277 let PythonFormatter { formatter, config } = get_python_formatter_with_config ( py) ?;
@@ -436,14 +405,12 @@ impl PyDataFrame {
436405 /// https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116
437406 /// - we have not decided on the table_provider approach yet
438407 #[ allow( clippy:: wrong_self_convention) ]
439- fn into_view ( & self ) -> PyDataFusionResult < PyTable > {
408+ fn into_view ( & self ) -> PyDataFusionResult < PyTableProvider > {
440409 // Call the underlying Rust DataFrame::into_view method.
441410 // Note that the Rust method consumes self; here we clone the inner Arc<DataFrame>
442411 // so that we don’t invalidate this PyDataFrame.
443- let table_provider = self . df . as_ref ( ) . clone ( ) . into_view ( ) ;
444- let table_provider = PyTableProvider :: new ( table_provider) ;
445-
446- Ok ( table_provider. as_table ( ) )
412+ let table_provider = self . into_view_provider ( ) ;
413+ Ok ( PyTableProvider :: new ( table_provider) )
447414 }
448415
449416 #[ pyo3( signature = ( * args) ) ]
0 commit comments