datafusion_dft/tui/
execution.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! [`AppExecution`]: Handles executing queries for the TUI application.

use crate::execution::AppExecution;
use crate::tui::AppEvent;
use color_eyre::eyre::Result;
use datafusion::arrow::array::RecordBatch;
#[allow(unused_imports)] // No idea why this is being picked up as unused when I use it twice.
use datafusion::arrow::error::ArrowError;
use datafusion::execution::context::SessionContext;
use datafusion::execution::SendableRecordBatchStream;
use datafusion::physical_plan::execute_stream;
use futures::StreamExt;
use log::{error, info};
use std::sync::Arc;
use std::time::Duration;
use tokio::sync::mpsc::UnboundedSender;
use tokio::sync::Mutex;
#[cfg(feature = "flightsql")]
use tokio_stream::StreamMap;

#[cfg(feature = "flightsql")]
use {
    arrow_flight::decode::FlightRecordBatchStream,
    arrow_flight::sql::client::FlightSqlServiceClient, arrow_flight::Ticket,
    tonic::transport::Channel, tonic::IntoRequest,
};

#[derive(Clone, Debug)]
pub struct ExecutionError {
    query: String,
    error: String,
    duration: Duration,
}

impl ExecutionError {
    pub fn new(query: String, error: String, duration: Duration) -> Self {
        Self {
            query,
            error,
            duration,
        }
    }

    pub fn query(&self) -> &str {
        &self.query
    }

    pub fn error(&self) -> &str {
        &self.error
    }

    pub fn duration(&self) -> &Duration {
        &self.duration
    }
}

#[derive(Clone, Debug)]
pub struct ExecutionResultsBatch {
    pub query: String,
    pub batch: RecordBatch,
    pub duration: Duration,
}

impl ExecutionResultsBatch {
    pub fn new(query: String, batch: RecordBatch, duration: Duration) -> Self {
        Self {
            query,
            batch,
            duration,
        }
    }

    pub fn query(&self) -> &str {
        &self.query
    }

    pub fn batch(&self) -> &RecordBatch {
        &self.batch
    }

    pub fn duration(&self) -> &Duration {
        &self.duration
    }
}

/// Handles executing queries for the TUI application, formatting results
/// and sending them to the UI.
///
/// TODO: I think we want to store the SQL associated with a stream
pub struct TuiExecution {
    inner: Arc<AppExecution>,
    result_stream: Arc<Mutex<Option<SendableRecordBatchStream>>>,
    /// StreamMao of FlightSQL streams that could be coming from multiple endpoints / tickets.
    /// Often times there is only one but we need to be able to handle multiple.  We should test
    /// this at some point as well.
    #[cfg(feature = "flightsql")]
    flightsql_result_stream: Arc<Mutex<Option<StreamMap<String, FlightRecordBatchStream>>>>,
}

impl TuiExecution {
    /// Create a new instance of [`AppExecution`].
    pub fn new(inner: Arc<AppExecution>) -> Self {
        Self {
            inner,
            result_stream: Arc::new(Mutex::new(None)),
            #[cfg(feature = "flightsql")]
            flightsql_result_stream: Arc::new(Mutex::new(None)),
        }
    }

    pub fn session_ctx(&self) -> &SessionContext {
        self.inner.session_ctx()
    }

    pub async fn set_result_stream(&self, stream: SendableRecordBatchStream) {
        let mut s = self.result_stream.lock().await;
        *s = Some(stream)
    }

    #[cfg(feature = "flightsql")]
    pub async fn set_flightsql_result_stream(
        &self,
        ticket: Ticket,
        stream: FlightRecordBatchStream,
    ) {
        let mut s = self.flightsql_result_stream.lock().await;
        if let Some(ref mut streams) = *s {
            streams.insert(ticket.to_string(), stream);
        } else {
            let mut map: StreamMap<String, FlightRecordBatchStream> = StreamMap::new();
            let t = ticket.to_string();
            info!("Adding {t} to FlightSQL streams");
            map.insert(ticket.to_string(), stream);
            *s = Some(map);
        }
    }

    #[cfg(feature = "flightsql")]
    pub async fn reset_flightsql_result_stream(&self) {
        let mut s = self.flightsql_result_stream.lock().await;
        *s = None;
    }

