@@ -19,7 +19,10 @@ use std::sync::Arc;
1919
2020use arrow_flight:: {
2121 decode:: FlightRecordBatchStream ,
22- sql:: { client:: FlightSqlServiceClient , CommandGetDbSchemas , CommandGetTables } ,
22+ sql:: {
23+ client:: FlightSqlServiceClient , CommandGetDbSchemas , CommandGetTables ,
24+ CommandGetXdbcTypeInfo ,
25+ } ,
2326 FlightInfo ,
2427} ;
2528#[ cfg( feature = "flightsql" ) ]
@@ -287,6 +290,62 @@ impl FlightSQLContext {
287290 }
288291 }
289292
293+ pub async fn get_table_types_flight_info ( & self ) -> DFResult < FlightInfo > {
294+ let client = Arc :: clone ( & self . client ) ;
295+ let mut guard = client. lock ( ) . await ;
296+ if let Some ( client) = guard. as_mut ( ) {
297+ client
298+ . get_table_types ( )
299+ . await
300+ . map_err ( |e| DataFusionError :: ArrowError ( Box :: new ( e) , None ) )
301+ } else {
302+ Err ( DataFusionError :: External (
303+ "No FlightSQL client configured. Add one in `~/.config/dft/config.toml`" . into ( ) ,
304+ ) )
305+ }
306+ }
307+
308+ pub async fn get_sql_info_flight_info ( & self , info : Option < Vec < u32 > > ) -> DFResult < FlightInfo > {
309+ let client = Arc :: clone ( & self . client ) ;
310+ let mut guard = client. lock ( ) . await ;
311+ if let Some ( client) = guard. as_mut ( ) {
312+ use arrow_flight:: sql:: SqlInfo ;
313+ // Convert u32 IDs to SqlInfo enum variants if needed
314+ let sql_info_list: Vec < SqlInfo > = info
315+ . unwrap_or_default ( )
316+ . into_iter ( )
317+ . filter_map ( |id| SqlInfo :: try_from ( id as i32 ) . ok ( ) )
318+ . collect ( ) ;
319+ client
320+ . get_sql_info ( sql_info_list)
321+ . await
322+ . map_err ( |e| DataFusionError :: ArrowError ( Box :: new ( e) , None ) )
323+ } else {
324+ Err ( DataFusionError :: External (
325+ "No FlightSQL client configured. Add one in `~/.config/dft/config.toml`" . into ( ) ,
326+ ) )
327+ }
328+ }
329+
330+ pub async fn get_xdbc_type_info_flight_info (
331+ & self ,
332+ data_type : Option < i32 > ,
333+ ) -> DFResult < FlightInfo > {
334+ let client = Arc :: clone ( & self . client ) ;
335+ let mut guard = client. lock ( ) . await ;
336+ if let Some ( client) = guard. as_mut ( ) {
337+ let cmd = CommandGetXdbcTypeInfo { data_type } ;
338+ client
339+ . get_xdbc_type_info ( cmd)
340+ . await
341+ . map_err ( |e| DataFusionError :: ArrowError ( Box :: new ( e) , None ) )
342+ } else {
343+ Err ( DataFusionError :: External (
344+ "No FlightSQL client configured. Add one in `~/.config/dft/config.toml`" . into ( ) ,
345+ ) )
346+ }
347+ }
348+
290349 pub async fn do_get ( & self , flight_info : FlightInfo ) -> DFResult < Vec < FlightRecordBatchStream > > {
291350 let client = Arc :: clone ( & self . client ) ;
292351 let mut guard = client. lock ( ) . await ;
0 commit comments