@@ -34,7 +34,7 @@ use pyo3::prelude::*;
3434use crate :: catalog:: { PyCatalog , PyTable } ;
3535use crate :: dataframe:: PyDataFrame ;
3636use crate :: dataset:: Dataset ;
37- use crate :: errors:: { py_datafusion_err, PyDataFusionResult } ;
37+ use crate :: errors:: { py_datafusion_err, PyDataFusionError , PyDataFusionResult } ;
3838use crate :: expr:: sort_expr:: PySortExpr ;
3939use crate :: physical_plan:: PyExecutionPlan ;
4040use crate :: record_batch:: PyRecordBatchStream ;
@@ -375,7 +375,7 @@ impl PySessionContext {
375375 None => {
376376 let state = self . ctx . state ( ) ;
377377 let schema = options. infer_schema ( & state, & table_path) ;
378- wait_for_future ( py, schema) ?
378+ wait_for_future ( py, schema) ?. map_err ( PyDataFusionError :: from ) ?
379379 }
380380 } ;
381381 let config = ListingTableConfig :: new ( table_path)
@@ -400,7 +400,7 @@ impl PySessionContext {
400400 /// Returns a PyDataFrame whose plan corresponds to the SQL statement.
401401 pub fn sql ( & mut self , query : & str , py : Python ) -> PyDataFusionResult < PyDataFrame > {
402402 let result = self . ctx . sql ( query) ;
403- let df = wait_for_future ( py, result) ?;
403+ let df = wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ? ;
404404 Ok ( PyDataFrame :: new ( df) )
405405 }
406406
@@ -417,7 +417,7 @@ impl PySessionContext {
417417 SQLOptions :: new ( )
418418 } ;
419419 let result = self . ctx . sql_with_options ( query, options) ;
420- let df = wait_for_future ( py, result) ?;
420+ let df = wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ? ;
421421 Ok ( PyDataFrame :: new ( df) )
422422 }
423423
@@ -451,7 +451,8 @@ impl PySessionContext {
451451
452452 self . ctx . register_table ( & * table_name, Arc :: new ( table) ) ?;
453453
454- let table = wait_for_future ( py, self . _table ( & table_name) ) ?;
454+ let table =
455+ wait_for_future ( py, self . _table ( & table_name) ) ?. map_err ( PyDataFusionError :: from) ?;
455456
456457 let df = PyDataFrame :: new ( table) ;
457458 Ok ( df)
@@ -826,6 +827,7 @@ impl PySessionContext {
826827
827828 pub fn table ( & self , name : & str , py : Python ) -> PyResult < PyDataFrame > {
828829 let x = wait_for_future ( py, self . ctx . table ( name) )
830+ . map_err ( |e| PyKeyError :: new_err ( e. to_string ( ) ) ) ?
829831 . map_err ( |e| PyKeyError :: new_err ( e. to_string ( ) ) ) ?;
830832 Ok ( PyDataFrame :: new ( x) )
831833 }
@@ -865,10 +867,10 @@ impl PySessionContext {
865867 let df = if let Some ( schema) = schema {
866868 options. schema = Some ( & schema. 0 ) ;
867869 let result = self . ctx . read_json ( path, options) ;
868- wait_for_future ( py, result) ?
870+ wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ?
869871 } else {
870872 let result = self . ctx . read_json ( path, options) ;
871- wait_for_future ( py, result) ?
873+ wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ?
872874 } ;
873875 Ok ( PyDataFrame :: new ( df) )
874876 }
@@ -915,13 +917,13 @@ impl PySessionContext {
915917 let paths = path. extract :: < Vec < String > > ( ) ?;
916918 let paths = paths. iter ( ) . map ( |p| p as & str ) . collect :: < Vec < & str > > ( ) ;
917919 let result = self . ctx . read_csv ( paths, options) ;
918- let df = PyDataFrame :: new ( wait_for_future ( py, result) ?) ;
919- Ok ( df )
920+ let df = wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ? ;
921+ Ok ( PyDataFrame :: new ( df ) )
920922 } else {
921923 let path = path. extract :: < String > ( ) ?;
922924 let result = self . ctx . read_csv ( path, options) ;
923- let df = PyDataFrame :: new ( wait_for_future ( py, result) ?) ;
924- Ok ( df )
925+ let df = wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ? ;
926+ Ok ( PyDataFrame :: new ( df ) )
925927 }
926928 }
927929
@@ -958,7 +960,7 @@ impl PySessionContext {
958960 . collect ( ) ;
959961
960962 let result = self . ctx . read_parquet ( path, options) ;
961- let df = PyDataFrame :: new ( wait_for_future ( py, result) ?) ;
963+ let df = PyDataFrame :: new ( wait_for_future ( py, result) ?. map_err ( PyDataFusionError :: from ) ? ) ;
962964 Ok ( df)
963965 }
964966
@@ -978,10 +980,10 @@ impl PySessionContext {
978980 let df = if let Some ( schema) = schema {
979981 options. schema = Some ( & schema. 0 ) ;
980982 let read_future = self . ctx . read_avro ( path, options) ;
981- wait_for_future ( py, read_future) ?
983+ wait_for_future ( py, read_future) ?. map_err ( PyDataFusionError :: from ) ?
982984 } else {
983985 let read_future = self . ctx . read_avro ( path, options) ;
984- wait_for_future ( py, read_future) ?
986+ wait_for_future ( py, read_future) ?. map_err ( PyDataFusionError :: from ) ?
985987 } ;
986988 Ok ( PyDataFrame :: new ( df) )
987989 }
@@ -1021,8 +1023,10 @@ impl PySessionContext {
10211023 let plan = plan. plan . clone ( ) ;
10221024 let fut: JoinHandle < datafusion:: common:: Result < SendableRecordBatchStream > > =
10231025 rt. spawn ( async move { plan. execute ( part, Arc :: new ( ctx) ) } ) ;
1024- let stream = wait_for_future ( py, fut) . map_err ( py_datafusion_err) ?;
1025- Ok ( PyRecordBatchStream :: new ( stream?) )
1026+ let stream = wait_for_future ( py, fut)
1027+ . map_err ( py_datafusion_err) ?
1028+ . map_err ( PyDataFusionError :: from) ?;
1029+ Ok ( PyRecordBatchStream :: new ( stream) )
10261030 }
10271031}
10281032
0 commit comments