@@ -21,7 +21,8 @@ use arrow_flight::error::FlightError;
2121use arrow_flight:: flight_service_server:: { FlightService , FlightServiceServer } ;
2222use arrow_flight:: sql:: server:: FlightSqlService ;
2323use arrow_flight:: sql:: {
24- Any , CommandGetCatalogs , CommandStatementQuery , SqlInfo , TicketStatementQuery ,
24+ Any , CommandGetCatalogs , CommandGetDbSchemas , CommandStatementQuery , SqlInfo ,
25+ TicketStatementQuery ,
2526} ;
2627use arrow_flight:: { FlightDescriptor , FlightEndpoint , FlightInfo , Ticket } ;
2728use color_eyre:: Result ;
@@ -90,13 +91,13 @@ impl FlightSqlServiceImpl {
9091 . boxed ( ) ;
9192 Ok ( Response :: new ( flight_data_stream) )
9293 } else {
93- Err ( Status :: internal ( "Plan not found for id" ) )
94+ Err ( Status :: internal ( "plan not found for id" ) )
9495 }
9596 }
9697 Err ( e) => {
9798 error ! ( "error decoding handle to uuid for {request_id}: {:?}" , e) ;
9899 Err ( Status :: internal (
99- "Error decoding handle to uuid for {request_id}" ,
100+ "error decoding handle to uuid for {request_id}" ,
100101 ) )
101102 }
102103 }
@@ -131,20 +132,22 @@ impl FlightSqlServiceImpl {
131132 . try_record_request ( ctx, req)
132133 . await
133134 {
134- error ! ( "Error recording request: {}" , e. to_string( ) )
135+ error ! ( "error recording request: {}" , e. to_string( ) )
135136 }
136137
137138 histogram ! ( latency_metric) . record ( duration. get_milliseconds ( ) as f64 ) ;
138139 }
139140
140- async fn get_flight_info_statement_handler (
141+ async fn create_flight_info (
141142 & self ,
142143 query : String ,
143144 request_id : Uuid ,
144- request : Request < FlightDescriptor > ,
145+ _request : Request < FlightDescriptor > ,
145146 ) -> Result < Response < FlightInfo > , Status > {
146- info ! ( "get_flight_info_statement query: {:?}" , query) ;
147- debug ! ( "get_flight_info_statement request: {:?}" , request) ;
147+ debug ! (
148+ "creating flight info for request id {request_id} with query: {:?}" ,
149+ query
150+ ) ;
148151 let dialect = datafusion:: sql:: sqlparser:: dialect:: GenericDialect { } ;
149152 match DFParser :: parse_sql_with_dialect ( & query, & dialect) {
150153 Ok ( statements) => {
@@ -174,25 +177,25 @@ impl FlightSqlServiceImpl {
174177 debug ! ( "flight info: {:?}" , info) ;
175178
176179 let mut guard = self . requests . lock ( ) . map_err ( |_| {
177- Status :: internal ( "Failed to acquire lock on requests" )
180+ Status :: internal ( "failed to acquire lock on requests" )
178181 } ) ?;
179182 guard. insert ( request_id, logical_plan) ;
180183
181184 Ok ( Response :: new ( info) )
182185 } else {
183186 error ! ( "error encoding ticket" ) ;
184- Err ( Status :: internal ( "Error encoding ticket" ) )
187+ Err ( Status :: internal ( "error encoding ticket" ) )
185188 }
186189 }
187190 Err ( e) => {
188191 error ! ( "error planning SQL query: {:?}" , e) ;
189- Err ( Status :: internal ( "Error planning SQL query" ) )
192+ Err ( Status :: internal ( "error planning SQL query" ) )
190193 }
191194 }
192195 }
193196 Err ( e) => {
194197 error ! ( "error parsing SQL query: {:?}" , e) ;
195- Err ( Status :: internal ( "Error parsing SQL query" ) )
198+ Err ( Status :: internal ( "error parsing SQL query" ) )
196199 }
197200 }
198201 }
@@ -229,9 +232,40 @@ impl FlightSqlService for FlightSqlServiceImpl {
229232 let start = Timestamp :: now ( ) ;
230233 let request_id = uuid:: Uuid :: new_v4 ( ) ;
231234 let query = "SELECT DISTINCT table_catalog FROM information_schema.tables" . to_string ( ) ;
232- let res = self
233- . get_flight_info_statement_handler ( query, request_id, request)
234- . await ;
235+ let res = self . create_flight_info ( query, request_id, request) . await ;
236+
237+ // TODO: Move recording to after response is sent to not impact response latency
238+ self . record_request (
239+ start,
240+ Some ( request_id. to_string ( ) ) ,
241+ res. as_ref ( ) . err ( ) ,
242+ "/get_flight_info_catalogs" . to_string ( ) ,
243+ "get_flight_info_catalogs_latency_ms" ,
244+ )
245+ . await ;
246+ res
247+ }
248+
249+ async fn get_flight_info_schemas (
250+ & self ,
251+ command : CommandGetDbSchemas ,
252+ request : Request < FlightDescriptor > ,
253+ ) -> Result < Response < FlightInfo > , Status > {
254+ counter ! ( "requests" , "endpoint" => "get_flight_info" ) . increment ( 1 ) ;
255+ let start = Timestamp :: now ( ) ;
256+ let request_id = uuid:: Uuid :: new_v4 ( ) ;
257+ let CommandGetDbSchemas {
258+ catalog,
259+ db_schema_filter_pattern,
260+ } = command;
261+ let query = match ( catalog, db_schema_filter_pattern) {
262+ ( Some ( catalog) , Some ( filter) ) => format ! ( "SELECT DISTINCT table_catalog, table_schema FROM information_schema.tables WHERE table_catalog = '{catalog}' AND table_schema ILIKE '%{filter}%' ORDER BY table_catalog, table_schema" ) ,
263+ ( None , Some ( filter) ) => format ! ( "SELECT DISTINCT table_catalog, table_schema FROM information_schema.tables WHERE table_schema ILIKE '%{filter}%' ORDER BY table_catalog, table_schema" ) ,
264+ ( Some ( catalog) , None ) => format ! ( "SELECT DISTINCT table_catalog, table_schema FROM information_schema.tables WHERE table_catalog = '{catalog}' ORDER BY table_catalog, table_schema" ) ,
265+ ( None , None ) => "SELECT DISTINCT table_catalog, table_schema FROM information_schema.tables ORDER BY table_catalog, table_schema" . to_string ( )
266+ } ;
267+ println ! ( "QUERY: {query}" ) ;
268+ let res = self . create_flight_info ( query, request_id, request) . await ;
235269
236270 // TODO: Move recording to after response is sent to not impact response latency
237271 self . record_request (
@@ -255,7 +289,7 @@ impl FlightSqlService for FlightSqlServiceImpl {
255289 let CommandStatementQuery { query, .. } = query;
256290 let request_id = uuid:: Uuid :: new_v4 ( ) ;
257291 let res = self
258- . get_flight_info_statement_handler ( query. clone ( ) , request_id, request)
292+ . create_flight_info ( query. clone ( ) , request_id, request)
259293 . await ;
260294
261295 // TODO: Move recording to after response is sent to not impact response latency
0 commit comments