    /// Run the sequence of SQL queries, sending the results as
    /// [`AppEvent::ExecutionResultsBatch`].
    /// All queries except the last one will have their results discarded.
    ///
    /// Error handling: If an error occurs while executing a query, the error is
    /// logged and execution continues
    pub async fn run_sqls(
        self: Arc<Self>,
        sqls: Vec<String>,
        sender: UnboundedSender<AppEvent>,
    ) -> Result<()> {
        // We need to filter out empty strings to correctly determine the last query for displaying
        // results.
        info!("Running sqls: {:?}", sqls);
        let non_empty_sqls: Vec<String> = sqls.into_iter().filter(|s| !s.is_empty()).collect();
        info!("Non empty SQLs: {:?}", non_empty_sqls);
        let statement_count = non_empty_sqls.len();
        for (i, sql) in non_empty_sqls.into_iter().enumerate() {
            info!("Running query {}", i);
            let _sender = sender.clone();
            let start = std::time::Instant::now();
            if i == statement_count - 1 {
                info!("Executing last query and display results");
                sender.send(AppEvent::NewExecution)?;
                match self.inner.execution_ctx().create_physical_plan(&sql).await {
                    Ok(plan) => match execute_stream(plan, self.inner.session_ctx().task_ctx()) {
                        Ok(stream) => {
                            self.set_result_stream(stream).await;
                            let mut stream = self.result_stream.lock().await;
                            if let Some(s) = stream.as_mut() {
                                if let Some(b) = s.next().await {
                                    match b {
                                        Ok(b) => {
                                            let duration = start.elapsed();
                                            let results = ExecutionResultsBatch {
                                                query: sql.to_string(),
                                                batch: b,
                                                duration,
                                            };
                                            sender.send(AppEvent::ExecutionResultsNextBatch(
                                                results,
                                            ))?;
                                        }
                                        Err(e) => {
                                            error!("Error getting RecordBatch: {:?}", e);
                                        }
                                    }
                                }
                            }
                        }
                        Err(stream_err) => {
                            error!("Error executing stream: {:?}", stream_err);
                            let elapsed = start.elapsed();
                            let e = ExecutionError {
                                query: sql.to_string(),
                                error: stream_err.to_string(),
                                duration: elapsed,
                            };
                            sender.send(AppEvent::ExecutionResultsError(e))?;
                        }
                    },
                    Err(plan_err) => {
                        error!("Error creating physical plan: {:?}", plan_err);
                        let elapsed = start.elapsed();
                        let e = ExecutionError {
                            query: sql.to_string(),
                            error: plan_err.to_string(),
                            duration: elapsed,
                        };
                        sender.send(AppEvent::ExecutionResultsError(e))?;
                    }
                }
            } else {
                match self
                    .inner
                    .execution_ctx()
                    .execute_sql_and_discard_results(&sql)
                    .await
                {
                    Ok(_) => {}
                    Err(e) => {
                        // We only log failed queries, we don't want to stop the execution of the
                        // remaining queries. Perhaps there should be a configuration option for
                        // this though in case the user wants to stop execution on the first error.
                        error!("Error executing {sql}: {:?}", e);
                    }
                }
            }
        }
        Ok(())
    }

