@@ -34,6 +34,7 @@ use datafusion::error::DataFusionError;
3434use datafusion:: execution:: SendableRecordBatchStream ;
3535use datafusion:: parquet:: basic:: { BrotliLevel , Compression , GzipLevel , ZstdLevel } ;
3636use datafusion:: prelude:: * ;
37+ use datafusion_ffi:: table_provider:: FFI_TableProvider ;
3738use futures:: { StreamExt , TryStreamExt } ;
3839use pyo3:: exceptions:: PyValueError ;
3940use pyo3:: prelude:: * ;
@@ -58,7 +59,7 @@ use crate::{
5859// this is an interim implementation
5960#[ pyclass( name = "TableProvider" , module = "datafusion" ) ]
6061pub struct PyTableProvider {
61- provider : Arc < dyn TableProvider > ,
62+ provider : Arc < dyn TableProvider + Send > ,
6263}
6364
6465impl PyTableProvider {
@@ -72,6 +73,21 @@ impl PyTableProvider {
7273 }
7374}
7475
76+ #[ pymethods]
77+ impl PyTableProvider {
78+ fn __datafusion_table_provider__ < ' py > (
79+ & self ,
80+ py : Python < ' py > ,
81+ ) -> PyResult < Bound < ' py , PyCapsule > > {
82+ let name = CString :: new ( "datafusion_table_provider" ) . unwrap ( ) ;
83+
84+ let runtime = get_tokio_runtime ( ) . 0 . handle ( ) . clone ( ) ;
85+ let provider = FFI_TableProvider :: new ( Arc :: clone ( & self . provider ) , false , Some ( runtime) ) ;
86+
87+ PyCapsule :: new ( py, provider, Some ( name. clone ( ) ) )
88+ }
89+ }
90+
7591/// Configuration for DataFrame display formatting
7692#[ derive( Debug , Clone ) ]
7793pub struct FormatterConfig {
0 commit comments