Skip to main content

flow/batching_mode/
frontend_client.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! Frontend client to run flow as batching task which is time-window-aware normal query triggered every tick set by user
16
17use std::collections::HashMap;
18use std::sync::{Arc, Mutex, Weak};
19
20use api::v1::greptime_request::Request;
21use api::v1::query_request::Query;
22use api::v1::{CreateTableExpr, QueryRequest};
23use client::{Client, Database};
24use common_error::ext::BoxedError;
25use common_grpc::channel_manager::{ChannelConfig, ChannelManager, load_client_tls_config};
26use common_meta::peer::{Peer, PeerDiscovery};
27use common_query::Output;
28use common_telemetry::warn;
29use meta_client::client::MetaClient;
30use query::datafusion::QUERY_PARALLELISM_HINT;
31use query::options::QueryOptions;
32use rand::rng;
33use rand::seq::SliceRandom;
34use servers::query_handler::grpc::GrpcQueryHandler;
35use session::context::{QueryContextBuilder, QueryContextRef};
36use session::hints::READ_PREFERENCE_HINT;
37use snafu::{OptionExt, ResultExt};
38use tokio::sync::SetOnce;
39
40use crate::batching_mode::BatchingModeOptions;
41use crate::error::{
42    CreateSinkTableSnafu, ExternalSnafu, InvalidClientConfigSnafu, InvalidRequestSnafu,
43    NoAvailableFrontendSnafu, UnexpectedSnafu,
44};
45use crate::{Error, FlowAuthHeader};
46
47/// Adapter trait for [`GrpcQueryHandler`] that boxes the underlying error into [`BoxedError`].
48///
49/// This is mainly used by flownode to invoke a frontend instance in standalone mode.
50#[async_trait::async_trait]
51pub trait GrpcQueryHandlerWithBoxedError: Send + Sync + 'static {
52    async fn do_query(
53        &self,
54        query: Request,
55        ctx: QueryContextRef,
56    ) -> std::result::Result<Output, BoxedError>;
57}
58
59/// auto impl
60#[async_trait::async_trait]
61impl<T: GrpcQueryHandler + Send + Sync + 'static> GrpcQueryHandlerWithBoxedError for T {
62    async fn do_query(
63        &self,
64        query: Request,
65        ctx: QueryContextRef,
66    ) -> std::result::Result<Output, BoxedError> {
67        self.do_query(query, ctx).await.map_err(BoxedError::new)
68    }
69}
70
71#[derive(Debug, Clone)]
72pub struct HandlerMutable {
73    handler: Arc<Mutex<Option<Weak<dyn GrpcQueryHandlerWithBoxedError>>>>,
74    is_initialized: Arc<SetOnce<()>>,
75}
76
77impl HandlerMutable {
78    pub async fn set_handler(&self, handler: Weak<dyn GrpcQueryHandlerWithBoxedError>) {
79        *self.handler.lock().unwrap() = Some(handler);
80        // Ignore the error, as we allow the handler to be set multiple times.
81        let _ = self.is_initialized.set(());
82    }
83}
84
85/// A simple frontend client able to execute sql using grpc protocol
86///
87/// This is for computation-heavy query which need to offload computation to frontend, lifting the load from flownode
88#[derive(Debug, Clone)]
89pub enum FrontendClient {
90    Distributed {
91        meta_client: Arc<MetaClient>,
92        chnl_mgr: ChannelManager,
93        auth: Option<FlowAuthHeader>,
94        query: QueryOptions,
95        batch_opts: BatchingModeOptions,
96    },
97    Standalone {
98        /// for the sake of simplicity still use grpc even in standalone mode
99        /// notice the client here should all be lazy, so that can wait after frontend is booted then make conn
100        database_client: HandlerMutable,
101        query: QueryOptions,
102    },
103}
104
105impl FrontendClient {
106    /// Create a new empty frontend client, with a `HandlerMutable` to set the grpc handler later
107    pub fn from_empty_grpc_handler(query: QueryOptions) -> (Self, HandlerMutable) {
108        let is_initialized = Arc::new(SetOnce::new());
109        let handler = HandlerMutable {
110            handler: Arc::new(Mutex::new(None)),
111            is_initialized,
112        };
113        (
114            Self::Standalone {
115                database_client: handler.clone(),
116                query,
117            },
118            handler,
119        )
120    }
121
122    /// Waits until the frontend client is initialized.
123    pub async fn wait_initialized(&self) {
124        if let FrontendClient::Standalone {
125            database_client, ..
126        } = self
127        {
128            database_client.is_initialized.wait().await;
129        }
130    }
131
132    pub fn from_meta_client(
133        meta_client: Arc<MetaClient>,
134        auth: Option<FlowAuthHeader>,
135        query: QueryOptions,
136        batch_opts: BatchingModeOptions,
137    ) -> Result<Self, Error> {
138        common_telemetry::info!("Frontend client build with auth={:?}", auth);
139        Ok(Self::Distributed {
140            meta_client,
141            chnl_mgr: {
142                let cfg = ChannelConfig::new()
143                    .connect_timeout(batch_opts.grpc_conn_timeout)
144                    .timeout(Some(batch_opts.query_timeout));
145
146                let tls_config = load_client_tls_config(batch_opts.frontend_tls.clone())
147                    .context(InvalidClientConfigSnafu)?;
148                ChannelManager::with_config(cfg, tls_config)
149            },
150            auth,
151            query,
152            batch_opts,
153        })
154    }
155
156    pub fn from_grpc_handler(
157        grpc_handler: Weak<dyn GrpcQueryHandlerWithBoxedError>,
158        query: QueryOptions,
159    ) -> Self {
160        let is_initialized = Arc::new(SetOnce::new_with(Some(())));
161        let handler = HandlerMutable {
162            handler: Arc::new(Mutex::new(Some(grpc_handler))),
163            is_initialized: is_initialized.clone(),
164        };
165
166        Self::Standalone {
167            database_client: handler,
168            query,
169        }
170    }
171}
172
173#[derive(Debug, Clone)]
174pub struct DatabaseWithPeer {
175    pub database: Database,
176    pub peer: Peer,
177}
178
179impl DatabaseWithPeer {
180    fn new(database: Database, peer: Peer) -> Self {
181        Self { database, peer }
182    }
183
184    /// Try sending a "SELECT 1" to the database
185    async fn try_select_one(&self) -> Result<(), Error> {
186        // notice here use `sql` for `SELECT 1` return 1 row
187        let _ = self
188            .database
189            .sql("SELECT 1")
190            .await
191            .with_context(|_| InvalidRequestSnafu {
192                context: format!("Failed to handle `SELECT 1` request at {:?}", self.peer),
193            })?;
194        Ok(())
195    }
196}
197
198impl FrontendClient {
199    // TODO: support more fine-grained load balancing strategies for frontend
200    // selection, such as AZ (availability zone) awareness, to prefer frontends
201    // in the same zone as the flownode and reduce cross-AZ latency.
202    /// scan for available frontend from metadata
203    pub(crate) async fn scan_for_frontend(&self) -> Result<Vec<Peer>, Error> {
204        let Self::Distributed { meta_client, .. } = self else {
205            return Ok(vec![]);
206        };
207
208        meta_client
209            .active_frontends()
210            .await
211            .map_err(BoxedError::new)
212            .context(ExternalSnafu)
213    }
214
215    /// Get a frontend discovered by metasrv and verified with a query probe.
216    async fn get_random_active_frontend(
217        &self,
218        catalog: &str,
219        schema: &str,
220    ) -> Result<DatabaseWithPeer, Error> {
221        let Self::Distributed {
222            meta_client: _,
223            chnl_mgr,
224            auth,
225            query: _,
226            batch_opts,
227        } = self
228        else {
229            return UnexpectedSnafu {
230                reason: "Expect distributed mode",
231            }
232            .fail();
233        };
234
235        let mut interval = tokio::time::interval(batch_opts.grpc_conn_timeout);
236        interval.tick().await;
237        for retry in 0..batch_opts.experimental_grpc_max_retries {
238            let mut frontends = self.scan_for_frontend().await?;
239            // shuffle the frontends to avoid always pick the same one
240            frontends.shuffle(&mut rng());
241
242            for peer in frontends {
243                let addr = peer.addr.clone();
244                let client = Client::with_manager_and_urls(chnl_mgr.clone(), vec![addr.clone()]);
245                let database = {
246                    let mut db = Database::new(catalog, schema, client);
247                    if let Some(auth) = auth {
248                        db.set_auth(auth.auth().clone());
249                    }
250                    db
251                };
252                let db = DatabaseWithPeer::new(database, peer);
253                match db.try_select_one().await {
254                    Ok(_) => return Ok(db),
255                    Err(e) => {
256                        warn!(
257                            "Failed to connect to frontend {} on retry={}: \n{e:?}",
258                            addr, retry
259                        );
260                    }
261                }
262            }
263            // no available frontend
264            // sleep and retry
265            interval.tick().await;
266        }
267
268        NoAvailableFrontendSnafu {
269            timeout: batch_opts.grpc_conn_timeout,
270            context: "No available frontend found that is able to process query",
271        }
272        .fail()
273    }
274
275    pub async fn create(
276        &self,
277        create: CreateTableExpr,
278        catalog: &str,
279        schema: &str,
280    ) -> Result<u32, Error> {
281        self.handle(
282            Request::Ddl(api::v1::DdlRequest {
283                expr: Some(api::v1::ddl_request::Expr::CreateTable(create.clone())),
284            }),
285            catalog,
286            schema,
287            &mut None,
288        )
289        .await
290        .map_err(BoxedError::new)
291        .with_context(|_| CreateSinkTableSnafu {
292            create: create.clone(),
293        })
294    }
295
296    /// Execute a SQL statement on the frontend.
297    pub async fn sql(&self, catalog: &str, schema: &str, sql: &str) -> Result<Output, Error> {
298        match self {
299            FrontendClient::Distributed { .. } => {
300                let db = self.get_random_active_frontend(catalog, schema).await?;
301                db.database
302                    .sql(sql)
303                    .await
304                    .map_err(BoxedError::new)
305                    .context(ExternalSnafu)
306            }
307            FrontendClient::Standalone {
308                database_client, ..
309            } => {
310                let ctx = QueryContextBuilder::default()
311                    .current_catalog(catalog.to_string())
312                    .current_schema(schema.to_string())
313                    .build();
314                let ctx = Arc::new(ctx);
315                {
316                    let database_client = {
317                        database_client
318                            .handler
319                            .lock()
320                            .unwrap()
321                            .as_ref()
322                            .context(UnexpectedSnafu {
323                                reason: "Standalone's frontend instance is not set",
324                            })?
325                            .upgrade()
326                            .context(UnexpectedSnafu {
327                                reason: "Failed to upgrade database client",
328                            })?
329                    };
330                    let req = Request::Query(QueryRequest {
331                        query: Some(Query::Sql(sql.to_string())),
332                    });
333                    database_client
334                        .do_query(req, ctx)
335                        .await
336                        .map_err(BoxedError::new)
337                        .context(ExternalSnafu)
338                }
339            }
340        }
341    }
342
343    /// Handle a request to frontend
344    pub(crate) async fn handle(
345        &self,
346        req: api::v1::greptime_request::Request,
347        catalog: &str,
348        schema: &str,
349        peer_desc: &mut Option<PeerDesc>,
350    ) -> Result<u32, Error> {
351        match self {
352            FrontendClient::Distributed {
353                query, batch_opts, ..
354            } => {
355                let db = self.get_random_active_frontend(catalog, schema).await?;
356
357                *peer_desc = Some(PeerDesc::Dist {
358                    peer: db.peer.clone(),
359                });
360
361                db.database
362                    .handle_with_retry(
363                        req.clone(),
364                        batch_opts.experimental_grpc_max_retries,
365                        &[
366                            (QUERY_PARALLELISM_HINT, &query.parallelism.to_string()),
367                            (READ_PREFERENCE_HINT, batch_opts.read_preference.as_ref()),
368                        ],
369                    )
370                    .await
371                    .with_context(|_| InvalidRequestSnafu {
372                        context: format!("Failed to handle request at {:?}: {:?}", db.peer, req),
373                    })
374            }
375            FrontendClient::Standalone {
376                database_client,
377                query,
378            } => {
379                let ctx = QueryContextBuilder::default()
380                    .current_catalog(catalog.to_string())
381                    .current_schema(schema.to_string())
382                    .extensions(HashMap::from([(
383                        QUERY_PARALLELISM_HINT.to_string(),
384                        query.parallelism.to_string(),
385                    )]))
386                    .build();
387                let ctx = Arc::new(ctx);
388                {
389                    let database_client = {
390                        database_client
391                            .handler
392                            .lock()
393                            .unwrap()
394                            .as_ref()
395                            .context(UnexpectedSnafu {
396                                reason: "Standalone's frontend instance is not set",
397                            })?
398                            .upgrade()
399                            .context(UnexpectedSnafu {
400                                reason: "Failed to upgrade database client",
401                            })?
402                    };
403                    let resp: common_query::Output = database_client
404                        .do_query(req, ctx)
405                        .await
406                        .map_err(BoxedError::new)
407                        .context(ExternalSnafu)?;
408                    match resp.data {
409                        common_query::OutputData::AffectedRows(rows) => {
410                            Ok(rows.try_into().map_err(|_| {
411                                UnexpectedSnafu {
412                                    reason: format!("Failed to convert rows to u32: {}", rows),
413                                }
414                                .build()
415                            })?)
416                        }
417                        _ => UnexpectedSnafu {
418                            reason: "Unexpected output data",
419                        }
420                        .fail(),
421                    }
422                }
423            }
424        }
425    }
426}
427
428/// Describe a peer of frontend
429#[derive(Debug, Default)]
430pub(crate) enum PeerDesc {
431    /// Distributed mode's frontend peer address
432    Dist {
433        /// frontend peer address
434        peer: Peer,
435    },
436    /// Standalone mode
437    #[default]
438    Standalone,
439}
440
441impl std::fmt::Display for PeerDesc {
442    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
443        match self {
444            PeerDesc::Dist { peer } => write!(f, "{}", peer.addr),
445            PeerDesc::Standalone => write!(f, "standalone"),
446        }
447    }
448}
449
450#[cfg(test)]
451mod tests {
452    use std::time::Duration;
453
454    use common_query::Output;
455    use tokio::time::timeout;
456
457    use super::*;
458
459    #[derive(Debug)]
460    struct NoopHandler;
461
462    #[async_trait::async_trait]
463    impl GrpcQueryHandlerWithBoxedError for NoopHandler {
464        async fn do_query(
465            &self,
466            _query: Request,
467            _ctx: QueryContextRef,
468        ) -> std::result::Result<Output, BoxedError> {
469            Ok(Output::new_with_affected_rows(0))
470        }
471    }
472
473    #[tokio::test]
474    async fn wait_initialized() {
475        let (client, handler_mut) =
476            FrontendClient::from_empty_grpc_handler(QueryOptions::default());
477
478        assert!(
479            timeout(Duration::from_millis(50), client.wait_initialized())
480                .await
481                .is_err()
482        );
483
484        let handler: Arc<dyn GrpcQueryHandlerWithBoxedError> = Arc::new(NoopHandler);
485        handler_mut.set_handler(Arc::downgrade(&handler)).await;
486
487        timeout(Duration::from_secs(1), client.wait_initialized())
488            .await
489            .expect("wait_initialized should complete after handler is set");
490
491        timeout(Duration::from_millis(10), client.wait_initialized())
492            .await
493            .expect("wait_initialized should be a no-op once initialized");
494
495        let handler: Arc<dyn GrpcQueryHandlerWithBoxedError> = Arc::new(NoopHandler);
496        let client =
497            FrontendClient::from_grpc_handler(Arc::downgrade(&handler), QueryOptions::default());
498        assert!(
499            timeout(Duration::from_millis(10), client.wait_initialized())
500                .await
501                .is_ok()
502        );
503
504        let meta_client = Arc::new(MetaClient::new(0, api::v1::meta::Role::Frontend));
505        let client = FrontendClient::from_meta_client(
506            meta_client,
507            None,
508            QueryOptions::default(),
509            BatchingModeOptions::default(),
510        )
511        .unwrap();
512        assert!(
513            timeout(Duration::from_millis(10), client.wait_initialized())
514                .await
515                .is_ok()
516        );
517    }
518}