@@ -21,14 +21,14 @@ use std::sync::Arc;
2121
2222use arrow:: array:: { new_null_array, RecordBatch , RecordBatchIterator , RecordBatchReader } ;
2323use arrow:: compute:: can_cast_types;
24+ use arrow:: datatypes:: Field ;
2425use arrow:: error:: ArrowError ;
2526use arrow:: ffi:: FFI_ArrowSchema ;
2627use arrow:: ffi_stream:: FFI_ArrowArrayStream ;
27- use arrow:: pyarrow:: FromPyArrow ;
2828use datafusion:: arrow:: datatypes:: Schema ;
29- use datafusion:: arrow:: pyarrow:: { PyArrowType , ToPyArrow } ;
29+ use datafusion:: arrow:: pyarrow:: { FromPyArrow , PyArrowType , ToPyArrow } ;
3030use datafusion:: arrow:: util:: pretty;
31- use datafusion:: common:: UnnestOptions ;
31+ use datafusion:: common:: { TableReference , UnnestOptions } ;
3232use datafusion:: config:: { CsvOptions , ParquetColumnOptions , ParquetOptions , TableParquetOptions } ;
3333use datafusion:: dataframe:: { DataFrame , DataFrameWriteOptions } ;
3434use datafusion:: datasource:: TableProvider ;
@@ -45,6 +45,7 @@ use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods};
4545use tokio:: task:: JoinHandle ;
4646
4747use crate :: catalog:: PyTable ;
48+ use crate :: common:: table_reference:: PyTableReference ;
4849use crate :: errors:: { py_datafusion_err, to_datafusion_err, PyDataFusionError } ;
4950use crate :: expr:: sort_expr:: to_sort_expressions;
5051use crate :: physical_plan:: PyExecutionPlan ;
@@ -58,6 +59,81 @@ use crate::{
5859 expr:: { sort_expr:: PySortExpr , PyExpr } ,
5960} ;
6061
62+ /// A wrapper around Arc<str> that implements PyO3 traits for easier Python interop
63+ #[ derive( Clone , Debug , PartialEq , Eq , Hash ) ]
64+ pub struct PyArcStr ( Arc < str > ) ;
65+
66+ impl PyArcStr {
67+ pub fn new ( s : & str ) -> Self {
68+ Self ( Arc :: from ( s) )
69+ }
70+
71+ pub fn as_str ( & self ) -> & str {
72+ & self . 0
73+ }
74+
75+ pub fn into_arc ( self ) -> Arc < str > {
76+ self . 0
77+ }
78+ }
79+
80+ impl From < Arc < str > > for PyArcStr {
81+ fn from ( arc : Arc < str > ) -> Self {
82+ Self ( arc)
83+ }
84+ }
85+
86+ impl From < PyArcStr > for Arc < str > {
87+ fn from ( py_arc : PyArcStr ) -> Self {
88+ py_arc. 0
89+ }
90+ }
91+
92+ impl From < & str > for PyArcStr {
93+ fn from ( s : & str ) -> Self {
94+ Self :: new ( s)
95+ }
96+ }
97+
98+ impl From < String > for PyArcStr {
99+ fn from ( s : String ) -> Self {
100+ Self ( Arc :: from ( s) )
101+ }
102+ }
103+
104+ impl std:: fmt:: Display for PyArcStr {
105+ fn fmt ( & self , f : & mut std:: fmt:: Formatter < ' _ > ) -> std:: fmt:: Result {
106+ self . 0 . fmt ( f)
107+ }
108+ }
109+
110+ impl < ' py > pyo3:: IntoPyObject < ' py > for PyArcStr {
111+ type Target = pyo3:: types:: PyString ;
112+ type Output = pyo3:: Bound < ' py , Self :: Target > ;
113+ type Error = std:: convert:: Infallible ;
114+
115+ fn into_pyobject ( self , py : pyo3:: Python < ' py > ) -> Result < Self :: Output , Self :: Error > {
116+ Ok ( pyo3:: types:: PyString :: new ( py, & self . 0 ) )
117+ }
118+ }
119+
120+ impl < ' py > pyo3:: IntoPyObject < ' py > for & PyArcStr {
121+ type Target = pyo3:: types:: PyString ;
122+ type Output = pyo3:: Bound < ' py , Self :: Target > ;
123+ type Error = std:: convert:: Infallible ;
124+
125+ fn into_pyobject ( self , py : pyo3:: Python < ' py > ) -> Result < Self :: Output , Self :: Error > {
126+ Ok ( pyo3:: types:: PyString :: new ( py, & self . 0 ) )
127+ }
128+ }
129+
130+ impl < ' py > pyo3:: FromPyObject < ' py > for PyArcStr {
131+ fn extract_bound ( ob : & pyo3:: Bound < ' py , pyo3:: PyAny > ) -> pyo3:: PyResult < Self > {
132+ let s: String = ob. extract ( ) ?;
133+ Ok ( Self :: new ( & s) )
134+ }
135+ }
136+
61137// https://github.com/apache/datafusion-python/pull/1016#discussion_r1983239116
62138// - we have not decided on the table_provider approach yet
63139// this is an interim implementation
@@ -428,6 +504,17 @@ impl PyDataFrame {
428504 PyArrowType ( self . df . schema ( ) . into ( ) )
429505 }
430506
507+ fn fully_qualified_name ( & self , col : & str ) -> PyResult < ( PyTableReference , PyArrowType < Field > ) > {
508+ let result = self . df . schema ( ) . qualified_field_with_unqualified_name ( col) ;
509+ match result {
510+ Ok ( parts) => Ok ( ( parts. 0 . unwrap ( ) . clone ( ) . into ( ) , parts. 1 . clone ( ) . into ( ) ) ) ,
511+ Err ( err) => Err ( PyValueError :: new_err ( format ! (
512+ "Error: {:?}" ,
513+ err. to_string( )
514+ ) ) ) ,
515+ }
516+ }
517+
431518 /// Convert this DataFrame into a Table that can be used in register_table
432519 /// By convention, into_... methods consume self and return the new object.
433520 /// Disabling the clippy lint, so we can use &self
@@ -467,6 +554,20 @@ impl PyDataFrame {
467554 Ok ( Self :: new ( df) )
468555 }
469556
557+ #[ pyo3( signature = ( * _args) ) ]
558+ fn _drop (
559+ & self ,
560+ _args : Vec < ( PyTableReference , PyArrowType < Field > ) > ,
561+ ) -> PyDataFusionResult < Self > {
562+ // TODO need to finish plumbing through
563+ let cols = _args
564+ . iter ( )
565+ . map ( |( table, s) | ( Some ( table. clone ( ) . into ( ) ) , s. 0 . clone ( ) ) )
566+ . collect :: < Vec < ( Option < TableReference > , Field ) > > ( ) ;
567+ let df = self . df . as_ref ( ) . clone ( ) . drop_qualified_columns ( & cols) ?;
568+ Ok ( Self :: new ( df) )
569+ }
570+
470571 fn filter ( & self , predicate : PyExpr ) -> PyDataFusionResult < Self > {
471572 let df = self . df . as_ref ( ) . clone ( ) . filter ( predicate. into ( ) ) ?;
472573 Ok ( Self :: new ( df) )
0 commit comments