    #[cfg(feature = "flightsql")]
    pub async fn run_flightsqls(
        self: Arc<Self>,
        sqls: Vec<String>,
        sender: UnboundedSender<AppEvent>,
    ) -> Result<()> {
        info!("Running sqls: {:?}", sqls);
        self.reset_flightsql_result_stream().await;
        let non_empty_sqls: Vec<String> = sqls.into_iter().filter(|s| !s.is_empty()).collect();
        let statement_count = non_empty_sqls.len();
        for (i, sql) in non_empty_sqls.into_iter().enumerate() {
            let _sender = sender.clone();
            if i == statement_count - 1 {
                info!("Executing last query and display results");
                sender.send(AppEvent::FlightSQLNewExecution)?;
                if let Some(ref mut client) = *self.flightsql_client().lock().await {
                    let start = std::time::Instant::now();
                    match client.execute(sql.clone(), None).await {
                        Ok(flight_info) => {
                            for endpoint in flight_info.endpoint {
                                if let Some(ticket) = endpoint.ticket {
                                    match client.do_get(ticket.clone().into_request()).await {
                                        Ok(stream) => {
                                            self.set_flightsql_result_stream(ticket, stream).await;
                                            if let Some(streams) =
                                                self.flightsql_result_stream.lock().await.as_mut()
                                            {
                                                match streams.next().await {
                                                    Some((ticket, Ok(batch))) => {
                                                        info!("Received batch for {ticket}");
                                                        let duration = start.elapsed();
                                                        let results = ExecutionResultsBatch {
                                                            batch,
                                                            duration,
                                                            query: sql.to_string(),
                                                        };
                                                        sender.send(
                                                            AppEvent::FlightSQLExecutionResultsNextBatch(
                                                                results,
                                                            ),
                                                        )?;
                                                    }
                                                    Some((ticket, Err(e))) => {
                                                        error!(
                                                            "Error executing stream for ticket {ticket}: {:?}",
                                                            e
                                                        );
                                                        let elapsed = start.elapsed();
                                                        let e = ExecutionError {
                                                            query: sql.to_string(),
                                                            error: e.to_string(),
                                                            duration: elapsed,
                                                        };
                                                        sender.send(
                                                            AppEvent::FlightSQLExecutionResultsError(e),
                                                        )?;
                                                    }
                                                    None => {}
                                                }
                                            }
                                        }
                                        Err(e) => {
                                            error!("Error creating result stream: {:?}", e);
                                            if let ArrowError::IpcError(ipc_err) = &e {
                                                if ipc_err.contains("error trying to connect") {
                                                    let e = ExecutionError {
                                                        query: sql.to_string(),
                                                        error: "Error connecting to Flight server"
                                                            .to_string(),
                                                        duration: std::time::Duration::from_secs(0),
                                                    };
                                                    sender.send(
                                                        AppEvent::FlightSQLExecutionResultsError(e),
                                                    )?;
                                                    return Ok(());
                                                }
                                            }

                                            let elapsed = start.elapsed();
                                            let e = ExecutionError {
                                                query: sql.to_string(),
                                                error: e.to_string(),
                                                duration: elapsed,
                                            };
                                            sender.send(
                                                AppEvent::FlightSQLExecutionResultsError(e),
                                            )?;
                                        }
                                    }
                                }
                            }
                        }
                        Err(e) => {
                            error!("Error getting flight info: {:?}", e);
                            if let ArrowError::IpcError(ipc_err) = &e {
                                if ipc_err.contains("error trying to connect") {
                                    let e = ExecutionError {
                                        query: sql.to_string(),
                                        error: "Error connecting to Flight server".to_string(),
                                        duration: std::time::Duration::from_secs(0),
                                    };
                                    sender.send(AppEvent::FlightSQLExecutionResultsError(e))?;
                                    return Ok(());
                                }
                            }
                            let elapsed = start.elapsed();
                            let e = ExecutionError {
                                query: sql.to_string(),
                                error: e.to_string(),
                                duration: elapsed,
                            };
                            sender.send(AppEvent::FlightSQLExecutionResultsError(e))?;
                        }
                    }
                } else {
                    let e = ExecutionError {
                        query: sql.to_string(),
                        error: "No FlightSQL client".to_string(),
                        duration: std::time::Duration::from_secs(0),
                    };
                    sender.send(AppEvent::FlightSQLExecutionResultsError(e))?;
                }
            }
        }

        Ok(())
    }

    pub async fn next_batch(&self, sql: String, sender: UnboundedSender<AppEvent>) {
        let mut stream = self.result_stream.lock().await;
        if let Some(s) = stream.as_mut() {
            let start = std::time::Instant::now();
            if let Some(b) = s.next().await {
                match b {
                    Ok(b) => {
                        let duration = start.elapsed();
                        let results = ExecutionResultsBatch {
                            query: sql,
                            batch: b,
                            duration,
                        };
                        let _ = sender.send(AppEvent::ExecutionResultsNextBatch(results));
                    }
                    Err(e) => {
                        error!("Error getting RecordBatch: {:?}", e);
                    }
                }
            }
        }
    }

    // TODO: Maybe just expose `inner` and use that rather than re-implementing the same
    // functions here.
    #[cfg(feature = "flightsql")]
    pub async fn create_flightsql_client(&self, cli_host: Option<String>) -> Result<()> {
        self.inner.flightsql_ctx().create_client(cli_host).await
    }

    #[cfg(feature = "flightsql")]
    pub fn flightsql_client(&self) -> &Mutex<Option<FlightSqlServiceClient<Channel>>> {
        self.inner.flightsql_client()
    }

    pub fn load_ddl(&self) -> Option<String> {
        self.inner.execution_ctx().load_ddl()
    }

    pub fn save_ddl(&self, ddl: String) {
        self.inner.execution_ctx().save_ddl(ddl)
    }
}