1use std::collections::HashSet;
16use std::sync::Arc;
17use std::time::Instant;
18
19use itertools::Itertools;
20use pgwire::pg_field_descriptor::PgFieldDescriptor;
21use pgwire::pg_response::{PgResponse, StatementType};
22use pgwire::types::Format;
23use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector;
24use risingwave_common::bail_not_implemented;
25use risingwave_common::catalog::{FunctionId, Schema};
26use risingwave_common::id::ObjectId;
27use risingwave_common::session_config::QueryMode;
28use risingwave_common::types::{DataType, Datum};
29use risingwave_sqlparser::ast::{SetExpr, Statement};
30
31use super::extended_handle::{PortalResult, PrepareStatement, PreparedResult};
32use super::{PgResponseStream, RwPgResponse, create_mv, declare_cursor};
33use crate::binder::{Binder, BoundCreateView, BoundStatement};
34#[cfg(feature = "datafusion")]
35use crate::datafusion::DfBatchQueryPlanResult;
36use crate::error::{ErrorCode, Result, RwError};
37use crate::handler::HandlerArgs;
38use crate::handler::flush::do_flush;
39use crate::handler::util::{DataChunkToRowSetAdapter, to_pg_field};
40use crate::optimizer::plan_node::{BatchPlanRef, Explain};
41use crate::optimizer::{
42 BatchPlanRoot, ExecutionModeDecider, OptimizerContext, OptimizerContextRef,
43 RelationCollectorVisitor, SysTableVisitor,
44};
45use crate::planner::Planner;
46use crate::scheduler::plan_fragmenter::Query;
47use crate::scheduler::{
48 BatchPlanFragmenter, DistributedQueryStream, ExecutionContext, ExecutionContextRef,
49 LocalQueryExecution, LocalQueryStream,
50};
51use crate::session::SessionImpl;
52
53pub enum BatchPlanChoice {
55 Rw(RwBatchQueryPlanResult),
56 #[cfg(feature = "datafusion")]
57 Df(DfBatchQueryPlanResult),
58}
59
60impl BatchPlanChoice {
61 pub fn unwrap_rw(self) -> Result<RwBatchQueryPlanResult> {
62 match self {
63 BatchPlanChoice::Rw(result) => Ok(result),
64 #[cfg(feature = "datafusion")]
65 BatchPlanChoice::Df { .. } => {
66 risingwave_common::bail!(
67 "Expected RisingWave plan in BatchPlanChoice, but got DataFusion plan"
68 )
69 }
70 }
71 }
72}
73
74pub async fn handle_query(
75 handler_args: HandlerArgs,
76 stmt: Statement,
77 formats: Vec<Format>,
78) -> Result<RwPgResponse> {
79 let session = handler_args.session.clone();
80 let context = OptimizerContext::from_handler_args(handler_args);
81
82 #[cfg(feature = "datafusion")]
83 {
84 use futures::FutureExt;
88
89 use crate::datafusion::execute_datafusion_plan;
90
91 let future = match gen_batch_plan_by_statement(&session, context.into(), stmt)? {
92 BatchPlanChoice::Rw(plan_result) => {
93 let plan_fragmenter_result = risingwave_expr::expr_context::TIME_ZONE::sync_scope(
94 session.config().timezone(),
95 || gen_batch_plan_fragmenter(&session, plan_result),
96 )?;
97 execute_risingwave_plan(session, plan_fragmenter_result, formats).left_future()
98 }
99 BatchPlanChoice::Df(plan_result) => {
100 execute_datafusion_plan(session, plan_result, formats).right_future()
101 }
102 };
103 future.await
104 }
105
106 #[cfg(not(feature = "datafusion"))]
107 {
108 let future = match gen_batch_plan_by_statement(&session, context.into(), stmt)? {
109 BatchPlanChoice::Rw(plan_result) => {
110 let plan_fragmenter_result = risingwave_expr::expr_context::TIME_ZONE::sync_scope(
111 session.config().timezone(),
112 || gen_batch_plan_fragmenter(&session, plan_result),
113 )?;
114 execute_risingwave_plan(session, plan_fragmenter_result, formats)
115 }
116 };
117 future.await
118 }
119}
120
121fn handle_parse_inner(binder: Binder, statement: Statement) -> Result<PrepareStatement> {
122 let bound_result = gen_bound(binder, statement.clone())?;
123
124 Ok(PrepareStatement::Prepared(PreparedResult {
125 statement,
126 bound_result,
127 }))
128}
129
130pub fn handle_parse_for_batch(
131 handler_args: HandlerArgs,
132 statement: Statement,
133 specified_param_types: Vec<Option<DataType>>,
134) -> Result<PrepareStatement> {
135 let binder = Binder::new_for_batch(&handler_args.session)
136 .with_specified_params_types(specified_param_types);
137 handle_parse_inner(binder, statement)
138}
139
140pub fn handle_parse_for_stream(
141 handler_args: HandlerArgs,
142 statement: Statement,
143 specified_param_types: Vec<Option<DataType>>,
144) -> Result<PrepareStatement> {
145 let binder = Binder::new_for_stream(&handler_args.session)
146 .with_specified_params_types(specified_param_types);
147 handle_parse_inner(binder, statement)
148}
149
150pub async fn handle_execute(
152 handler_args: HandlerArgs,
153 portal: PortalResult,
154) -> Result<RwPgResponse> {
155 let PortalResult {
156 bound_result,
157 result_formats,
158 statement,
159 } = portal;
160 match statement {
161 Statement::Query(_)
162 | Statement::Insert { .. }
163 | Statement::Delete { .. }
164 | Statement::Update { .. } => {
165 let session = handler_args.session.clone();
167 let plan_fragmenter_result = {
168 let context = OptimizerContext::from_handler_args(handler_args);
169 let plan_result =
170 gen_batch_query_plan(&session, context.into(), bound_result)?.unwrap_rw()?;
171 risingwave_expr::expr_context::TIME_ZONE::sync_scope(
173 session.config().timezone(),
174 || gen_batch_plan_fragmenter(&session, plan_result),
175 )?
176 };
177 execute_risingwave_plan(session, plan_fragmenter_result, result_formats).await
178 }
179 Statement::CreateView { materialized, .. } if materialized => {
180 let BoundResult {
182 bound,
183 dependent_relations,
184 dependent_udfs,
185 ..
186 } = bound_result;
187 let create_mv = if let BoundStatement::CreateView(create_mv) = bound {
188 create_mv
189 } else {
190 unreachable!("expect a BoundStatement::CreateView")
191 };
192 let BoundCreateView {
193 or_replace,
194 materialized: _,
195 if_not_exists,
196 name,
197 columns,
198 query,
199 emit_mode,
200 with_options,
201 } = *create_mv;
202 if or_replace {
203 bail_not_implemented!("CREATE OR REPLACE VIEW");
204 }
205
206 let handler_args = HandlerArgs {
208 session: handler_args.session.clone(),
209 sql: handler_args.sql.clone(),
210 normalized_sql: handler_args.normalized_sql.clone(),
211 with_options: crate::WithOptions::try_from(with_options.as_slice())?,
212 };
213
214 create_mv::handle_create_mv_bound(
215 handler_args,
216 if_not_exists,
217 name,
218 *query,
219 dependent_relations,
220 dependent_udfs,
221 columns,
222 emit_mode,
223 )
224 .await
225 }
226 Statement::DeclareCursor { stmt } => match stmt.declare_cursor {
227 risingwave_sqlparser::ast::DeclareCursor::Query(_) => {
228 let session = handler_args.session.clone();
229 let plan_fragmenter_result = {
230 let context = OptimizerContext::from_handler_args(handler_args.clone());
231 let plan_result = gen_batch_query_plan(&session, context.into(), bound_result)?
232 .unwrap_rw()?;
233 gen_batch_plan_fragmenter(&session, plan_result)?
234 };
235 declare_cursor::handle_bound_declare_query_cursor(
236 handler_args,
237 stmt.cursor_name,
238 plan_fragmenter_result,
239 )
240 .await
241 }
242 risingwave_sqlparser::ast::DeclareCursor::Subscription(sub_name, rw_timestamp) => {
243 declare_cursor::handle_declare_subscription_cursor(
244 handler_args,
245 sub_name,
246 stmt.cursor_name,
247 rw_timestamp,
248 )
249 .await
250 }
251 },
252 _ => unreachable!(),
253 }
254}
255
256pub fn gen_batch_plan_by_statement(
257 session: &SessionImpl,
258 context: OptimizerContextRef,
259 stmt: Statement,
260) -> Result<BatchPlanChoice> {
261 let binder = Binder::new_for_batch(session);
262 let bound_result = gen_bound(binder, stmt)?;
263 gen_batch_query_plan(session, context, bound_result)
264}
265
266#[derive(Clone)]
267pub struct BoundResult {
268 pub(crate) stmt_type: StatementType,
269 pub(crate) must_dist: bool,
270 pub(crate) bound: BoundStatement,
271 pub(crate) param_types: Vec<DataType>,
272 pub(crate) parsed_params: Option<Vec<Datum>>,
273 pub(crate) dependent_relations: HashSet<ObjectId>,
274 pub(crate) dependent_udfs: HashSet<FunctionId>,
276}
277
278fn gen_bound(mut binder: Binder, stmt: Statement) -> Result<BoundResult> {
279 let stmt_type = StatementType::infer_from_statement(&stmt)
280 .map_err(|err| RwError::from(ErrorCode::InvalidInputSyntax(err)))?;
281 let must_dist = must_run_in_distributed_mode(&stmt)?;
282
283 let bound = binder.bind(stmt)?;
284
285 Ok(BoundResult {
286 stmt_type,
287 must_dist,
288 bound,
289 param_types: binder.export_param_types()?,
290 parsed_params: None,
291 dependent_relations: binder.included_relations().clone(),
292 dependent_udfs: binder.included_udfs().clone(),
293 })
294}
295
296pub struct RwBatchQueryPlanResult {
297 pub(crate) plan: BatchPlanRef,
298 pub(crate) query_mode: QueryMode,
299 pub(crate) schema: Schema,
300 pub(crate) stmt_type: StatementType,
301 pub(crate) dependent_relations: Vec<ObjectId>,
305}
306
307fn gen_batch_query_plan(
308 session: &SessionImpl,
309 context: OptimizerContextRef,
310 bind_result: BoundResult,
311) -> Result<BatchPlanChoice> {
312 let BoundResult {
313 stmt_type,
314 must_dist,
315 bound,
316 dependent_relations,
317 ..
318 } = bind_result;
319
320 let mut planner = if matches!(bound, BoundStatement::Query(_)) {
321 Planner::new_for_batch_dql(context)
322 } else {
323 Planner::new_for_batch(context)
324 };
325
326 let logical = planner.plan(bound)?;
327 let schema = logical.schema();
328 let optimized_logical = logical.gen_optimized_logical_plan_for_batch()?;
329
330 #[cfg(feature = "datafusion")]
331 {
332 use crate::optimizer::DatafusionExecuteCheckerExt;
333
334 if session.config().enable_datafusion_engine()
335 && optimized_logical.plan.able_to_run_by_datafusion()
336 {
337 let plan = optimized_logical.gen_datafusion_logical_plan()?;
338 return Ok(BatchPlanChoice::Df(DfBatchQueryPlanResult {
339 plan,
340 schema,
341 stmt_type,
342 }));
343 }
344 }
345
346 let batch_plan = optimized_logical.gen_batch_plan()?;
347
348 let dependent_relations =
349 RelationCollectorVisitor::collect_with(dependent_relations, batch_plan.plan.clone());
350
351 let must_local = must_run_in_local_mode(&batch_plan);
352
353 let query_mode = match (must_dist, must_local) {
354 (true, true) => {
355 return Err(ErrorCode::InternalError(
356 "the query is forced to both local and distributed mode by optimizer".to_owned(),
357 )
358 .into());
359 }
360 (true, false) => QueryMode::Distributed,
361 (false, true) => QueryMode::Local,
362 (false, false) => match session.config().query_mode() {
363 QueryMode::Auto => determine_query_mode(&batch_plan),
364 QueryMode::Local => QueryMode::Local,
365 QueryMode::Distributed => QueryMode::Distributed,
366 },
367 };
368
369 let physical = match query_mode {
370 QueryMode::Auto => unreachable!(),
371 QueryMode::Local => batch_plan.gen_batch_local_plan()?,
372 QueryMode::Distributed => batch_plan.gen_batch_distributed_plan()?,
373 };
374
375 let result = RwBatchQueryPlanResult {
376 plan: physical,
377 query_mode,
378 schema,
379 stmt_type,
380 dependent_relations: dependent_relations.into_iter().collect_vec(),
381 };
382 Ok(BatchPlanChoice::Rw(result))
383}
384
385fn must_run_in_distributed_mode(stmt: &Statement) -> Result<bool> {
386 fn is_insert_using_select(stmt: &Statement) -> bool {
387 fn has_select_query(set_expr: &SetExpr) -> bool {
388 match set_expr {
389 SetExpr::Select(_) => true,
390 SetExpr::Query(query) => has_select_query(&query.body),
391 SetExpr::SetOperation { left, right, .. } => {
392 has_select_query(left) || has_select_query(right)
393 }
394 SetExpr::Values(_) => false,
395 }
396 }
397
398 matches!(
399 stmt,
400 Statement::Insert {source, ..} if has_select_query(&source.body)
401 )
402 }
403
404 let stmt_type = StatementType::infer_from_statement(stmt)
405 .map_err(|err| RwError::from(ErrorCode::InvalidInputSyntax(err)))?;
406
407 Ok(matches!(
408 stmt_type,
409 StatementType::UPDATE
410 | StatementType::DELETE
411 | StatementType::UPDATE_RETURNING
412 | StatementType::DELETE_RETURNING
413 ) | is_insert_using_select(stmt))
414}
415
416fn must_run_in_local_mode(batch_plan: &BatchPlanRoot) -> bool {
417 SysTableVisitor::has_sys_table(batch_plan)
418}
419
420fn determine_query_mode(batch_plan: &BatchPlanRoot) -> QueryMode {
421 if ExecutionModeDecider::run_in_local_mode(batch_plan) {
422 QueryMode::Local
423 } else {
424 QueryMode::Distributed
425 }
426}
427
428pub struct BatchPlanFragmenterResult {
429 pub(crate) plan_fragmenter: BatchPlanFragmenter,
430 pub(crate) query_mode: QueryMode,
431 pub(crate) schema: Schema,
432 pub(crate) stmt_type: StatementType,
433}
434
435pub fn gen_batch_plan_fragmenter(
436 session: &SessionImpl,
437 plan_result: RwBatchQueryPlanResult,
438) -> Result<BatchPlanFragmenterResult> {
439 let RwBatchQueryPlanResult {
440 plan,
441 query_mode,
442 schema,
443 stmt_type,
444 ..
445 } = plan_result;
446
447 tracing::trace!(
448 "Generated query plan: {:?}, query_mode:{:?}",
449 plan.explain_to_string(),
450 query_mode
451 );
452 let worker_node_manager_reader = WorkerNodeSelector::new(
453 session.env().worker_node_manager_ref(),
454 session.is_barrier_read(),
455 );
456 let plan_fragmenter = BatchPlanFragmenter::new(
457 worker_node_manager_reader,
458 session.env().catalog_reader().clone(),
459 session.config().batch_parallelism().0,
460 plan,
461 )?;
462
463 Ok(BatchPlanFragmenterResult {
464 plan_fragmenter,
465 query_mode,
466 schema,
467 stmt_type,
468 })
469}
470
471pub async fn create_stream(
472 session: Arc<SessionImpl>,
473 plan_fragmenter_result: BatchPlanFragmenterResult,
474 formats: Vec<Format>,
475) -> Result<(PgResponseStream, Vec<PgFieldDescriptor>)> {
476 let BatchPlanFragmenterResult {
477 plan_fragmenter,
478 query_mode,
479 schema,
480 stmt_type,
481 ..
482 } = plan_fragmenter_result;
483
484 let mut can_timeout_cancel = true;
485 match stmt_type {
487 StatementType::INSERT
488 | StatementType::INSERT_RETURNING
489 | StatementType::DELETE
490 | StatementType::DELETE_RETURNING
491 | StatementType::UPDATE
492 | StatementType::UPDATE_RETURNING => {
493 session.txn_write_guard()?;
494 can_timeout_cancel = false;
495 }
496 _ => {}
497 }
498
499 let query = plan_fragmenter.generate_complete_query().await?;
500 tracing::trace!("Generated query after plan fragmenter: {:?}", &query);
501
502 let pg_descs = schema
503 .fields()
504 .iter()
505 .map(to_pg_field)
506 .collect::<Vec<PgFieldDescriptor>>();
507 let column_types = schema.fields().iter().map(|f| f.data_type()).collect_vec();
508
509 let row_stream = match query_mode {
510 QueryMode::Auto => unreachable!(),
511 QueryMode::Local => PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new(
512 local_execute(session.clone(), query, can_timeout_cancel).await?,
513 column_types,
514 formats,
515 session.clone(),
516 )),
517 QueryMode::Distributed => {
519 PgResponseStream::DistributedQuery(DataChunkToRowSetAdapter::new(
520 distribute_execute(session.clone(), query, can_timeout_cancel).await?,
521 column_types,
522 formats,
523 session.clone(),
524 ))
525 }
526 };
527
528 Ok((row_stream, pg_descs))
529}
530
531async fn execute_risingwave_plan(
532 session: Arc<SessionImpl>,
533 plan_fragmenter_result: BatchPlanFragmenterResult,
534 formats: Vec<Format>,
535) -> Result<RwPgResponse> {
536 let first_field_format = formats.first().copied().unwrap_or(Format::Text);
538 let query_mode = plan_fragmenter_result.query_mode;
539 let stmt_type = plan_fragmenter_result.stmt_type;
540
541 let query_start_time = Instant::now();
542 let (row_stream, pg_descs) =
543 create_stream(session.clone(), plan_fragmenter_result, formats).await?;
544
545 let callback = async move {
548 if session.config().implicit_flush() && stmt_type.is_dml() {
550 do_flush(&session).await?;
551 }
552
553 match query_mode {
555 QueryMode::Auto => unreachable!(),
556 QueryMode::Local => {
557 session
558 .env()
559 .frontend_metrics
560 .latency_local_execution
561 .observe(query_start_time.elapsed().as_secs_f64());
562
563 session
564 .env()
565 .frontend_metrics
566 .query_counter_local_execution
567 .inc();
568 }
569 QueryMode::Distributed => {
570 session
571 .env()
572 .query_manager()
573 .query_metrics
574 .query_latency
575 .observe(query_start_time.elapsed().as_secs_f64());
576
577 session
578 .env()
579 .query_manager()
580 .query_metrics
581 .completed_query_counter
582 .inc();
583 }
584 }
585
586 Ok(())
587 };
588
589 Ok(PgResponse::builder(stmt_type)
590 .row_cnt_format_opt(Some(first_field_format))
591 .values(row_stream, pg_descs)
592 .callback(callback)
593 .into())
594}
595
596pub async fn distribute_execute(
597 session: Arc<SessionImpl>,
598 query: Query,
599 can_timeout_cancel: bool,
600) -> Result<DistributedQueryStream> {
601 let timeout = if cfg!(madsim) {
602 None
603 } else if can_timeout_cancel {
604 Some(session.statement_timeout())
605 } else {
606 None
607 };
608 let execution_context: ExecutionContextRef =
609 ExecutionContext::new(session.clone(), timeout).into();
610 let query_manager = session.env().query_manager().clone();
611
612 query_manager
613 .schedule(execution_context, query)
614 .await
615 .map_err(|err| err.into())
616}
617
618pub async fn local_execute(
619 session: Arc<SessionImpl>,
620 mut query: Query,
621 can_timeout_cancel: bool,
622) -> Result<LocalQueryStream> {
623 let timeout = if cfg!(madsim) {
624 None
625 } else if can_timeout_cancel {
626 Some(session.statement_timeout())
627 } else {
628 None
629 };
630 let front_env = session.env();
631
632 let snapshot = session.pinned_snapshot();
633
634 snapshot.fill_batch_query_epoch(&mut query)?;
635
636 let execution = LocalQueryExecution::new(
637 query,
638 front_env.clone(),
639 snapshot.support_barrier_read(),
640 session,
641 timeout,
642 );
643
644 Ok(execution.stream_rows())
645}