mz_adapter/frontend_peek.rs
1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10use std::collections::BTreeMap;
11use std::collections::BTreeSet;
12use std::sync::Arc;
13
14use itertools::{Either, Itertools};
15use mz_adapter_types::dyncfgs::CONSTRAINT_BASED_TIMESTAMP_SELECTION;
16use mz_adapter_types::timestamp_selection::ConstraintBasedTimestampSelection;
17use mz_compute_types::ComputeInstanceId;
18use mz_controller_types::ClusterId;
19use mz_expr::{CollectionPlan, ResultSpec};
20use mz_ore::cast::{CastFrom, CastLossy};
21use mz_ore::collections::CollectionExt;
22use mz_ore::now::EpochMillis;
23use mz_ore::{soft_assert_eq_or_log, soft_assert_or_log, soft_panic_or_log};
24use mz_repr::optimize::{OptimizerFeatures, OverrideFrom};
25use mz_repr::role_id::RoleId;
26use mz_repr::{Datum, GlobalId, IntoRowIterator, Timestamp};
27use mz_sql::ast::Raw;
28use mz_sql::catalog::CatalogCluster;
29use mz_sql::plan::Params;
30use mz_sql::plan::{self, Explainee, ExplaineeStatement, Plan, QueryWhen};
31use mz_sql::rbac;
32use mz_sql::session::metadata::SessionMetadata;
33use mz_sql::session::vars::IsolationLevel;
34use mz_sql_parser::ast::{CopyDirection, CopyRelation, ExplainStage, ShowStatement, Statement};
35use mz_transform::EmptyStatisticsOracle;
36use mz_transform::dataflow::DataflowMetainfo;
37use opentelemetry::trace::TraceContextExt;
38use tracing::{Span, debug, warn};
39use tracing_opentelemetry::OpenTelemetrySpanExt;
40
41use crate::catalog::{Catalog, CatalogState};
42use crate::command::Command;
43use crate::coord::peek::{FastPathPlan, PeekPlan};
44use crate::coord::sequencer::{eval_copy_to_uri, statistics_oracle};
45use crate::coord::timeline::timedomain_for;
46use crate::coord::timestamp_selection::TimestampDetermination;
47use crate::coord::{
48 Coordinator, CopyToContext, ExecuteContextGuard, ExplainContext, ExplainPlanContext,
49 TargetCluster,
50};
51use crate::explain::insights::PlanInsightsContext;
52use crate::explain::optimizer_trace::OptimizerTrace;
53use crate::optimize::dataflows::{ComputeInstanceSnapshot, DataflowBuilder};
54use crate::optimize::{Optimize, OptimizerError};
55use crate::session::{Session, TransactionOps, TransactionStatus};
56use crate::statement_logging::WatchSetCreation;
57use crate::statement_logging::{StatementEndedExecutionReason, StatementLifecycleEvent};
58use crate::{
59 AdapterError, AdapterNotice, CollectionIdBundle, ExecuteResponse, PeekClient, ReadHolds,
60 TimelineContext, TimestampContext, TimestampProvider, optimize,
61};
62use crate::{coord, metrics};
63
64impl PeekClient {
65 /// Attempt to sequence a peek from the session task.
66 ///
67 /// Returns `Ok(Some(response))` if we handled the peek, or `Ok(None)` to fall back to the
68 /// Coordinator's sequencing. If it returns an error, it should be returned to the user.
69 ///
70 /// `outer_ctx_extra` is Some when we are executing as part of an outer statement, e.g., a FETCH
71 /// triggering the execution of the underlying query.
72 pub(crate) async fn try_frontend_peek(
73 &mut self,
74 portal_name: &str,
75 session: &mut Session,
76 outer_ctx_extra: &mut Option<ExecuteContextGuard>,
77 ) -> Result<Option<ExecuteResponse>, AdapterError> {
78 // # From handle_execute
79
80 if session.vars().emit_trace_id_notice() {
81 let span_context = tracing::Span::current()
82 .context()
83 .span()
84 .span_context()
85 .clone();
86 if span_context.is_valid() {
87 session.add_notice(AdapterNotice::QueryTrace {
88 trace_id: span_context.trace_id(),
89 });
90 }
91 }
92
93 // TODO(peek-seq): This snapshot is wasted when we end up bailing out from the frontend peek
94 // sequencing. We could solve this is with that optimization where we
95 // continuously keep a catalog snapshot in the session, and only get a new one when the
96 // catalog revision has changed, which we could see with an atomic read.
97 // But anyhow, this problem will just go away when we reach the point that we never fall
98 // back to the old sequencing.
99 let catalog = self.catalog_snapshot("try_frontend_peek").await;
100
101 // Extract things from the portal.
102 let (stmt, params, logging, lifecycle_timestamps) = {
103 if let Err(err) = Coordinator::verify_portal(&*catalog, session, portal_name) {
104 outer_ctx_extra
105 .take()
106 .and_then(|guard| guard.defuse().retire());
107 return Err(err);
108 }
109 let portal = session
110 .get_portal_unverified(portal_name)
111 // The portal is a session-level thing, so it couldn't have concurrently disappeared
112 // since the above verification.
113 .expect("called verify_portal above");
114 let params = portal.parameters.clone();
115 let stmt = portal.stmt.clone();
116 let logging = Arc::clone(&portal.logging);
117 let lifecycle_timestamps = portal.lifecycle_timestamps.clone();
118 (stmt, params, logging, lifecycle_timestamps)
119 };
120
121 // Before planning, check if this is a statement type we can handle.
122 // This must happen BEFORE statement logging setup to avoid orphaned execution records.
123 if let Some(ref stmt) = stmt {
124 match &**stmt {
125 Statement::Select(_)
126 | Statement::ExplainAnalyzeObject(_)
127 | Statement::ExplainAnalyzeCluster(_)
128 | Statement::Show(ShowStatement::ShowObjects(_))
129 | Statement::Show(ShowStatement::ShowColumns(_)) => {
130 // These are always fine, just continue.
131 // Note: EXPLAIN ANALYZE will `plan` to `Plan::Select`.
132 // Note: ShowObjects plans to `Plan::Select`, ShowColumns plans to `Plan::ShowColumns`.
133 // We handle `Plan::ShowColumns` specially in `try_frontend_peek_inner`.
134 }
135 Statement::ExplainPlan(explain_stmt) => {
136 // Only handle ExplainPlan for SELECT statements.
137 // We don't want to handle e.g. EXPLAIN CREATE MATERIALIZED VIEW here, because that
138 // requires purification before planning, which the frontend peek sequencing doesn't
139 // do.
140 match &explain_stmt.explainee {
141 mz_sql_parser::ast::Explainee::Select(..) => {
142 // This is a SELECT, continue
143 }
144 _ => {
145 debug!(
146 "Bailing out from try_frontend_peek, because EXPLAIN is not for a SELECT query"
147 );
148 return Ok(None);
149 }
150 }
151 }
152 Statement::ExplainPushdown(explain_stmt) => {
153 // Only handle EXPLAIN FILTER PUSHDOWN for non-BROKEN SELECT statements
154 match &explain_stmt.explainee {
155 mz_sql_parser::ast::Explainee::Select(_, false) => {}
156 _ => {
157 debug!(
158 "Bailing out from try_frontend_peek, because EXPLAIN FILTER PUSHDOWN is not for a SELECT query or is for EXPLAIN BROKEN"
159 );
160 return Ok(None);
161 }
162 }
163 }
164 Statement::Copy(copy_stmt) => {
165 match ©_stmt.direction {
166 CopyDirection::To => {
167 // Check for SUBSCRIBE inside COPY TO - we don't handle Plan::Subscribe
168 if matches!(©_stmt.relation, CopyRelation::Subscribe(_)) {
169 debug!(
170 "Bailing out from try_frontend_peek, because COPY (SUBSCRIBE ...) TO is not supported"
171 );
172 return Ok(None);
173 }
174 // This is COPY TO (SELECT), continue
175 }
176 CopyDirection::From => {
177 debug!(
178 "Bailing out from try_frontend_peek, because COPY FROM is not supported"
179 );
180 return Ok(None);
181 }
182 }
183 }
184 _ => {
185 debug!(
186 "Bailing out from try_frontend_peek, because statement type is not supported"
187 );
188 return Ok(None);
189 }
190 }
191 }
192
193 // Set up statement logging, and log the beginning of execution.
194 // (But only if we're not executing in the context of another statement.)
195 let statement_logging_id = if outer_ctx_extra.is_none() {
196 // This is a new statement, so begin statement logging
197 let result = self.statement_logging_frontend.begin_statement_execution(
198 session,
199 ¶ms,
200 &logging,
201 catalog.system_config(),
202 lifecycle_timestamps,
203 );
204
205 if let Some((logging_id, began_execution, mseh_update, prepared_statement)) = result {
206 self.log_began_execution(began_execution, mseh_update, prepared_statement);
207 Some(logging_id)
208 } else {
209 None
210 }
211 } else {
212 // We're executing in the context of another statement (e.g., FETCH),
213 // so extract the statement logging ID from the outer context if present.
214 // We take ownership and retire the outer context here. The end of execution will be
215 // logged in one of the following ways:
216 // - At the end of this function, if the execution is finished by then.
217 // - Later by the Coordinator, either due to RegisterFrontendPeek or ExecuteSlowPathPeek.
218 outer_ctx_extra
219 .take()
220 .and_then(|guard| guard.defuse().retire())
221 };
222
223 let result = self
224 .try_frontend_peek_inner(session, catalog, stmt, params, statement_logging_id)
225 .await;
226
227 // Log the end of execution if we are logging this statement and execution has already
228 // ended.
229 if let Some(logging_id) = statement_logging_id {
230 let reason = match &result {
231 // Streaming results are handled asynchronously by the coordinator
232 Ok(Some(ExecuteResponse::SendingRowsStreaming { .. })) => {
233 // Don't log here - the peek is still executing.
234 // It will be logged when handle_peek_notification is called.
235 return result;
236 }
237 // COPY TO needs to check its inner response
238 Ok(Some(resp @ ExecuteResponse::CopyTo { resp: inner, .. })) => {
239 match inner.as_ref() {
240 ExecuteResponse::SendingRowsStreaming { .. } => {
241 // Don't log here - the peek is still executing.
242 // It will be logged when handle_peek_notification is called.
243 return result;
244 }
245 // For non-streaming COPY TO responses, use the outer CopyTo for conversion
246 _ => resp.into(),
247 }
248 }
249 // Bailout case, which should not happen
250 Ok(None) => {
251 soft_panic_or_log!(
252 "Bailed out from `try_frontend_peek_inner` after we already logged the beginning of statement execution."
253 );
254 // This statement will be handled by the old peek sequencing, which will do its
255 // own statement logging from the beginning. So, let's close out this one.
256 self.log_ended_execution(
257 logging_id,
258 StatementEndedExecutionReason::Errored {
259 error: "Internal error: bailed out from `try_frontend_peek_inner`"
260 .to_string(),
261 },
262 );
263 return result;
264 }
265 // All other success responses - use the From implementation
266 // TODO(peek-seq): After we delete the old peek sequencing, we'll be able to adjust
267 // the From implementation to do exactly what we need in the frontend peek
268 // sequencing, so that the above special cases won't be needed.
269 Ok(Some(resp)) => resp.into(),
270 Err(e) => StatementEndedExecutionReason::Errored {
271 error: e.to_string(),
272 },
273 };
274
275 self.log_ended_execution(logging_id, reason);
276 }
277
278 result
279 }
280
281 /// This is encapsulated in an inner function so that the outer function can still do statement
282 /// logging after the `?` returns of the inner function.
283 async fn try_frontend_peek_inner(
284 &mut self,
285 session: &mut Session,
286 catalog: Arc<Catalog>,
287 stmt: Option<Arc<Statement<Raw>>>,
288 params: Params,
289 statement_logging_id: Option<crate::statement_logging::StatementLoggingId>,
290 ) -> Result<Option<ExecuteResponse>, AdapterError> {
291 let stmt = match stmt {
292 Some(stmt) => stmt,
293 None => {
294 debug!("try_frontend_peek_inner succeeded on an empty query");
295 return Ok(Some(ExecuteResponse::EmptyQuery));
296 }
297 };
298
299 session
300 .metrics()
301 .query_total(&[
302 metrics::session_type_label_value(session.user()),
303 metrics::statement_type_label_value(&stmt),
304 ])
305 .inc();
306
307 // # From handle_execute_inner
308
309 let conn_catalog = catalog.for_session(session);
310 // (`resolved_ids` should be derivable from `stmt`. If `stmt` is later transformed to
311 // remove/add IDs, then `resolved_ids` should be updated to also remove/add those IDs.)
312 let (stmt, resolved_ids) = mz_sql::names::resolve(&conn_catalog, (*stmt).clone())?;
313
314 let pcx = session.pcx();
315 let plan = mz_sql::plan::plan(Some(pcx), &conn_catalog, stmt, ¶ms, &resolved_ids)?;
316
317 let (select_plan, explain_ctx, copy_to_ctx) = match &plan {
318 Plan::Select(select_plan) => {
319 let explain_ctx = if session.vars().emit_plan_insights_notice() {
320 let optimizer_trace = OptimizerTrace::new(ExplainStage::PlanInsights.paths());
321 ExplainContext::PlanInsightsNotice(optimizer_trace)
322 } else {
323 ExplainContext::None
324 };
325 (select_plan, explain_ctx, None)
326 }
327 Plan::ShowColumns(show_columns_plan) => {
328 // ShowColumns wraps a SelectPlan, extract it and proceed as normal.
329 (&show_columns_plan.select_plan, ExplainContext::None, None)
330 }
331 Plan::ExplainPlan(plan::ExplainPlanPlan {
332 stage,
333 format,
334 config,
335 explainee: Explainee::Statement(ExplaineeStatement::Select { broken, plan, desc }),
336 }) => {
337 // Create OptimizerTrace to collect optimizer plans
338 let optimizer_trace = OptimizerTrace::new(stage.paths());
339 let explain_ctx = ExplainContext::Plan(ExplainPlanContext {
340 broken: *broken,
341 config: config.clone(),
342 format: *format,
343 stage: *stage,
344 replan: None,
345 desc: Some(desc.clone()),
346 optimizer_trace,
347 });
348 (plan, explain_ctx, None)
349 }
350 // COPY TO S3
351 Plan::CopyTo(plan::CopyToPlan {
352 select_plan,
353 desc,
354 to,
355 connection,
356 connection_id,
357 format,
358 max_file_size,
359 }) => {
360 let uri = eval_copy_to_uri(to.clone(), session, catalog.state())?;
361
362 // (output_batch_count will be set later)
363 let copy_to_ctx = CopyToContext {
364 desc: desc.clone(),
365 uri,
366 connection: connection.clone(),
367 connection_id: *connection_id,
368 format: format.clone(),
369 max_file_size: *max_file_size,
370 output_batch_count: None,
371 };
372
373 (select_plan, ExplainContext::None, Some(copy_to_ctx))
374 }
375 Plan::ExplainPushdown(plan::ExplainPushdownPlan { explainee }) => {
376 // Only handle EXPLAIN FILTER PUSHDOWN for SELECT statements
377 match explainee {
378 plan::Explainee::Statement(plan::ExplaineeStatement::Select {
379 broken: false,
380 plan,
381 desc: _,
382 }) => {
383 let explain_ctx = ExplainContext::Pushdown;
384 (plan, explain_ctx, None)
385 }
386 _ => {
387 // This shouldn't happen because we already checked for this at the AST
388 // level before calling `try_frontend_peek_inner`.
389 soft_panic_or_log!(
390 "unexpected EXPLAIN FILTER PUSHDOWN plan kind in frontend peek sequencing: {:?}",
391 explainee
392 );
393 debug!(
394 "Bailing out from try_frontend_peek_inner, because EXPLAIN FILTER PUSHDOWN is not for a SELECT query or is EXPLAIN BROKEN"
395 );
396 return Ok(None);
397 }
398 }
399 }
400 Plan::SideEffectingFunc(sef_plan) => {
401 // Side-effecting functions need Coordinator state (e.g., active_conns),
402 // so delegate to the Coordinator via a Command.
403 // The RBAC check is performed in the Coordinator where active_conns is available.
404 let response = self
405 .call_coordinator(|tx| Command::ExecuteSideEffectingFunc {
406 plan: sef_plan.clone(),
407 conn_id: session.conn_id().clone(),
408 current_role: session.role_metadata().current_role,
409 tx,
410 })
411 .await?;
412 return Ok(Some(response));
413 }
414 _ => {
415 // This shouldn't happen because we already checked for this at the AST
416 // level before calling `try_frontend_peek_inner`.
417 soft_panic_or_log!(
418 "Unexpected plan kind in frontend peek sequencing: {:?}",
419 plan
420 );
421 debug!(
422 "Bailing out from try_frontend_peek_inner, because the Plan is not a SELECT, side-effecting SELECT, EXPLAIN SELECT, EXPLAIN FILTER PUSHDOWN, or COPY TO S3"
423 );
424 return Ok(None);
425 }
426 };
427
428 // # From sequence_plan
429
430 // We have checked the plan kind above.
431 assert!(plan.allowed_in_read_only());
432
433 let target_cluster = match session.transaction().cluster() {
434 // Use the current transaction's cluster.
435 Some(cluster_id) => TargetCluster::Transaction(cluster_id),
436 // If there isn't a current cluster set for a transaction, then try to auto route.
437 None => {
438 coord::catalog_serving::auto_run_on_catalog_server(&conn_catalog, session, &plan)
439 }
440 };
441 let (cluster, target_cluster_id, target_cluster_name) = {
442 let cluster = catalog.resolve_target_cluster(target_cluster, session)?;
443 (cluster, cluster.id, &cluster.name)
444 };
445
446 // Log cluster selection
447 if let Some(logging_id) = &statement_logging_id {
448 self.log_set_cluster(*logging_id, target_cluster_id);
449 }
450
451 coord::catalog_serving::check_cluster_restrictions(
452 target_cluster_name.as_str(),
453 &conn_catalog,
454 &plan,
455 )?;
456
457 rbac::check_plan(
458 &conn_catalog,
459 // We can't look at `active_conns` here, but that's ok, because this case was handled
460 // above already inside `Command::ExecuteSideEffectingFunc`.
461 None::<fn(u32) -> Option<RoleId>>,
462 session,
463 &plan,
464 Some(target_cluster_id),
465 &resolved_ids,
466 )?;
467
468 if let Some((_, wait_future)) =
469 coord::appends::waiting_on_startup_appends(&*catalog, session, &plan)
470 {
471 wait_future.await;
472 }
473
474 let max_query_result_size = Some(session.vars().max_query_result_size());
475
476 // # From sequence_peek
477
478 // # From peek_validate
479
480 let compute_instance_snapshot =
481 ComputeInstanceSnapshot::new_without_collections(cluster.id());
482
483 let optimizer_config = optimize::OptimizerConfig::from(catalog.system_config())
484 .override_from(&catalog.get_cluster(cluster.id()).config.features())
485 .override_from(&explain_ctx);
486
487 if cluster.replicas().next().is_none() && explain_ctx.needs_cluster() {
488 return Err(AdapterError::NoClusterReplicasAvailable {
489 name: cluster.name.clone(),
490 is_managed: cluster.is_managed(),
491 });
492 }
493
494 let (_, view_id) = self.transient_id_gen.allocate_id();
495 let (_, index_id) = self.transient_id_gen.allocate_id();
496
497 let mut optimizer = if let Some(mut copy_to_ctx) = copy_to_ctx {
498 // COPY TO path: calculate output_batch_count and create copy_to optimizer
499 let worker_counts = cluster.replicas().map(|r| {
500 let loc = &r.config.location;
501 loc.workers().unwrap_or_else(|| loc.num_processes())
502 });
503 let max_worker_count = match worker_counts.max() {
504 Some(count) => u64::cast_from(count),
505 None => {
506 return Err(AdapterError::NoClusterReplicasAvailable {
507 name: cluster.name.clone(),
508 is_managed: cluster.is_managed(),
509 });
510 }
511 };
512 copy_to_ctx.output_batch_count = Some(max_worker_count);
513
514 Either::Right(optimize::copy_to::Optimizer::new(
515 Arc::clone(&catalog),
516 compute_instance_snapshot.clone(),
517 view_id,
518 copy_to_ctx,
519 optimizer_config,
520 self.optimizer_metrics.clone(),
521 ))
522 } else {
523 // SELECT/EXPLAIN path: create peek optimizer
524 Either::Left(optimize::peek::Optimizer::new(
525 Arc::clone(&catalog),
526 compute_instance_snapshot.clone(),
527 select_plan.finishing.clone(),
528 view_id,
529 index_id,
530 optimizer_config,
531 self.optimizer_metrics.clone(),
532 ))
533 };
534
535 let target_replica_name = session.vars().cluster_replica();
536 let mut target_replica = target_replica_name
537 .map(|name| {
538 cluster
539 .replica_id(name)
540 .ok_or(AdapterError::UnknownClusterReplica {
541 cluster_name: cluster.name.clone(),
542 replica_name: name.to_string(),
543 })
544 })
545 .transpose()?;
546
547 let source_ids = select_plan.source.depends_on();
548 // TODO(peek-seq): validate_timeline_context can be expensive in real scenarios (not in
549 // simple benchmarks), because it traverses transitive dependencies even of indexed views and
550 // materialized views (also traversing their MIR plans).
551 let mut timeline_context = catalog.validate_timeline_context(source_ids.iter().copied())?;
552 if matches!(timeline_context, TimelineContext::TimestampIndependent)
553 && select_plan.source.contains_temporal()?
554 {
555 // If the source IDs are timestamp independent but the query contains temporal functions,
556 // then the timeline context needs to be upgraded to timestamp dependent. This is
557 // required because `source_ids` doesn't contain functions.
558 timeline_context = TimelineContext::TimestampDependent;
559 }
560
561 let notices = coord::sequencer::check_log_reads(
562 &catalog,
563 cluster,
564 &source_ids,
565 &mut target_replica,
566 session.vars(),
567 )?;
568 session.add_notices(notices);
569
570 // # From peek_linearize_timestamp
571
572 let isolation_level = session.vars().transaction_isolation().clone();
573 let timeline = Coordinator::get_timeline(&timeline_context);
574 let needs_linearized_read_ts =
575 Coordinator::needs_linearized_read_ts(&isolation_level, &select_plan.when);
576
577 let oracle_read_ts = match timeline {
578 Some(timeline) if needs_linearized_read_ts => {
579 let oracle = self.ensure_oracle(timeline).await?;
580 let oracle_read_ts = oracle.read_ts().await;
581 Some(oracle_read_ts)
582 }
583 Some(_) | None => None,
584 };
585
586 // # From peek_real_time_recency
587
588 let vars = session.vars();
589 let real_time_recency_ts: Option<Timestamp> = if vars.real_time_recency()
590 && vars.transaction_isolation() == &IsolationLevel::StrictSerializable
591 && !session.contains_read_timestamp()
592 {
593 // Only call the coordinator when we actually need real-time recency
594 self.call_coordinator(|tx| Command::DetermineRealTimeRecentTimestamp {
595 source_ids: source_ids.clone(),
596 real_time_recency_timeout: *vars.real_time_recency_timeout(),
597 tx,
598 })
599 .await?
600 } else {
601 None
602 };
603
604 // # From peek_timestamp_read_hold
605
606 let dataflow_builder = DataflowBuilder::new(catalog.state(), compute_instance_snapshot);
607 let input_id_bundle = dataflow_builder.sufficient_collections(source_ids.clone());
608
609 // ## From sequence_peek_timestamp
610
611 // Warning: This will be false for AS OF queries, even if we are otherwise inside a
612 // multi-statement transaction. (It's also false for FreshestTableWrite, which is currently
613 // only read-then-write queries, which can't be part of multi-statement transactions, so
614 // FreshestTableWrite doesn't matter.)
615 //
616 // TODO(peek-seq): It's not totally clear to me what the intended semantics are for AS OF
617 // queries inside a transaction: We clearly can't use the transaction timestamp, but the old
618 // peek sequencing still does a timedomain validation. The new peek sequencing does not do
619 // timedomain validation for AS OF queries, which seems more natural. But I'm thinking that
620 // it would be the cleanest to just simply disallow AS OF queries inside transactions.
621 let in_immediate_multi_stmt_txn = session
622 .transaction()
623 .in_immediate_multi_stmt_txn(&select_plan.when);
624
625 // Fetch or generate a timestamp for this query and fetch or acquire read holds.
626 let (determination, read_holds) = match session.get_transaction_timestamp_determination() {
627 // Use the transaction's timestamp if it exists and this isn't an AS OF query.
628 // (`in_immediate_multi_stmt_txn` is false for AS OF queries.)
629 Some(
630 determination @ TimestampDetermination {
631 timestamp_context: TimestampContext::TimelineTimestamp { .. },
632 ..
633 },
634 ) if in_immediate_multi_stmt_txn => {
635 // This is a subsequent (non-AS OF, non-constant) query in a multi-statement
636 // transaction. We now:
637 // - Validate that the query only accesses collections within the transaction's
638 // timedomain (which we know from the stored read holds).
639 // - Use the transaction's stored timestamp determination.
640 // - Use the (relevant subset of the) transaction's read holds.
641
642 let txn_read_holds_opt = self
643 .call_coordinator(|tx| Command::GetTransactionReadHoldsBundle {
644 conn_id: session.conn_id().clone(),
645 tx,
646 })
647 .await;
648
649 if let Some(txn_read_holds) = txn_read_holds_opt {
650 let allowed_id_bundle = txn_read_holds.id_bundle();
651 let outside = input_id_bundle.difference(&allowed_id_bundle);
652
653 // Queries without a timestamp and timeline can belong to any existing timedomain.
654 if determination.timestamp_context.contains_timestamp() && !outside.is_empty() {
655 let valid_names =
656 allowed_id_bundle.resolve_names(&*catalog, session.conn_id());
657 let invalid_names = outside.resolve_names(&*catalog, session.conn_id());
658 return Err(AdapterError::RelationOutsideTimeDomain {
659 relations: invalid_names,
660 names: valid_names,
661 });
662 }
663
664 // Extract the subset of read holds for the collections this query accesses.
665 let read_holds = txn_read_holds.subset(&input_id_bundle);
666
667 (determination, read_holds)
668 } else {
669 // This should never happen: we're in a subsequent query of a multi-statement
670 // transaction (we have a transaction timestamp), but the coordinator has no
671 // transaction read holds stored. This indicates a bug in the transaction
672 // handling.
673 return Err(AdapterError::Internal(
674 "Missing transaction read holds for multi-statement transaction"
675 .to_string(),
676 ));
677 }
678 }
679 _ => {
680 // There is no timestamp determination yet for this transaction. Either:
681 // - We are not in a multi-statement transaction.
682 // - This is the first (non-AS OF) query in a multi-statement transaction.
683 // - This is an AS OF query.
684 // - This is a constant query (`TimestampContext::NoTimestamp`).
685
686 let timedomain_bundle;
687 let determine_bundle = if in_immediate_multi_stmt_txn {
688 // This is the first (non-AS OF) query in a multi-statement transaction.
689 // Determine a timestamp that will be valid for anything in any schema
690 // referenced by the first query.
691 timedomain_bundle = timedomain_for(
692 &*catalog,
693 &dataflow_builder,
694 &source_ids,
695 &timeline_context,
696 session.conn_id(),
697 target_cluster_id,
698 )?;
699 &timedomain_bundle
700 } else {
701 // Simply use the inputs of the current query.
702 &input_id_bundle
703 };
704 let (determination, read_holds) = self
705 .frontend_determine_timestamp(
706 catalog.state(),
707 session,
708 determine_bundle,
709 &select_plan.when,
710 target_cluster_id,
711 &timeline_context,
712 oracle_read_ts,
713 real_time_recency_ts,
714 )
715 .await?;
716
717 // If this is the first (non-AS OF) query in a multi-statement transaction, store
718 // the read holds in the coordinator, so subsequent queries can validate against
719 // them.
720 if in_immediate_multi_stmt_txn {
721 self.call_coordinator(|tx| Command::StoreTransactionReadHolds {
722 conn_id: session.conn_id().clone(),
723 read_holds: read_holds.clone(),
724 tx,
725 })
726 .await;
727 }
728
729 (determination, read_holds)
730 }
731 };
732
733 {
734 // Assert that we have a read hold for all the collections in our `input_id_bundle`.
735 for id in input_id_bundle.iter() {
736 let s = read_holds.storage_holds.contains_key(&id);
737 let c = read_holds
738 .compute_ids()
739 .map(|(_instance, coll)| coll)
740 .contains(&id);
741 soft_assert_or_log!(
742 s || c,
743 "missing read hold for collection {} in `input_id_bundle",
744 id
745 );
746 }
747
748 // Assert that each part of the `input_id_bundle` corresponds to the right part of
749 // `read_holds`.
750 for id in input_id_bundle.storage_ids.iter() {
751 soft_assert_or_log!(
752 read_holds.storage_holds.contains_key(id),
753 "missing storage read hold for collection {} in `input_id_bundle",
754 id
755 );
756 }
757 for id in input_id_bundle
758 .compute_ids
759 .iter()
760 .flat_map(|(_instance, colls)| colls)
761 {
762 soft_assert_or_log!(
763 read_holds
764 .compute_ids()
765 .map(|(_instance, coll)| coll)
766 .contains(id),
767 "missing compute read hold for collection {} in `input_id_bundle",
768 id,
769 );
770 }
771 }
772
773 // (TODO(peek-seq): The below TODO is copied from the old peek sequencing. We should resolve
774 // this when we decide what to with `AS OF` in transactions.)
775 // TODO: Checking for only `InTransaction` and not `Implied` (also `Started`?) seems
776 // arbitrary and we don't recall why we did it (possibly an error!). Change this to always
777 // set the transaction ops. Decide and document what our policy should be on AS OF queries.
778 // Maybe they shouldn't be allowed in transactions at all because it's hard to explain
779 // what's going on there. This should probably get a small design document.
780
781 // We only track the peeks in the session if the query doesn't use AS
782 // OF or we're inside an explicit transaction. The latter case is
783 // necessary to support PG's `BEGIN` semantics, whose behavior can
784 // depend on whether or not reads have occurred in the txn.
785 let requires_linearization = (&explain_ctx).into();
786 let mut transaction_determination = determination.clone();
787 if select_plan.when.is_transactional() {
788 session.add_transaction_ops(TransactionOps::Peeks {
789 determination: transaction_determination,
790 cluster_id: target_cluster_id,
791 requires_linearization,
792 })?;
793 } else if matches!(session.transaction(), &TransactionStatus::InTransaction(_)) {
794 // If the query uses AS OF, then ignore the timestamp.
795 transaction_determination.timestamp_context = TimestampContext::NoTimestamp;
796 session.add_transaction_ops(TransactionOps::Peeks {
797 determination: transaction_determination,
798 cluster_id: target_cluster_id,
799 requires_linearization,
800 })?;
801 };
802
803 // # From peek_optimize
804
805 let stats = statistics_oracle(
806 session,
807 &source_ids,
808 &determination.timestamp_context.antichain(),
809 true,
810 catalog.system_config(),
811 &*self.storage_collections,
812 )
813 .await
814 .unwrap_or_else(|_| Box::new(EmptyStatisticsOracle));
815
816 // Generate data structures that can be moved to another task where we will perform possibly
817 // expensive optimizations.
818 let timestamp_context = determination.timestamp_context.clone();
819 let session_meta = session.meta();
820 let now = catalog.config().now.clone();
821 let select_plan = select_plan.clone();
822 let target_cluster_name = target_cluster_name.clone();
823 let needs_plan_insights = explain_ctx.needs_plan_insights();
824 let determination_for_pushdown = if matches!(explain_ctx, ExplainContext::Pushdown) {
825 // This is a hairy data structure, so avoid this clone if we are not in
826 // EXPLAIN FILTER PUSHDOWN.
827 Some(determination.clone())
828 } else {
829 None
830 };
831
832 let span = Span::current();
833
834 // Prepare data for plan insights if needed
835 let catalog_for_insights = if needs_plan_insights {
836 Some(Arc::clone(&catalog))
837 } else {
838 None
839 };
840 let mut compute_instances = BTreeMap::new();
841 if needs_plan_insights {
842 for user_cluster in catalog.user_clusters() {
843 let snapshot = ComputeInstanceSnapshot::new_without_collections(user_cluster.id);
844 compute_instances.insert(user_cluster.name.clone(), snapshot);
845 }
846 }
847
848 let source_ids_for_closure = source_ids.clone();
849 let optimization_future = mz_ore::task::spawn_blocking(
850 || "optimize peek",
851 move || {
852 span.in_scope(|| {
853 let _dispatch_guard = explain_ctx.dispatch_guard();
854
855 let raw_expr = select_plan.source.clone();
856
857 // The purpose of wrapping the following in a closure is to control where the
858 // `?`s return from, so that even when a `catch_unwind_optimize` call fails,
859 // we can still handle `EXPLAIN BROKEN`.
860 let pipeline = || -> Result<
861 Either<
862 optimize::peek::GlobalLirPlan,
863 optimize::copy_to::GlobalLirPlan,
864 >,
865 OptimizerError,
866 > {
867 match optimizer.as_mut() {
868 Either::Left(optimizer) => {
869 // SELECT/EXPLAIN path
870 // HIR ⇒ MIR lowering and MIR optimization (local)
871 let local_mir_plan =
872 optimizer.catch_unwind_optimize(raw_expr.clone())?;
873 // Attach resolved context required to continue the pipeline.
874 let local_mir_plan = local_mir_plan.resolve(
875 timestamp_context.clone(),
876 &session_meta,
877 stats,
878 );
879 // MIR optimization (global), MIR ⇒ LIR lowering, and LIR optimization (global)
880 let global_lir_plan =
881 optimizer.catch_unwind_optimize(local_mir_plan)?;
882 Ok(Either::Left(global_lir_plan))
883 }
884 Either::Right(optimizer) => {
885 // COPY TO path
886 // HIR ⇒ MIR lowering and MIR optimization (local)
887 let local_mir_plan =
888 optimizer.catch_unwind_optimize(raw_expr.clone())?;
889 // Attach resolved context required to continue the pipeline.
890 let local_mir_plan = local_mir_plan.resolve(
891 timestamp_context.clone(),
892 &session_meta,
893 stats,
894 );
895 // MIR optimization (global), MIR ⇒ LIR lowering, and LIR optimization (global)
896 let global_lir_plan =
897 optimizer.catch_unwind_optimize(local_mir_plan)?;
898 Ok(Either::Right(global_lir_plan))
899 }
900 }
901 };
902
903 let global_lir_plan_result = pipeline();
904 let optimization_finished_at = now();
905
906 let create_insights_ctx =
907 |optimizer: &optimize::peek::Optimizer,
908 is_notice: bool|
909 -> Option<Box<PlanInsightsContext>> {
910 if !needs_plan_insights {
911 return None;
912 }
913
914 let catalog = catalog_for_insights.as_ref()?;
915
916 let enable_re_optimize = if needs_plan_insights {
917 // Disable any plan insights that use the optimizer if we only want the
918 // notice and plan optimization took longer than the threshold. This is
919 // to prevent a situation where optimizing takes a while and there are
920 // lots of clusters, which would delay peek execution by the product of
921 // those.
922 //
923 // (This heuristic doesn't work well, see #9492.)
924 let dyncfgs = catalog.system_config().dyncfgs();
925 let opt_limit = mz_adapter_types::dyncfgs
926 ::PLAN_INSIGHTS_NOTICE_FAST_PATH_CLUSTERS_OPTIMIZE_DURATION
927 .get(dyncfgs);
928 !(is_notice && optimizer.duration() > opt_limit)
929 } else {
930 false
931 };
932
933 Some(Box::new(PlanInsightsContext {
934 stmt: select_plan
935 .select
936 .as_deref()
937 .map(Clone::clone)
938 .map(Statement::Select),
939 raw_expr: raw_expr.clone(),
940 catalog: Arc::clone(catalog),
941 compute_instances,
942 target_instance: target_cluster_name,
943 metrics: optimizer.metrics().clone(),
944 finishing: optimizer.finishing().clone(),
945 optimizer_config: optimizer.config().clone(),
946 session: session_meta,
947 timestamp_context,
948 view_id: optimizer.select_id(),
949 index_id: optimizer.index_id(),
950 enable_re_optimize,
951 }))
952 };
953
954 match global_lir_plan_result {
955 Ok(Either::Left(global_lir_plan)) => {
956 // SELECT/EXPLAIN path
957 let optimizer = optimizer.unwrap_left();
958 match explain_ctx {
959 ExplainContext::Plan(explain_ctx) => {
960 let (_, df_meta, _) = global_lir_plan.unapply();
961 let insights_ctx = create_insights_ctx(&optimizer, false);
962 Ok(Execution::ExplainPlan {
963 df_meta,
964 explain_ctx,
965 optimizer,
966 insights_ctx,
967 })
968 }
969 ExplainContext::None => Ok(Execution::Peek {
970 global_lir_plan,
971 optimization_finished_at,
972 plan_insights_optimizer_trace: None,
973 insights_ctx: None,
974 }),
975 ExplainContext::PlanInsightsNotice(optimizer_trace) => {
976 let insights_ctx = create_insights_ctx(&optimizer, true);
977 Ok(Execution::Peek {
978 global_lir_plan,
979 optimization_finished_at,
980 plan_insights_optimizer_trace: Some(optimizer_trace),
981 insights_ctx,
982 })
983 }
984 ExplainContext::Pushdown => {
985 let (plan, _, _) = global_lir_plan.unapply();
986 let imports = match plan {
987 PeekPlan::SlowPath(plan) => plan
988 .desc
989 .source_imports
990 .into_iter()
991 .filter_map(|(id, import)| {
992 import.desc.arguments.operators.map(|mfp| (id, mfp))
993 })
994 .collect(),
995 PeekPlan::FastPath(_) => {
996 std::collections::BTreeMap::default()
997 }
998 };
999 Ok(Execution::ExplainPushdown {
1000 imports,
1001 determination: determination_for_pushdown
1002 .expect("it's present for the ExplainPushdown case"),
1003 })
1004 }
1005 }
1006 }
1007 Ok(Either::Right(global_lir_plan)) => {
1008 // COPY TO S3 path
1009 Ok(Execution::CopyToS3 {
1010 global_lir_plan,
1011 source_ids: source_ids_for_closure,
1012 })
1013 }
1014 Err(err) => {
1015 if optimizer.is_right() {
1016 // COPY TO has no EXPLAIN BROKEN support
1017 return Err(err);
1018 }
1019 // SELECT/EXPLAIN error handling
1020 let optimizer = optimizer.expect_left("checked above");
1021 if let ExplainContext::Plan(explain_ctx) = explain_ctx {
1022 if explain_ctx.broken {
1023 // EXPLAIN BROKEN: log error and continue with defaults
1024 tracing::error!(
1025 "error while handling EXPLAIN statement: {}",
1026 err
1027 );
1028 Ok(Execution::ExplainPlan {
1029 df_meta: Default::default(),
1030 explain_ctx,
1031 optimizer,
1032 insights_ctx: None,
1033 })
1034 } else {
1035 Err(err)
1036 }
1037 } else {
1038 Err(err)
1039 }
1040 }
1041 }
1042 })
1043 },
1044 );
1045 let optimization_timeout = *session.vars().statement_timeout();
1046 let optimization_result =
1047 // Note: spawn_blocking tasks cannot be cancelled, so on timeout we stop waiting but the
1048 // optimization task continues running in the background until completion. See
1049 // https://github.com/MaterializeInc/database-issues/issues/8644 for properly cancelling
1050 // optimizer runs.
1051 match tokio::time::timeout(optimization_timeout, optimization_future).await {
1052 Ok(Ok(result)) => result,
1053 Ok(Err(optimizer_error)) => {
1054 return Err(AdapterError::Internal(format!(
1055 "internal error in optimizer: {}",
1056 optimizer_error
1057 )));
1058 }
1059 Err(_elapsed) => {
1060 warn!("optimize peek timed out after {:?}", optimization_timeout);
1061 return Err(AdapterError::StatementTimeout);
1062 }
1063 };
1064
1065 // Log optimization finished
1066 if let Some(logging_id) = &statement_logging_id {
1067 self.log_lifecycle_event(*logging_id, StatementLifecycleEvent::OptimizationFinished);
1068 }
1069
1070 // Assert that read holds are correct for the execution plan
1071 Self::assert_read_holds_correct(
1072 &read_holds,
1073 &optimization_result,
1074 &determination,
1075 target_cluster_id,
1076 );
1077
1078 // Handle the optimization result: either generate EXPLAIN output or continue with execution
1079 match optimization_result {
1080 Execution::ExplainPlan {
1081 df_meta,
1082 explain_ctx,
1083 optimizer,
1084 insights_ctx,
1085 } => {
1086 let rows = coord::sequencer::explain_plan_inner(
1087 session,
1088 &catalog,
1089 df_meta,
1090 explain_ctx,
1091 optimizer,
1092 insights_ctx,
1093 )
1094 .await?;
1095
1096 Ok(Some(ExecuteResponse::SendingRowsImmediate {
1097 rows: Box::new(rows.into_row_iter()),
1098 }))
1099 }
1100 Execution::ExplainPushdown {
1101 imports,
1102 determination,
1103 } => {
1104 // # From peek_explain_pushdown
1105
1106 let as_of = determination.timestamp_context.antichain();
1107 let mz_now = determination
1108 .timestamp_context
1109 .timestamp()
1110 .map(|t| ResultSpec::value(Datum::MzTimestamp(*t)))
1111 .unwrap_or_else(ResultSpec::value_all);
1112
1113 Ok(Some(
1114 coord::sequencer::explain_pushdown_future_inner(
1115 session,
1116 &*catalog,
1117 &self.storage_collections,
1118 as_of,
1119 mz_now,
1120 imports,
1121 )
1122 .await
1123 .await?,
1124 ))
1125 }
1126 Execution::Peek {
1127 global_lir_plan,
1128 optimization_finished_at: _optimization_finished_at,
1129 plan_insights_optimizer_trace,
1130 insights_ctx,
1131 } => {
1132 // Continue with normal execution
1133 // # From peek_finish
1134
1135 // The typ here was generated from the HIR SQL type and simply stored in LIR.
1136 let (peek_plan, df_meta, typ) = global_lir_plan.unapply();
1137
1138 coord::sequencer::emit_optimizer_notices(
1139 &*catalog,
1140 session,
1141 &df_meta.optimizer_notices,
1142 );
1143
1144 // Generate plan insights notice if needed
1145 if let Some(trace) = plan_insights_optimizer_trace {
1146 let target_cluster = catalog.get_cluster(target_cluster_id);
1147 let features = OptimizerFeatures::from(catalog.system_config())
1148 .override_from(&target_cluster.config.features());
1149 let insights = trace
1150 .into_plan_insights(
1151 &features,
1152 &catalog.for_session(session),
1153 Some(select_plan.finishing.clone()),
1154 Some(target_cluster),
1155 df_meta.clone(),
1156 insights_ctx,
1157 )
1158 .await?;
1159 session.add_notice(AdapterNotice::PlanInsights(insights));
1160 }
1161
1162 // # Now back to peek_finish
1163
1164 let watch_set = statement_logging_id.map(|logging_id| {
1165 WatchSetCreation::new(
1166 logging_id,
1167 catalog.state(),
1168 &input_id_bundle,
1169 determination.timestamp_context.timestamp_or_default(),
1170 )
1171 });
1172
1173 let max_result_size = catalog.system_config().max_result_size();
1174
1175 // Clone determination if we need it for emit_timestamp_notice, since it may be
1176 // moved into Command::ExecuteSlowPathPeek.
1177 let determination_for_notice = if session.vars().emit_timestamp_notice() {
1178 Some(determination.clone())
1179 } else {
1180 None
1181 };
1182
1183 let response = match peek_plan {
1184 PeekPlan::FastPath(fast_path_plan) => {
1185 if let Some(logging_id) = &statement_logging_id {
1186 // TODO(peek-seq): Actually, we should log it also for
1187 // FastPathPlan::Constant. The only reason we are not doing so at the
1188 // moment is to match the old peek sequencing, so that statement logging
1189 // tests pass with the frontend peek sequencing turned both on and off.
1190 //
1191 // When the old sequencing is removed, we should make a couple of
1192 // changes in how we log timestamps:
1193 // - Move this up to just after timestamp determination, so that it
1194 // appears in the log as soon as possible.
1195 // - Do it also for Constant peeks.
1196 // - Currently, slow-path peeks' timestamp logging is done by
1197 // `implement_peek_plan`. We could remove it from there, and just do
1198 // it here.
1199 if !matches!(fast_path_plan, FastPathPlan::Constant(..)) {
1200 self.log_set_timestamp(
1201 *logging_id,
1202 determination.timestamp_context.timestamp_or_default(),
1203 );
1204 }
1205 }
1206
1207 let row_set_finishing_seconds =
1208 session.metrics().row_set_finishing_seconds().clone();
1209
1210 let peek_stash_read_batch_size_bytes =
1211 mz_compute_types::dyncfgs::PEEK_RESPONSE_STASH_READ_BATCH_SIZE_BYTES
1212 .get(catalog.system_config().dyncfgs());
1213 let peek_stash_read_memory_budget_bytes =
1214 mz_compute_types::dyncfgs::PEEK_RESPONSE_STASH_READ_MEMORY_BUDGET_BYTES
1215 .get(catalog.system_config().dyncfgs());
1216
1217 self.implement_fast_path_peek_plan(
1218 fast_path_plan,
1219 determination.timestamp_context.timestamp_or_default(),
1220 select_plan.finishing,
1221 target_cluster_id,
1222 target_replica,
1223 typ,
1224 max_result_size,
1225 max_query_result_size,
1226 row_set_finishing_seconds,
1227 read_holds,
1228 peek_stash_read_batch_size_bytes,
1229 peek_stash_read_memory_budget_bytes,
1230 session.conn_id().clone(),
1231 source_ids,
1232 watch_set,
1233 )
1234 .await?
1235 }
1236 PeekPlan::SlowPath(dataflow_plan) => {
1237 if let Some(logging_id) = &statement_logging_id {
1238 self.log_set_transient_index_id(*logging_id, dataflow_plan.id);
1239 }
1240
1241 self.call_coordinator(|tx| Command::ExecuteSlowPathPeek {
1242 dataflow_plan: Box::new(dataflow_plan),
1243 determination,
1244 finishing: select_plan.finishing,
1245 compute_instance: target_cluster_id,
1246 target_replica,
1247 intermediate_result_type: typ,
1248 source_ids,
1249 conn_id: session.conn_id().clone(),
1250 max_result_size,
1251 max_query_result_size,
1252 watch_set,
1253 tx,
1254 })
1255 .await?
1256 }
1257 };
1258
1259 // Add timestamp notice if emit_timestamp_notice is enabled
1260 if let Some(determination) = determination_for_notice {
1261 let explanation = self
1262 .call_coordinator(|tx| Command::ExplainTimestamp {
1263 conn_id: session.conn_id().clone(),
1264 session_wall_time: session.pcx().wall_time,
1265 cluster_id: target_cluster_id,
1266 id_bundle: input_id_bundle.clone(),
1267 determination,
1268 tx,
1269 })
1270 .await;
1271 session.add_notice(AdapterNotice::QueryTimestamp { explanation });
1272 }
1273
1274 Ok(Some(match select_plan.copy_to {
1275 None => response,
1276 // COPY TO STDOUT
1277 Some(format) => ExecuteResponse::CopyTo {
1278 format,
1279 resp: Box::new(response),
1280 },
1281 }))
1282 }
1283 Execution::CopyToS3 {
1284 global_lir_plan,
1285 source_ids,
1286 } => {
1287 let (df_desc, df_meta) = global_lir_plan.unapply();
1288
1289 coord::sequencer::emit_optimizer_notices(
1290 &*catalog,
1291 session,
1292 &df_meta.optimizer_notices,
1293 );
1294
1295 // Extract S3 sink connection info for preflight check
1296 let sink_id = df_desc.sink_id();
1297 let sinks = &df_desc.sink_exports;
1298 if sinks.len() != 1 {
1299 return Err(AdapterError::Internal(
1300 "expected exactly one copy to s3 sink".into(),
1301 ));
1302 }
1303 let (_, sink_desc) = sinks
1304 .first_key_value()
1305 .expect("known to be exactly one copy to s3 sink");
1306 let s3_sink_connection = match &sink_desc.connection {
1307 mz_compute_types::sinks::ComputeSinkConnection::CopyToS3Oneshot(conn) => {
1308 conn.clone()
1309 }
1310 _ => {
1311 return Err(AdapterError::Internal(
1312 "expected copy to s3 oneshot sink".into(),
1313 ));
1314 }
1315 };
1316
1317 // Perform S3 preflight check in background task (via coordinator).
1318 // This runs slow S3 operations without blocking the coordinator's main task.
1319 self.call_coordinator(|tx| Command::CopyToPreflight {
1320 s3_sink_connection,
1321 sink_id,
1322 tx,
1323 })
1324 .await?;
1325
1326 // Preflight succeeded, now execute the actual COPY TO dataflow
1327 let watch_set = statement_logging_id.map(|logging_id| {
1328 WatchSetCreation::new(
1329 logging_id,
1330 catalog.state(),
1331 &input_id_bundle,
1332 determination.timestamp_context.timestamp_or_default(),
1333 )
1334 });
1335
1336 let response = self
1337 .call_coordinator(|tx| Command::ExecuteCopyTo {
1338 df_desc: Box::new(df_desc),
1339 compute_instance: target_cluster_id,
1340 target_replica,
1341 source_ids,
1342 conn_id: session.conn_id().clone(),
1343 watch_set,
1344 tx,
1345 })
1346 .await?;
1347
1348 Ok(Some(response))
1349 }
1350 }
1351 }
1352
1353 /// (Similar to Coordinator::determine_timestamp)
1354 /// Determines the timestamp for a query, acquires read holds that ensure the
1355 /// query remains executable at that time, and returns those.
1356 /// The caller is responsible for eventually dropping those read holds.
1357 ///
1358 /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`.
1359 pub(crate) async fn frontend_determine_timestamp(
1360 &mut self,
1361 catalog_state: &CatalogState,
1362 session: &Session,
1363 id_bundle: &CollectionIdBundle,
1364 when: &QueryWhen,
1365 compute_instance: ComputeInstanceId,
1366 timeline_context: &TimelineContext,
1367 oracle_read_ts: Option<Timestamp>,
1368 real_time_recency_ts: Option<Timestamp>,
1369 ) -> Result<(TimestampDetermination<Timestamp>, ReadHolds<Timestamp>), AdapterError> {
1370 // this is copy-pasted from Coordinator
1371
1372 let constraint_based = ConstraintBasedTimestampSelection::from_str(
1373 &CONSTRAINT_BASED_TIMESTAMP_SELECTION.get(catalog_state.system_config().dyncfgs()),
1374 );
1375
1376 let isolation_level = session.vars().transaction_isolation();
1377
1378 let (read_holds, upper) = self
1379 .acquire_read_holds_and_least_valid_write(id_bundle)
1380 .await
1381 .map_err(|err| {
1382 AdapterError::concurrent_dependency_drop_from_collection_lookup_error(
1383 err,
1384 compute_instance,
1385 )
1386 })?;
1387 let (det, read_holds) = <Coordinator as TimestampProvider>::determine_timestamp_for_inner(
1388 session,
1389 id_bundle,
1390 when,
1391 compute_instance,
1392 timeline_context,
1393 oracle_read_ts,
1394 real_time_recency_ts,
1395 isolation_level,
1396 &constraint_based,
1397 read_holds,
1398 upper.clone(),
1399 )?;
1400
1401 session
1402 .metrics()
1403 .determine_timestamp(&[
1404 match det.respond_immediately() {
1405 true => "true",
1406 false => "false",
1407 },
1408 isolation_level.as_str(),
1409 &compute_instance.to_string(),
1410 constraint_based.as_str(),
1411 ])
1412 .inc();
1413 if !det.respond_immediately()
1414 && isolation_level == &IsolationLevel::StrictSerializable
1415 && real_time_recency_ts.is_none()
1416 {
1417 // Note down the difference between StrictSerializable and Serializable into a metric.
1418 if let Some(strict) = det.timestamp_context.timestamp() {
1419 let (serializable_det, _tmp_read_holds) =
1420 <Coordinator as TimestampProvider>::determine_timestamp_for_inner(
1421 session,
1422 id_bundle,
1423 when,
1424 compute_instance,
1425 timeline_context,
1426 oracle_read_ts,
1427 real_time_recency_ts,
1428 isolation_level,
1429 &constraint_based,
1430 read_holds.clone(),
1431 upper,
1432 )?;
1433 if let Some(serializable) = serializable_det.timestamp_context.timestamp() {
1434 session
1435 .metrics()
1436 .timestamp_difference_for_strict_serializable_ms(&[
1437 compute_instance.to_string().as_ref(),
1438 constraint_based.as_str(),
1439 ])
1440 .observe(f64::cast_lossy(u64::from(
1441 strict.saturating_sub(*serializable),
1442 )));
1443 }
1444 }
1445 }
1446
1447 Ok((det, read_holds))
1448 }
1449
1450 fn assert_read_holds_correct(
1451 read_holds: &ReadHolds<Timestamp>,
1452 execution: &Execution,
1453 determination: &TimestampDetermination<Timestamp>,
1454 target_cluster_id: ClusterId,
1455 ) {
1456 // Extract source_imports, index_imports, as_of, and execution_name based on Execution variant
1457 let (source_imports, index_imports, as_of, execution_name): (
1458 Vec<GlobalId>,
1459 Vec<GlobalId>,
1460 Timestamp,
1461 &str,
1462 ) = match execution {
1463 Execution::Peek {
1464 global_lir_plan, ..
1465 } => match global_lir_plan.peek_plan() {
1466 PeekPlan::FastPath(fast_path_plan) => {
1467 let (sources, indexes) = match fast_path_plan {
1468 FastPathPlan::Constant(..) => (vec![], vec![]),
1469 FastPathPlan::PeekExisting(_coll_id, idx_id, ..) => (vec![], vec![*idx_id]),
1470 FastPathPlan::PeekPersist(global_id, ..) => (vec![*global_id], vec![]),
1471 };
1472 (
1473 sources,
1474 indexes,
1475 determination.timestamp_context.timestamp_or_default(),
1476 "FastPath",
1477 )
1478 }
1479 PeekPlan::SlowPath(dataflow_plan) => {
1480 let as_of = dataflow_plan
1481 .desc
1482 .as_of
1483 .clone()
1484 .expect("dataflow has an as_of")
1485 .into_element();
1486 (
1487 dataflow_plan.desc.source_imports.keys().cloned().collect(),
1488 dataflow_plan.desc.index_imports.keys().cloned().collect(),
1489 as_of,
1490 "SlowPath",
1491 )
1492 }
1493 },
1494 Execution::CopyToS3 {
1495 global_lir_plan, ..
1496 } => {
1497 let df_desc = global_lir_plan.df_desc();
1498 let as_of = df_desc
1499 .as_of
1500 .clone()
1501 .expect("dataflow has an as_of")
1502 .into_element();
1503 (
1504 df_desc.source_imports.keys().cloned().collect(),
1505 df_desc.index_imports.keys().cloned().collect(),
1506 as_of,
1507 "CopyToS3",
1508 )
1509 }
1510 Execution::ExplainPlan { .. } | Execution::ExplainPushdown { .. } => {
1511 // No read holds assertions needed for EXPLAIN variants
1512 return;
1513 }
1514 };
1515
1516 // Assert that we have some read holds for all the imports of the dataflow.
1517 for id in source_imports.iter() {
1518 soft_assert_or_log!(
1519 read_holds.storage_holds.contains_key(id),
1520 "[{}] missing read hold for the source import {}",
1521 execution_name,
1522 id
1523 );
1524 }
1525 for id in index_imports.iter() {
1526 soft_assert_or_log!(
1527 read_holds
1528 .compute_ids()
1529 .map(|(_instance, coll)| coll)
1530 .contains(id),
1531 "[{}] missing read hold for the index import {}",
1532 execution_name,
1533 id,
1534 );
1535 }
1536
1537 // Also check the holds against the as_of.
1538 for (id, h) in read_holds.storage_holds.iter() {
1539 soft_assert_or_log!(
1540 h.since().less_equal(&as_of),
1541 "[{}] storage read hold at {:?} for collection {} is not enough for as_of {:?}, determination: {:?}",
1542 execution_name,
1543 h.since(),
1544 id,
1545 as_of,
1546 determination
1547 );
1548 }
1549 for ((instance, id), h) in read_holds.compute_holds.iter() {
1550 soft_assert_eq_or_log!(
1551 *instance,
1552 target_cluster_id,
1553 "[{}] the read hold on {} is on the wrong cluster",
1554 execution_name,
1555 id
1556 );
1557 soft_assert_or_log!(
1558 h.since().less_equal(&as_of),
1559 "[{}] compute read hold at {:?} for collection {} is not enough for as_of {:?}, determination: {:?}",
1560 execution_name,
1561 h.since(),
1562 id,
1563 as_of,
1564 determination
1565 );
1566 }
1567 }
1568}
1569
1570/// Enum for branching among various execution steps after optimization
1571enum Execution {
1572 Peek {
1573 global_lir_plan: optimize::peek::GlobalLirPlan,
1574 optimization_finished_at: EpochMillis,
1575 plan_insights_optimizer_trace: Option<OptimizerTrace>,
1576 insights_ctx: Option<Box<PlanInsightsContext>>,
1577 },
1578 CopyToS3 {
1579 global_lir_plan: optimize::copy_to::GlobalLirPlan,
1580 source_ids: BTreeSet<GlobalId>,
1581 },
1582 ExplainPlan {
1583 df_meta: DataflowMetainfo,
1584 explain_ctx: ExplainPlanContext,
1585 optimizer: optimize::peek::Optimizer,
1586 insights_ctx: Option<Box<PlanInsightsContext>>,
1587 },
1588 ExplainPushdown {
1589 imports: BTreeMap<GlobalId, mz_expr::MapFilterProject>,
1590 determination: TimestampDetermination<Timestamp>,
1591 },
1592}