|
17 | 17 |
|
18 | 18 | use std::sync::Arc; |
19 | 19 |
|
20 | | -use arrow_flight::sql::client::FlightSqlServiceClient; |
| 20 | +use arrow_flight::{ |
| 21 | + decode::FlightRecordBatchStream, sql::client::FlightSqlServiceClient, FlightInfo, |
| 22 | +}; |
21 | 23 | #[cfg(feature = "flightsql")] |
22 | 24 | use base64::engine::{general_purpose::STANDARD, Engine as _}; |
23 | 25 | use datafusion::{ |
24 | 26 | error::{DataFusionError, Result as DFResult}, |
25 | 27 | physical_plan::stream::RecordBatchStreamAdapter, |
26 | 28 | sql::parser::DFParser, |
27 | 29 | }; |
28 | | -use log::{error, info, warn}; |
| 30 | +use log::{debug, error, info, warn}; |
29 | 31 |
|
30 | 32 | use color_eyre::eyre::{self, Result}; |
31 | 33 | use tokio::sync::Mutex; |
@@ -203,4 +205,43 @@ impl FlightSQLContext { |
203 | 205 | return Err(DataFusionError::External("Missing client".into())); |
204 | 206 | } |
205 | 207 | } |
| 208 | + |
| 209 | + pub async fn get_catalogs_flight_info(&self) -> DFResult<FlightInfo> { |
| 210 | + let client = self.client.clone(); |
| 211 | + let mut guard = client.lock().await; |
| 212 | + if let Some(client) = guard.as_mut() { |
| 213 | + client |
| 214 | + .get_catalogs() |
| 215 | + .await |
| 216 | + .map_err(|e| DataFusionError::ArrowError(e, None)) |
| 217 | + } else { |
| 218 | + Err(DataFusionError::External( |
| 219 | + "No FlightSQL client configured. Add one in `~/.config/dft/config.toml`".into(), |
| 220 | + )) |
| 221 | + } |
| 222 | + } |
| 223 | + |
| 224 | + pub async fn do_get(&self, flight_info: FlightInfo) -> DFResult<Vec<FlightRecordBatchStream>> { |
| 225 | + let client = self.client.clone(); |
| 226 | + let mut guard = client.lock().await; |
| 227 | + if let Some(client) = guard.as_mut() { |
| 228 | + let mut streams = Vec::new(); |
| 229 | + for endpoint in flight_info.endpoint { |
| 230 | + if let Some(ticket) = endpoint.ticket { |
| 231 | + let stream = client |
| 232 | + .do_get(ticket.into_request()) |
| 233 | + .await |
| 234 | + .map_err(|e| DataFusionError::ArrowError(e, None))?; |
| 235 | + streams.push(stream); |
| 236 | + } else { |
| 237 | + debug!("No ticket for endpoint: {endpoint}"); |
| 238 | + } |
| 239 | + } |
| 240 | + Ok(streams) |
| 241 | + } else { |
| 242 | + Err(DataFusionError::External( |
| 243 | + "No FlightSQL client configured. Add one in `~/.config/dft/config.toml`".into(), |
| 244 | + )) |
| 245 | + } |
| 246 | + } |
206 | 247 | } |
0 commit comments