@@ -60,7 +60,7 @@ use datafusion::prelude::{
60
60
AvroReadOptions , CsvReadOptions , DataFrame , NdJsonReadOptions , ParquetReadOptions ,
61
61
} ;
62
62
use datafusion_common:: ScalarValue ;
63
- use pyo3:: types:: PyTuple ;
63
+ use pyo3:: types:: { PyDict , PyList , PyTuple } ;
64
64
use tokio:: task:: JoinHandle ;
65
65
66
66
/// Configuration options for a SessionContext
@@ -291,24 +291,17 @@ impl PySessionContext {
291
291
pub fn register_object_store (
292
292
& mut self ,
293
293
scheme : & str ,
294
- store : & Bound < ' _ , PyAny > ,
294
+ store : StorageContexts ,
295
295
host : Option < & str > ,
296
296
) -> PyResult < ( ) > {
297
- let res: Result < ( Arc < dyn ObjectStore > , String ) , PyErr > =
298
- match StorageContexts :: extract_bound ( store) {
299
- Ok ( store) => match store {
300
- StorageContexts :: AmazonS3 ( s3) => Ok ( ( s3. inner , s3. bucket_name ) ) ,
301
- StorageContexts :: GoogleCloudStorage ( gcs) => Ok ( ( gcs. inner , gcs. bucket_name ) ) ,
302
- StorageContexts :: MicrosoftAzure ( azure) => {
303
- Ok ( ( azure. inner , azure. container_name ) )
304
- }
305
- StorageContexts :: LocalFileSystem ( local) => Ok ( ( local. inner , "" . to_string ( ) ) ) ,
306
- } ,
307
- Err ( _e) => Err ( PyValueError :: new_err ( "Invalid object store" ) ) ,
308
- } ;
309
-
310
297
// for most stores the "host" is the bucket name and can be inferred from the store
311
- let ( store, upstream_host) = res?;
298
+ let ( store, upstream_host) : ( Arc < dyn ObjectStore > , String ) = match store {
299
+ StorageContexts :: AmazonS3 ( s3) => ( s3. inner , s3. bucket_name ) ,
300
+ StorageContexts :: GoogleCloudStorage ( gcs) => ( gcs. inner , gcs. bucket_name ) ,
301
+ StorageContexts :: MicrosoftAzure ( azure) => ( azure. inner , azure. container_name ) ,
302
+ StorageContexts :: LocalFileSystem ( local) => ( local. inner , "" . to_string ( ) ) ,
303
+ } ;
304
+
312
305
// let users override the host to match the api signature from upstream
313
306
let derived_host = if let Some ( host) = host {
314
307
host
@@ -434,105 +427,96 @@ impl PySessionContext {
434
427
}
435
428
436
429
/// Construct datafusion dataframe from Python list
437
- #[ allow( clippy:: wrong_self_convention) ]
438
430
pub fn from_pylist (
439
431
& mut self ,
440
- data : PyObject ,
432
+ data : Bound < ' _ , PyList > ,
441
433
name : Option < & str > ,
442
- _py : Python ,
443
434
) -> PyResult < PyDataFrame > {
444
- Python :: with_gil ( |py| {
445
- // Instantiate pyarrow Table object & convert to Arrow Table
446
- let table_class = py. import_bound ( "pyarrow" ) ?. getattr ( "Table" ) ?;
447
- let args = PyTuple :: new_bound ( py, & [ data] ) ;
448
- let table = table_class. call_method1 ( "from_pylist" , args) ?. into ( ) ;
449
-
450
- // Convert Arrow Table to datafusion DataFrame
451
- let df = self . from_arrow_table ( table, name, py) ?;
452
- Ok ( df)
453
- } )
435
+ // Acquire GIL Token
436
+ let py = data. py ( ) ;
437
+
438
+ // Instantiate pyarrow Table object & convert to Arrow Table
439
+ let table_class = py. import_bound ( "pyarrow" ) ?. getattr ( "Table" ) ?;
440
+ let args = PyTuple :: new_bound ( py, & [ data] ) ;
441
+ let table = table_class. call_method1 ( "from_pylist" , args) ?;
442
+
443
+ // Convert Arrow Table to datafusion DataFrame
444
+ let df = self . from_arrow_table ( table, name, py) ?;
445
+ Ok ( df)
454
446
}
455
447
456
448
/// Construct datafusion dataframe from Python dictionary
457
- #[ allow( clippy:: wrong_self_convention) ]
458
449
pub fn from_pydict (
459
450
& mut self ,
460
- data : PyObject ,
451
+ data : Bound < ' _ , PyDict > ,
461
452
name : Option < & str > ,
462
- _py : Python ,
463
453
) -> PyResult < PyDataFrame > {
464
- Python :: with_gil ( |py| {
465
- // Instantiate pyarrow Table object & convert to Arrow Table
466
- let table_class = py. import_bound ( "pyarrow" ) ?. getattr ( "Table" ) ?;
467
- let args = PyTuple :: new_bound ( py, & [ data] ) ;
468
- let table = table_class. call_method1 ( "from_pydict" , args) ?. into ( ) ;
469
-
470
- // Convert Arrow Table to datafusion DataFrame
471
- let df = self . from_arrow_table ( table, name, py) ?;
472
- Ok ( df)
473
- } )
454
+ // Acquire GIL Token
455
+ let py = data. py ( ) ;
456
+
457
+ // Instantiate pyarrow Table object & convert to Arrow Table
458
+ let table_class = py. import_bound ( "pyarrow" ) ?. getattr ( "Table" ) ?;
459
+ let args = PyTuple :: new_bound ( py, & [ data] ) ;
460
+ let table = table_class. call_method1 ( "from_pydict" , args) ?;
461
+
462
+ // Convert Arrow Table to datafusion DataFrame
463
+ let df = self . from_arrow_table ( table, name, py) ?;
464
+ Ok ( df)
474
465
}
475
466
476
467
/// Construct datafusion dataframe from Arrow Table
477
- #[ allow( clippy:: wrong_self_convention) ]
478
468
pub fn from_arrow_table (
479
469
& mut self ,
480
- data : PyObject ,
470
+ data : Bound < ' _ , PyAny > ,
481
471
name : Option < & str > ,
482
- _py : Python ,
472
+ py : Python ,
483
473
) -> PyResult < PyDataFrame > {
484
- Python :: with_gil ( |py| {
485
- // Instantiate pyarrow Table object & convert to batches
486
- let table = data. call_method0 ( py, "to_batches" ) ?;
487
-
488
- let schema = data. getattr ( py, "schema" ) ?;
489
- let schema = schema. extract :: < PyArrowType < Schema > > ( py) ?;
490
-
491
- // Cast PyObject to RecordBatch type
492
- // Because create_dataframe() expects a vector of vectors of record batches
493
- // here we need to wrap the vector of record batches in an additional vector
494
- let batches = table. extract :: < PyArrowType < Vec < RecordBatch > > > ( py) ?;
495
- let list_of_batches = PyArrowType :: from ( vec ! [ batches. 0 ] ) ;
496
- self . create_dataframe ( list_of_batches, name, Some ( schema) , py)
497
- } )
474
+ // Instantiate pyarrow Table object & convert to batches
475
+ let table = data. call_method0 ( "to_batches" ) ?;
476
+
477
+ let schema = data. getattr ( "schema" ) ?;
478
+ let schema = schema. extract :: < PyArrowType < Schema > > ( ) ?;
479
+
480
+ // Cast PyAny to RecordBatch type
481
+ // Because create_dataframe() expects a vector of vectors of record batches
482
+ // here we need to wrap the vector of record batches in an additional vector
483
+ let batches = table. extract :: < PyArrowType < Vec < RecordBatch > > > ( ) ?;
484
+ let list_of_batches = PyArrowType :: from ( vec ! [ batches. 0 ] ) ;
485
+ self . create_dataframe ( list_of_batches, name, Some ( schema) , py)
498
486
}
499
487
500
488
/// Construct datafusion dataframe from pandas
501
489
#[ allow( clippy:: wrong_self_convention) ]
502
490
pub fn from_pandas (
503
491
& mut self ,
504
- data : PyObject ,
492
+ data : Bound < ' _ , PyAny > ,
505
493
name : Option < & str > ,
506
- _py : Python ,
507
494
) -> PyResult < PyDataFrame > {
508
- Python :: with_gil ( |py| {
509
- // Instantiate pyarrow Table object & convert to Arrow Table
510
- let table_class = py. import_bound ( "pyarrow" ) ?. getattr ( "Table" ) ?;
511
- let args = PyTuple :: new_bound ( py, & [ data] ) ;
512
- let table = table_class. call_method1 ( "from_pandas" , args) ?. into ( ) ;
513
-
514
- // Convert Arrow Table to datafusion DataFrame
515
- let df = self . from_arrow_table ( table, name, py) ?;
516
- Ok ( df)
517
- } )
495
+ // Obtain GIL token
496
+ let py = data. py ( ) ;
497
+
498
+ // Instantiate pyarrow Table object & convert to Arrow Table
499
+ let table_class = py. import_bound ( "pyarrow" ) ?. getattr ( "Table" ) ?;
500
+ let args = PyTuple :: new_bound ( py, & [ data] ) ;
501
+ let table = table_class. call_method1 ( "from_pandas" , args) ?;
502
+
503
+ // Convert Arrow Table to datafusion DataFrame
504
+ let df = self . from_arrow_table ( table, name, py) ?;
505
+ Ok ( df)
518
506
}
519
507
520
508
/// Construct datafusion dataframe from polars
521
- #[ allow( clippy:: wrong_self_convention) ]
522
509
pub fn from_polars (
523
510
& mut self ,
524
- data : PyObject ,
511
+ data : Bound < ' _ , PyAny > ,
525
512
name : Option < & str > ,
526
- _py : Python ,
527
513
) -> PyResult < PyDataFrame > {
528
- Python :: with_gil ( |py| {
529
- // Convert Polars dataframe to Arrow Table
530
- let table = data. call_method0 ( py, "to_arrow" ) ?;
514
+ // Convert Polars dataframe to Arrow Table
515
+ let table = data. call_method0 ( "to_arrow" ) ?;
531
516
532
- // Convert Arrow Table to datafusion DataFrame
533
- let df = self . from_arrow_table ( table, name, py) ?;
534
- Ok ( df)
535
- } )
517
+ // Convert Arrow Table to datafusion DataFrame
518
+ let df = self . from_arrow_table ( table, name, data. py ( ) ) ?;
519
+ Ok ( df)
536
520
}
537
521
538
522
pub fn register_table ( & mut self , name : & str , table : & PyTable ) -> PyResult < ( ) > {
0 commit comments