From 534d7669ed30c771fb1befe0cdd562cda561423a Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 24 Aug 2025 15:31:59 +0200 Subject: [PATCH 01/55] adapter: Peek sequencing in Adapter Frontend --- .../checks/all_checks/statement_logging.py | 4 + src/adapter/src/client.rs | 93 ++- src/adapter/src/command.rs | 39 +- src/adapter/src/coord.rs | 14 +- src/adapter/src/coord/command_handler.rs | 25 +- src/adapter/src/coord/peek.rs | 6 +- src/adapter/src/coord/read_policy.rs | 2 +- src/adapter/src/coord/sequencer.rs | 102 ++- src/adapter/src/coord/sequencer/inner.rs | 99 +-- src/adapter/src/coord/sequencer/inner/peek.rs | 15 +- .../src/coord/sequencer/inner/subscribe.rs | 5 +- src/adapter/src/coord/sql.rs | 4 +- src/adapter/src/coord/timestamp_selection.rs | 1 + src/adapter/src/frontend_peek.rs | 621 ++++++++++++++++++ src/adapter/src/lib.rs | 4 + src/adapter/src/metrics.rs | 26 + src/adapter/src/notice.rs | 9 + src/adapter/src/optimize/copy_to.rs | 4 +- src/adapter/src/optimize/dataflows.rs | 32 +- src/adapter/src/optimize/index.rs | 2 +- src/adapter/src/optimize/materialized_view.rs | 4 +- src/adapter/src/optimize/peek.rs | 4 +- src/adapter/src/optimize/subscribe.rs | 4 +- src/adapter/src/optimize/view.rs | 2 +- src/adapter/src/peek_client.rs | 362 ++++++++++ src/compute-client/src/controller.rs | 26 +- src/compute-client/src/controller/instance.rs | 141 +++- src/sql/src/optimizer_metrics.rs | 12 +- src/sql/src/session/vars/definitions.rs | 6 + src/transform/src/lib.rs | 8 +- test/cluster/mzcompose.py | 2 + 31 files changed, 1493 insertions(+), 185 deletions(-) create mode 100644 src/adapter/src/frontend_peek.rs create mode 100644 src/adapter/src/peek_client.rs diff --git a/misc/python/materialize/checks/all_checks/statement_logging.py b/misc/python/materialize/checks/all_checks/statement_logging.py index 65ebaf153b58c..85b7fa08f4a65 100644 --- a/misc/python/materialize/checks/all_checks/statement_logging.py +++ b/misc/python/materialize/checks/all_checks/statement_logging.py @@ -16,9 +16,13 @@ class StatementLogging(Check): def initialize(self) -> Testdrive: return Testdrive( dedent( + # TODO: enable_frontend_peek_sequencing when it supports statement logging. """ $ postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET statement_logging_max_sample_rate TO 1.0 + + $ postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} + ALTER SYSTEM SET enable_frontend_peek_sequencing = false; """ ) ) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 92d7d2b2c5c10..5ee4dbf28f559 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -39,12 +39,14 @@ use mz_sql::catalog::{EnvironmentId, SessionCatalog}; use mz_sql::session::hint::ApplicationNameHint; use mz_sql::session::metadata::SessionMetadata; use mz_sql::session::user::SUPPORT_USER; -use mz_sql::session::vars::{CLUSTER, OwnedVarInput, SystemVars, Var}; +use mz_sql::session::vars::{ + CLUSTER, ENABLE_FRONTEND_PEEK_SEQUENCING, OwnedVarInput, SystemVars, Var, +}; use mz_sql_parser::parser::{ParserStatementError, StatementParseResult}; use prometheus::Histogram; use serde_json::json; use tokio::sync::{mpsc, oneshot}; -use tracing::error; +use tracing::{debug, error}; use uuid::Uuid; use crate::catalog::Catalog; @@ -63,7 +65,7 @@ use crate::session::{ use crate::statement_logging::{StatementEndedExecutionReason, StatementExecutionStrategy}; use crate::telemetry::{self, EventDetails, SegmentClientExt, StatementFailureType}; use crate::webhook::AppendWebhookResponse; -use crate::{AdapterNotice, AppendWebhookError, PeekResponseUnary, StartupResponse}; +use crate::{AdapterNotice, AppendWebhookError, PeekClient, PeekResponseUnary, StartupResponse}; /// A handle to a running coordinator. /// @@ -252,21 +254,34 @@ impl Client { // Create the client as soon as startup succeeds (before any await points) so its `Drop` can // handle termination. + // Build the PeekClient with controller handles returned from startup. + let StartupResponse { + role_id, + write_notify, + session_defaults, + catalog, + storage_collections, + transient_id_gen, + optimizer_metrics, + } = response; + + let peek_client = PeekClient::new( + self.clone(), + storage_collections, + transient_id_gen, + optimizer_metrics, + // enable_frontend_peek_sequencing is initialized below, once we have a ConnCatalog + ); + let mut client = SessionClient { inner: Some(self.clone()), session: Some(session), timeouts: Timeout::new(), environment_id: self.environment_id.clone(), segment_client: self.segment_client.clone(), + peek_client, }; - let StartupResponse { - role_id, - write_notify, - session_defaults, - catalog, - } = response; - let session = client.session(); session.initialize_role_metadata(role_id); let vars_mut = session.vars_mut(); @@ -396,6 +411,10 @@ Issue a SQL query to get started. Need help? } } + client.peek_client.enable_frontend_peek_sequencing = ENABLE_FRONTEND_PEEK_SEQUENCING + .require(catalog.system_vars()) + .is_ok(); + Ok(client) } @@ -412,7 +431,7 @@ Issue a SQL query to get started. Need help? pub async fn support_execute_one( &self, sql: &str, - ) -> Result + Send + Sync>>, anyhow::Error> { + ) -> Result + Send>>, anyhow::Error> { // Connect to the coordinator. let conn_id = self.new_conn_id()?; let session = self.new_session(SessionConfig { @@ -503,7 +522,7 @@ Issue a SQL query to get started. Need help? } #[instrument(level = "debug")] - fn send(&self, cmd: Command) { + pub(crate) fn send(&self, cmd: Command) { self.inner_cmd_tx .send((OpenTelemetryContext::obtain(), cmd)) .expect("coordinator unexpectedly gone"); @@ -524,6 +543,8 @@ pub struct SessionClient { timeouts: Timeout, segment_client: Option, environment_id: EnvironmentId, + /// Thin client for frontend peek sequencing; populated at connection startup. + peek_client: PeekClient, } impl SessionClient { @@ -672,6 +693,17 @@ impl SessionClient { outer_ctx_extra: Option, ) -> Result<(ExecuteResponse, Instant), AdapterError> { let execute_started = Instant::now(); + + // Attempt peek sequencing in the session task. + // If unsupported, fall back to the Coordinator path. + // TODO: wire up cancel_future + if let Some(resp) = self.try_frontend_peek(&portal_name).await? { + debug!("frontend peek succeeded"); + return Ok((resp, execute_started)); + } else { + debug!("frontend peek did not happen"); + } + let response = self .send_with_cancel( |tx, session| Command::Execute { @@ -973,7 +1005,9 @@ impl SessionClient { | Command::Terminate { .. } | Command::RetireExecute { .. } | Command::CheckConsistency { .. } - | Command::Dump { .. } => {} + | Command::Dump { .. } + | Command::GetComputeInstanceClient { .. } + | Command::GetOracle { .. } => {} }; cmd }); @@ -1045,6 +1079,39 @@ impl SessionClient { pub async fn recv_timeout(&mut self) -> Option { self.timeouts.recv().await } + + /// Returns a reference to the PeekClient used for frontend peek sequencing. + pub fn peek_client(&self) -> &PeekClient { + &self.peek_client + } + + /// Returns a reference to the PeekClient used for frontend peek sequencing. + pub fn peek_client_mut(&mut self) -> &mut PeekClient { + &mut self.peek_client + } + + /// Attempt to sequence a peek from the session task. + /// + /// Returns Some(response) if we handled the peek, or None to fall back to the Coordinator's + /// peek sequencing. + pub(crate) async fn try_frontend_peek( + &mut self, + portal_name: &str, + ) -> Result, AdapterError> { + if self.peek_client().enable_frontend_peek_sequencing { + // We need to snatch the session out of self here, because if we just had a reference to it, + // then we couldn't do other things with self. + let mut session = self.session.take().expect("SessionClient invariant"); + let r = self + .try_frontend_peek_inner(portal_name, &mut session) + .await; + // Always put it back, even if we got an error. + self.session = Some(session); + r + } else { + Ok(None) + } + } } impl Drop for SessionClient { diff --git a/src/adapter/src/command.rs b/src/adapter/src/command.rs index 32c62dc7742ed..2bb4e6fa672b7 100644 --- a/src/adapter/src/command.rs +++ b/src/adapter/src/command.rs @@ -22,14 +22,18 @@ use mz_ore::collections::CollectionExt; use mz_ore::soft_assert_no_log; use mz_ore::tracing::OpenTelemetryContext; use mz_pgcopy::CopyFormatParams; +use mz_repr::global_id::TransientIdGen; use mz_repr::role_id::RoleId; -use mz_repr::{CatalogItemId, ColumnIndex, RowIterator}; +use mz_repr::{CatalogItemId, ColumnIndex, RowIterator, Timestamp}; use mz_sql::ast::{FetchDirection, Raw, Statement}; use mz_sql::catalog::ObjectType; +use mz_sql::optimizer_metrics::OptimizerMetrics; use mz_sql::plan::{ExecuteTimeout, Plan, PlanKind}; use mz_sql::session::user::User; use mz_sql::session::vars::{OwnedVarInput, SystemVars}; use mz_sql_parser::ast::{AlterObjectRenameStatement, AlterOwnerStatement, DropObjectsStatement}; +use mz_storage_types::sources::Timeline; +use mz_timestamp_oracle::TimestampOracle; use tokio::sync::{mpsc, oneshot}; use uuid::Uuid; @@ -153,6 +157,23 @@ pub enum Command { Dump { tx: oneshot::Sender>, }, + + GetComputeInstanceClient { + instance_id: ComputeInstanceId, + tx: oneshot::Sender< + Result< + mz_compute_client::controller::instance::Client, + AdapterError, + >, + >, + }, + + GetOracle { + timeline: Timeline, + tx: oneshot::Sender< + Result + Send + Sync>, AdapterError>, + >, + }, } impl Command { @@ -172,7 +193,9 @@ impl Command { | Command::SetSystemVars { .. } | Command::RetireExecute { .. } | Command::CheckConsistency { .. } - | Command::Dump { .. } => None, + | Command::Dump { .. } + | Command::GetComputeInstanceClient { .. } + | Command::GetOracle { .. } => None, } } @@ -192,7 +215,9 @@ impl Command { | Command::SetSystemVars { .. } | Command::RetireExecute { .. } | Command::CheckConsistency { .. } - | Command::Dump { .. } => None, + | Command::Dump { .. } + | Command::GetComputeInstanceClient { .. } + | Command::GetOracle { .. } => None, } } } @@ -216,6 +241,14 @@ pub struct StartupResponse { /// Map of (name, VarInput::Flat) tuples of session default variables that should be set. pub session_defaults: BTreeMap, pub catalog: Arc, + pub storage_collections: Arc< + dyn mz_storage_client::storage_collections::StorageCollections< + Timestamp = mz_repr::Timestamp, + > + Send + + Sync, + >, + pub transient_id_gen: Arc, + pub optimizer_metrics: OptimizerMetrics, } /// The response to [`Client::authenticate`](crate::Client::authenticate). diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index f1ba888c1f065..5282e02d4a511 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -208,26 +208,26 @@ use crate::util::{ClientTransmitter, ResultExt}; use crate::webhook::{WebhookAppenderInvalidator, WebhookConcurrencyLimiter}; use crate::{AdapterNotice, ReadHolds, flags}; +pub(crate) mod appends; +pub(crate) mod catalog_serving; +pub(crate) mod cluster_scheduling; +pub(crate) mod consistency; pub(crate) mod id_bundle; pub(crate) mod in_memory_oracle; pub(crate) mod peek; +pub(crate) mod read_policy; +pub(crate) mod sequencer; pub(crate) mod statement_logging; pub(crate) mod timeline; pub(crate) mod timestamp_selection; -pub mod appends; -mod catalog_serving; mod caught_up; -pub mod cluster_scheduling; mod command_handler; -pub mod consistency; mod ddl; mod indexes; mod introspection; mod message_handler; mod privatelink_status; -pub mod read_policy; -mod sequencer; mod sql; mod validity; @@ -361,6 +361,8 @@ impl Message { Command::AuthenticatePassword { .. } => "command-auth_check", Command::AuthenticateGetSASLChallenge { .. } => "command-auth_get_sasl_challenge", Command::AuthenticateVerifySASLProof { .. } => "command-auth_verify_sasl_proof", + Command::GetComputeInstanceClient { .. } => "get-compute-instance-client", + Command::GetOracle { .. } => "get-oracle", }, Message::ControllerReady { controller: ControllerReadiness::Compute, diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index 99c011a5dc9ed..9f80420974c64 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -270,6 +270,26 @@ impl Coordinator { Command::Dump { tx } => { let _ = tx.send(self.dump().await); } + + Command::GetComputeInstanceClient { instance_id, tx } => { + let _ = tx.send( + self.controller + .compute + .thin_instance_client(instance_id) + .map_err(|_| AdapterError::ConcurrentClusterDrop), + ); + } + + Command::GetOracle { timeline, tx } => { + let oracle = self + .global_timelines + .get(&timeline) + .map(|timeline_state| Arc::clone(&timeline_state.oracle)) + .ok_or(AdapterError::ChangedPlan( + "timeline has disappeared during planning".to_string(), + )); + let _ = tx.send(oracle); + } } } .instrument(debug_span!("handle_command")) @@ -528,6 +548,9 @@ impl Coordinator { write_notify: notify, session_defaults, catalog: self.owned_catalog(), + storage_collections: Arc::clone(&self.controller.storage_collections), + transient_id_gen: Arc::clone(&self.transient_id_gen), + optimizer_metrics: self.optimizer_metrics.clone(), }); if tx.send(resp).is_err() { // Failed to send to adapter, but everything is setup so we can terminate @@ -691,7 +714,7 @@ impl Coordinator { } } - if let Err(err) = self.verify_portal(&mut session, &portal_name) { + if let Err(err) = Self::verify_portal(self.catalog(), &mut session, &portal_name) { // If statement logging hasn't started yet, we don't need // to add any "end" event, so just make up a no-op // `ExecuteContextExtra` here, via `Default::default`. diff --git a/src/adapter/src/coord/peek.rs b/src/adapter/src/coord/peek.rs index 77ff54cea015c..fee595d17a7ce 100644 --- a/src/adapter/src/coord/peek.rs +++ b/src/adapter/src/coord/peek.rs @@ -379,7 +379,7 @@ pub struct PlannedPeek { /// [MapFilterProject](mz_expr::MapFilterProject), but _before_ applying a /// [RowSetFinishing]. /// - /// This is _the_ `result_type` as far as compute is concerned and futher + /// This is _the_ `result_type` as far as compute is concerned and further /// changes through projections happen purely in the adapter. pub intermediate_result_type: SqlRelationType, pub source_arity: usize, @@ -634,7 +634,7 @@ impl crate::coord::Coordinator { target_replica: Option, max_result_size: u64, max_returned_query_size: Option, - ) -> Result { + ) -> Result { let PlannedPeek { plan: fast_path, determination, @@ -827,7 +827,7 @@ impl crate::coord::Coordinator { let (literal_constraints, timestamp, map_filter_project) = peek_command; // At this stage we don't know column names for the result because we - // only know the peek's result type as a bare ResultType. + // only know the peek's result type as a bare SqlRelationType. let peek_result_column_names = (0..intermediate_result_type.arity()).map(|i| format!("peek_{i}")); let peek_result_desc = diff --git a/src/adapter/src/coord/read_policy.rs b/src/adapter/src/coord/read_policy.rs index d17ba5f211378..0efda732826c5 100644 --- a/src/adapter/src/coord/read_policy.rs +++ b/src/adapter/src/coord/read_policy.rs @@ -41,7 +41,7 @@ use crate::util::ResultExt; /// that read frontiers cannot advance past the held time as long as they exist. /// Dropping a [`ReadHolds`] also drops the [`ReadHold`] tokens within and /// relinquishes the associated read capabilities. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ReadHolds { pub storage_holds: BTreeMap>, pub compute_holds: BTreeMap<(ComputeInstanceId, GlobalId), ReadHold>, diff --git a/src/adapter/src/coord/sequencer.rs b/src/adapter/src/coord/sequencer.rs index 6da68400de7bd..a7be9afd751c9 100644 --- a/src/adapter/src/coord/sequencer.rs +++ b/src/adapter/src/coord/sequencer.rs @@ -12,9 +12,14 @@ //! Logic for executing a planned SQL query. +use std::collections::BTreeSet; +use std::sync::Arc; + use futures::FutureExt; use futures::future::LocalBoxFuture; use inner::return_if_err; +use mz_catalog::memory::objects::Cluster; +use mz_controller_types::ReplicaId; use mz_expr::row::RowCollection; use mz_expr::{MirRelationExpr, RowSetFinishing}; use mz_ore::tracing::OpenTelemetryContext; @@ -27,11 +32,11 @@ use mz_sql::plan::{ }; use mz_sql::rbac; use mz_sql::session::metadata::SessionMetadata; +use mz_sql::session::vars::SessionVars; use mz_sql_parser::ast::{Raw, Statement}; use mz_storage_client::client::TableData; use mz_storage_types::connections::inline::IntoInlineConnection; -use std::collections::BTreeSet; -use std::sync::Arc; +use mz_transform::notice::{OptimizerNoticeApi, OptimizerNoticeKind, RawOptimizerNotice}; use tokio::sync::oneshot; use tracing::{Instrument, Level, Span, event}; @@ -900,3 +905,96 @@ impl Coordinator { }) } } + +/// Checks whether we should emit diagnostic +/// information associated with reading per-replica sources. +/// +/// If an unrecoverable error is found (today: an untargeted read on a +/// cluster with a non-1 number of replicas), return that. Otherwise, +/// return a list of associated notices (today: we always emit exactly +/// one notice if there are any per-replica log dependencies and if +/// `emit_introspection_query_notice` is set, and none otherwise.) +pub(crate) fn check_log_reads( + catalog: &Catalog, + cluster: &Cluster, + source_ids: &BTreeSet, + target_replica: &mut Option, + vars: &SessionVars, +) -> Result, AdapterError> +where +{ + let log_names = source_ids + .iter() + .map(|gid| catalog.resolve_item_id(gid)) + .flat_map(|item_id| catalog.introspection_dependencies(item_id)) + .map(|item_id| catalog.get_entry(&item_id).name().item.clone()) + .collect::>(); + + if log_names.is_empty() { + return Ok(None); + } + + // Reading from log sources on replicated clusters is only allowed if a + // target replica is selected. Otherwise, we have no way of knowing which + // replica we read the introspection data from. + let num_replicas = cluster.replicas().count(); + if target_replica.is_none() { + if num_replicas == 1 { + *target_replica = cluster.replicas().map(|r| r.replica_id).next(); + } else { + return Err(AdapterError::UntargetedLogRead { log_names }); + } + } + + // Ensure that logging is initialized for the target replica, lest + // we try to read from a non-existing arrangement. + let replica_id = target_replica.expect("set to `Some` above"); + let replica = &cluster.replica(replica_id).expect("Replica must exist"); + if !replica.config.compute.logging.enabled() { + return Err(AdapterError::IntrospectionDisabled { log_names }); + } + + Ok(vars + .emit_introspection_query_notice() + .then_some(AdapterNotice::PerReplicaLogRead { log_names })) +} + +/// Forward notices that we got from the optimizer. +pub(crate) fn emit_optimizer_notices( + catalog: &Catalog, + session: &Session, + notices: &Vec, +) { + // `for_session` below is expensive, so return early if there's nothing to do. + if notices.is_empty() { + return; + } + let humanizer = catalog.for_session(session); + let system_vars = catalog.system_config(); + for notice in notices { + let kind = OptimizerNoticeKind::from(notice); + let notice_enabled = match kind { + OptimizerNoticeKind::IndexAlreadyExists => { + system_vars.enable_notices_for_index_already_exists() + } + OptimizerNoticeKind::IndexTooWideForLiteralConstraints => { + system_vars.enable_notices_for_index_too_wide_for_literal_constraints() + } + OptimizerNoticeKind::IndexKeyEmpty => system_vars.enable_notices_for_index_empty_key(), + }; + if notice_enabled { + // We don't need to redact the notice parts because + // `emit_optimizer_notices` is only called by the `sequence_~` + // method for the statement that produces that notice. + session.add_notice(AdapterNotice::OptimizerNotice { + notice: notice.message(&humanizer, false).to_string(), + hint: notice.hint(&humanizer, false).to_string(), + }); + } + session + .metrics() + .optimization_notices() + .with_label_values(&[kind.metric_label()]) + .inc_by(1); + } +} diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index add3389b26404..a1b9370bac6d0 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -24,10 +24,9 @@ use mz_adapter_types::compaction::CompactionWindow; use mz_adapter_types::connection::ConnectionId; use mz_adapter_types::dyncfgs::{ENABLE_MULTI_REPLICA_SOURCES, ENABLE_PASSWORD_AUTH}; use mz_catalog::memory::objects::{ - CatalogItem, Cluster, Connection, DataSourceDesc, Sink, Source, Table, TableDataSource, Type, + CatalogItem, Connection, DataSourceDesc, Sink, Source, Table, TableDataSource, Type, }; use mz_cloud_resources::VpcEndpointConfig; -use mz_controller_types::ReplicaId; use mz_expr::{ CollectionPlan, MapFilterProject, OptimizedMirRelationExpr, ResultSpec, RowSetFinishing, }; @@ -74,7 +73,7 @@ use mz_sql::plan::{ use mz_sql::session::metadata::SessionMetadata; use mz_sql::session::user::UserKind; use mz_sql::session::vars::{ - self, IsolationLevel, NETWORK_POLICY, OwnedVarInput, SCHEMA_ALIAS, SessionVars, + self, IsolationLevel, NETWORK_POLICY, OwnedVarInput, SCHEMA_ALIAS, TRANSACTION_ISOLATION_VAR_NAME, Var, VarError, VarInput, }; use mz_sql::{plan, rbac}; @@ -92,7 +91,6 @@ use mz_storage_types::controller::StorageError; use mz_storage_types::stats::RelationPartStats; use mz_transform::EmptyStatisticsOracle; use mz_transform::dataflow::DataflowMetainfo; -use mz_transform::notice::{OptimizerNoticeApi, OptimizerNoticeKind, RawOptimizerNotice}; use smallvec::SmallVec; use timely::progress::Antichain; use timely::progress::Timestamp as TimelyTimestamp; @@ -142,6 +140,7 @@ macro_rules! return_if_err { }; } +use crate::coord::sequencer::emit_optimizer_notices; pub(super) use return_if_err; struct DropOps { @@ -5292,97 +5291,7 @@ impl Coordinator { } } -/// Checks whether we should emit diagnostic -/// information associated with reading per-replica sources. -/// -/// If an unrecoverable error is found (today: an untargeted read on a -/// cluster with a non-1 number of replicas), return that. Otherwise, -/// return a list of associated notices (today: we always emit exactly -/// one notice if there are any per-replica log dependencies and if -/// `emit_introspection_query_notice` is set, and none otherwise.) -pub(super) fn check_log_reads( - catalog: &Catalog, - cluster: &Cluster, - source_ids: &BTreeSet, - target_replica: &mut Option, - vars: &SessionVars, -) -> Result, AdapterError> -where -{ - let log_names = source_ids - .iter() - .map(|gid| catalog.resolve_item_id(gid)) - .flat_map(|item_id| catalog.introspection_dependencies(item_id)) - .map(|item_id| catalog.get_entry(&item_id).name().item.clone()) - .collect::>(); - - if log_names.is_empty() { - return Ok(None); - } - - // Reading from log sources on replicated clusters is only allowed if a - // target replica is selected. Otherwise, we have no way of knowing which - // replica we read the introspection data from. - let num_replicas = cluster.replicas().count(); - if target_replica.is_none() { - if num_replicas == 1 { - *target_replica = cluster.replicas().map(|r| r.replica_id).next(); - } else { - return Err(AdapterError::UntargetedLogRead { log_names }); - } - } - - // Ensure that logging is initialized for the target replica, lest - // we try to read from a non-existing arrangement. - let replica_id = target_replica.expect("set to `Some` above"); - let replica = &cluster.replica(replica_id).expect("Replica must exist"); - if !replica.config.compute.logging.enabled() { - return Err(AdapterError::IntrospectionDisabled { log_names }); - } - - Ok(vars - .emit_introspection_query_notice() - .then_some(AdapterNotice::PerReplicaLogRead { log_names })) -} - impl Coordinator { - /// Forward notices that we got from the optimizer. - fn emit_optimizer_notices(&self, session: &Session, notices: &Vec) { - // `for_session` below is expensive, so return early if there's nothing to do. - if notices.is_empty() { - return; - } - let humanizer = self.catalog.for_session(session); - let system_vars = self.catalog.system_config(); - for notice in notices { - let kind = OptimizerNoticeKind::from(notice); - let notice_enabled = match kind { - OptimizerNoticeKind::IndexAlreadyExists => { - system_vars.enable_notices_for_index_already_exists() - } - OptimizerNoticeKind::IndexTooWideForLiteralConstraints => { - system_vars.enable_notices_for_index_too_wide_for_literal_constraints() - } - OptimizerNoticeKind::IndexKeyEmpty => { - system_vars.enable_notices_for_index_empty_key() - } - }; - if notice_enabled { - // We don't need to redact the notice parts because - // `emit_optimizer_notices` is only called by the `sequence_~` - // method for the statement that produces that notice. - session.add_notice(AdapterNotice::OptimizerNotice { - notice: notice.message(&humanizer, false).to_string(), - hint: notice.hint(&humanizer, false).to_string(), - }); - } - self.metrics - .optimization_notices - .with_label_values(&[kind.metric_label()]) - .inc_by(1); - } - } - /// Process the metainfo from a newly created non-transient dataflow. async fn process_dataflow_metainfo( &mut self, @@ -5393,7 +5302,7 @@ impl Coordinator { ) -> Option { // Emit raw notices to the user. if let Some(ctx) = ctx { - self.emit_optimizer_notices(ctx.session(), &df_meta.optimizer_notices); + emit_optimizer_notices(&*self.catalog, ctx.session(), &df_meta.optimizer_notices); } // Create a metainfo with rendered notices. diff --git a/src/adapter/src/coord/sequencer/inner/peek.rs b/src/adapter/src/coord/sequencer/inner/peek.rs index f8a608832d7ec..98c204356e7c4 100644 --- a/src/adapter/src/coord/sequencer/inner/peek.rs +++ b/src/adapter/src/coord/sequencer/inner/peek.rs @@ -39,7 +39,8 @@ use crate::active_compute_sink::{ActiveComputeSink, ActiveCopyTo}; use crate::command::ExecuteResponse; use crate::coord::id_bundle::CollectionIdBundle; use crate::coord::peek::{self, PeekDataflowPlan, PeekPlan, PlannedPeek}; -use crate::coord::sequencer::inner::{check_log_reads, return_if_err}; +use crate::coord::sequencer::inner::return_if_err; +use crate::coord::sequencer::{check_log_reads, emit_optimizer_notices}; use crate::coord::timeline::TimelineContext; use crate::coord::timestamp_selection::{ TimestampContext, TimestampDetermination, TimestampProvider, @@ -838,14 +839,12 @@ impl Coordinator { let (peek_plan, df_meta, typ) = global_lir_plan.unapply(); let source_arity = typ.arity(); - self.emit_optimizer_notices(&*session, &df_meta.optimizer_notices); - - let target_cluster = self.catalog().get_cluster(cluster_id); - - let features = OptimizerFeatures::from(self.catalog().system_config()) - .override_from(&target_cluster.config.features()); + emit_optimizer_notices(&*self.catalog, &*session, &df_meta.optimizer_notices); if let Some(trace) = plan_insights_optimizer_trace { + let target_cluster = self.catalog().get_cluster(cluster_id); + let features = OptimizerFeatures::from(self.catalog().system_config()) + .override_from(&target_cluster.config.features()); let insights = trace .into_plan_insights( &features, @@ -1016,7 +1015,7 @@ impl Coordinator { let (df_desc, df_meta) = global_lir_plan.unapply(); - self.emit_optimizer_notices(ctx.session(), &df_meta.optimizer_notices); + emit_optimizer_notices(&*self.catalog, ctx.session(), &df_meta.optimizer_notices); // Callback for the active copy to. let (tx, rx) = oneshot::channel(); diff --git a/src/adapter/src/coord/sequencer/inner/subscribe.rs b/src/adapter/src/coord/sequencer/inner/subscribe.rs index 843ab1fee1d71..dcb1acff02eea 100644 --- a/src/adapter/src/coord/sequencer/inner/subscribe.rs +++ b/src/adapter/src/coord/sequencer/inner/subscribe.rs @@ -17,7 +17,8 @@ use tracing::Span; use crate::active_compute_sink::{ActiveComputeSink, ActiveSubscribe}; use crate::command::ExecuteResponse; -use crate::coord::sequencer::inner::{check_log_reads, return_if_err}; +use crate::coord::sequencer::inner::return_if_err; +use crate::coord::sequencer::{check_log_reads, emit_optimizer_notices}; use crate::coord::{ Coordinator, Message, PlanValidity, StageResult, Staged, SubscribeFinish, SubscribeOptimizeMir, SubscribeStage, SubscribeTimestampOptimizeLir, TargetCluster, @@ -356,7 +357,7 @@ impl Coordinator { let (df_desc, df_meta) = global_lir_plan.unapply(); // Emit notices. - self.emit_optimizer_notices(ctx.session(), &df_meta.optimizer_notices); + emit_optimizer_notices(&*self.catalog, ctx.session(), &df_meta.optimizer_notices); // Add metadata for the new SUBSCRIBE. let write_notify_fut = self diff --git a/src/adapter/src/coord/sql.rs b/src/adapter/src/coord/sql.rs index f77d14f3ed940..feec9e1384e30 100644 --- a/src/adapter/src/coord/sql.rs +++ b/src/adapter/src/coord/sql.rs @@ -141,7 +141,7 @@ impl Coordinator { /// Verify a portal is still valid. pub(crate) fn verify_portal( - &self, + catalog: &Catalog, session: &mut Session, name: &str, ) -> Result<(), AdapterError> { @@ -150,7 +150,7 @@ impl Coordinator { None => return Err(AdapterError::UnknownCursor(name.to_string())), }; if let Some(new_revision) = Self::verify_statement_revision( - self.catalog(), + catalog, session, portal.stmt.as_deref(), &portal.desc, diff --git a/src/adapter/src/coord/timestamp_selection.rs b/src/adapter/src/coord/timestamp_selection.rs index e7ea6a67934ef..808baf6ee442e 100644 --- a/src/adapter/src/coord/timestamp_selection.rs +++ b/src/adapter/src/coord/timestamp_selection.rs @@ -894,6 +894,7 @@ impl Coordinator { && isolation_level == &IsolationLevel::StrictSerializable && real_time_recency_ts.is_none() { + // Note down the difference between StrictSerializable and Serializable into a metric. if let Some(strict) = det.timestamp_context.timestamp() { let (serializable_det, _tmp_read_holds) = self.determine_timestamp_for( session, diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs new file mode 100644 index 0000000000000..6f6e143091aa3 --- /dev/null +++ b/src/adapter/src/frontend_peek.rs @@ -0,0 +1,621 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use std::sync::Arc; + +use mz_adapter_types::dyncfgs::CONSTRAINT_BASED_TIMESTAMP_SELECTION; +use mz_adapter_types::timestamp_selection::ConstraintBasedTimestampSelection; +use mz_compute_types::ComputeInstanceId; +use mz_expr::CollectionPlan; +use mz_ore::cast::CastLossy; +use mz_repr::Timestamp; +use mz_repr::optimize::OverrideFrom; +use mz_sql::catalog::CatalogCluster; +use mz_sql::plan::{Plan, QueryWhen}; +use mz_sql::rbac; +use mz_sql::session::metadata::SessionMetadata; +use mz_sql::session::vars::IsolationLevel; +use mz_sql_parser::ast::Statement; +use mz_transform::EmptyStatisticsOracle; +use opentelemetry::trace::TraceContextExt; +use tracing::{Span, debug}; +use tracing_opentelemetry::OpenTelemetrySpanExt; + +use crate::catalog::CatalogState; +use crate::coord::peek::{FastPathPlan, PeekPlan}; +use crate::coord::timestamp_selection::TimestampDetermination; +use crate::coord::{Coordinator, ExplainContext, TargetCluster}; +use crate::optimize::Optimize; +use crate::optimize::dataflows::{ComputeInstanceSnapshot, DataflowBuilder}; +use crate::session::{Session, TransactionOps, TransactionStatus}; +use crate::{ + AdapterError, AdapterNotice, CollectionIdBundle, ExecuteResponse, ReadHolds, SessionClient, + TimelineContext, TimestampContext, TimestampProvider, optimize, +}; +use crate::{coord, metrics}; + +impl SessionClient { + pub(crate) async fn try_frontend_peek_inner( + &mut self, + portal_name: &str, + session: &mut Session, + ) -> Result, AdapterError> { + if session.transaction().is_in_multi_statement_transaction() { + // TODO: handle multi-statement transactions + debug!( + "Bailing out from try_frontend_peek_inner, because is_in_multi_statement_transaction" + ); + return Ok(None); + } + + if session.vars().emit_timestamp_notice() { + // TODO: implement this. See end of peek_finish + debug!("Bailing out from try_frontend_peek_inner, because emit_timestamp_notice"); + return Ok(None); + } + + // # From handle_execute + + if session.vars().emit_trace_id_notice() { + let span_context = tracing::Span::current() + .context() + .span() + .span_context() + .clone(); + if span_context.is_valid() { + // We emit a slightly different notice here than in handle_execute. + session.add_notice(AdapterNotice::FrontendQueryTrace { + trace_id: span_context.trace_id(), + }); + } + } + + // This is from handle_execute_inner, but we do it already here because of lifetime issues, + // and also to be able to give a catalog to `verify_portal`. + // + // TODO: This snapshot is wasted when we end up bailing out from the frontend peek + // sequencing. I think the best way to solve this is with that optimization where we + // continuously keep a catalog snapshot in the session, and only get a new one when the + // catalog revision has changed, which we could see with an atomic read. + let catalog = self.catalog_snapshot("try_frontend_peek").await; + + if let Err(_) = Coordinator::verify_portal(&*catalog, session, portal_name) { + // TODO: Don't fall back to the coordinator's peek sequencing here, but retire already. + debug!( + "Bailing out from try_frontend_peek_inner, because verify_portal returned an error" + ); + return Ok(None); + } + + // TODO: statement logging (and then re-enable it in workflow_statement_logging) + let (stmt, params) = { + let portal = session + .get_portal_unverified(portal_name) + .expect("called verify_portal above"); + let params = portal.parameters.clone(); + let stmt = portal.stmt.clone(); + (stmt, params) + }; + + let stmt = match stmt { + Some(stmt) => stmt, + None => { + debug!("try_frontend_peek_inner succeeded on an empty query"); + return Ok(Some(ExecuteResponse::EmptyQuery)); + } + }; + + if !matches!(*stmt, Statement::Select(_)) { + debug!("Bailing out from try_frontend_peek_inner, because it's not a SELECT"); + return Ok(None); + } + + let session_type = metrics::session_type_label_value(session.user()); + let stmt_type = metrics::statement_type_label_value(&stmt); + + // # From handle_execute_inner + + let conn_catalog = catalog.for_session(session); + // `resolved_ids` should be derivable from `stmt`. If `stmt` is transformed to remove/add + // IDs, then `resolved_ids` should be updated to also remove/add those IDs. + let (stmt, resolved_ids) = mz_sql::names::resolve(&conn_catalog, (*stmt).clone())?; + + let pcx = session.pcx(); + let plan = mz_sql::plan::plan(Some(pcx), &conn_catalog, stmt, ¶ms, &resolved_ids)?; + let select_plan = match &plan { + Plan::Select(select_plan) => select_plan, + _ => { + debug!("Bailing out from try_frontend_peek_inner, because it's not a SELECT"); + return Ok(None); + } + }; + let explain_ctx = ExplainContext::None; // EXPLAIN is not handled here for now, only SELECT + + // # From sequence_plan + + // We have checked the plan kind above. + assert!(plan.allowed_in_read_only()); + + let target_cluster = match session.transaction().cluster() { + // Use the current transaction's cluster. + Some(cluster_id) => TargetCluster::Transaction(cluster_id), + // If there isn't a current cluster set for a transaction, then try to auto route. + None => { + coord::catalog_serving::auto_run_on_catalog_server(&conn_catalog, session, &plan) + } + }; + let (cluster, target_cluster_id, target_cluster_name) = { + let cluster = catalog.resolve_target_cluster(target_cluster, session)?; + (cluster, cluster.id, &cluster.name) + }; + + // TODO: statement logging: set_statement_execution_cluster + + if let Err(e) = coord::catalog_serving::check_cluster_restrictions( + target_cluster_name.as_str(), + &conn_catalog, + &plan, + ) { + return Err(e); + } + + if let Err(e) = rbac::check_plan( + &conn_catalog, + |_id| { + // This is only used by `Plan::SideEffectingFunc`, so it is irrelevant for us here + // TODO: refactor `check_plan` to make this nicer + unreachable!() + }, + session, + &plan, + Some(target_cluster_id), + &resolved_ids, + ) { + return Err(e.into()); + } + + // Check if we're still waiting for any of the builtin table appends from when we + // started the Session to complete. + // + // (This is done slightly earlier in the normal peek sequencing, but we have to be past the + // last use of `conn_catalog` here.) + if let Some(_) = coord::appends::waiting_on_startup_appends(&*catalog, session, &plan) { + // TODO: Don't fall back to the coordinator's peek sequencing here, but call `defer_op`. + debug!("Bailing out from try_frontend_peek_inner, because waiting_on_startup_appends"); + return Ok(None); + } + + let max_query_result_size = Some(session.vars().max_query_result_size()); + + // # From sequence_peek + + if session.vars().emit_plan_insights_notice() { + // TODO: We'll need to do this when we want the frontend peek sequencing to take over + // from the old sequencing code. + debug!("Bailing out from try_frontend_peek_inner, because emit_plan_insights_notice"); + return Ok(None); + } + + // # From peek_validate + + //let compute_instance_snapshot = self.peek_client().snapshot(cluster.id()).await.unwrap(); + let compute_instance_snapshot = + ComputeInstanceSnapshot::new_without_collections(cluster.id()); + + let (_, view_id) = self.peek_client().transient_id_gen.allocate_id(); + let (_, index_id) = self.peek_client().transient_id_gen.allocate_id(); + + let optimizer_config = optimize::OptimizerConfig::from(catalog.system_config()) + .override_from(&catalog.get_cluster(cluster.id()).config.features()); + + if cluster.replicas().next().is_none() { + return Err(AdapterError::NoClusterReplicasAvailable { + name: cluster.name.clone(), + is_managed: cluster.is_managed(), + }); + } + + let mut optimizer = optimize::peek::Optimizer::new( + Arc::clone(&catalog), + compute_instance_snapshot.clone(), + select_plan.finishing.clone(), + view_id, + index_id, + optimizer_config, + self.peek_client().optimizer_metrics.clone(), + ); + + let target_replica_name = session.vars().cluster_replica(); + let mut target_replica = target_replica_name + .map(|name| { + cluster + .replica_id(name) + .ok_or(AdapterError::UnknownClusterReplica { + cluster_name: cluster.name.clone(), + replica_name: name.to_string(), + }) + }) + .transpose()?; + + let source_ids = select_plan.source.depends_on(); + // TODO: validate_timeline_context can be expensive in real scenarios (not in simple + // benchmarks), because it traverses transitive dependencies even of indexed views and + // materialized views (also traversing their MIR plans). + let mut timeline_context = catalog.validate_timeline_context(source_ids.iter().copied())?; + if matches!(timeline_context, TimelineContext::TimestampIndependent) + && select_plan.source.contains_temporal()? + { + // If the source IDs are timestamp independent but the query contains temporal functions, + // then the timeline context needs to be upgraded to timestamp dependent. This is + // required because `source_ids` doesn't contain functions. + timeline_context = TimelineContext::TimestampDependent; + } + + let notices = coord::sequencer::check_log_reads( + &catalog, + cluster, + &source_ids, + &mut target_replica, + session.vars(), + )?; + session.add_notices(notices); + + // # From peek_linearize_timestamp + + let isolation_level = session.vars().transaction_isolation().clone(); + let timeline = Coordinator::get_timeline(&timeline_context); + let needs_linearized_read_ts = + Coordinator::needs_linearized_read_ts(&isolation_level, &select_plan.when); + + let oracle_read_ts = match timeline { + Some(timeline) if needs_linearized_read_ts => { + let oracle = self.peek_client_mut().get_oracle(timeline).await?; + let oracle_read_ts = oracle.read_ts().await; + Some(oracle_read_ts) + } + Some(_) | None => None, + }; + + // # From peek_real_time_recency + + // TODO: Real-time recency is slow anyhow, so we don't handle it in frontend peek sequencing + // for now. + let vars = session.vars(); + if vars.real_time_recency() + && vars.transaction_isolation() == &IsolationLevel::StrictSerializable + && !session.contains_read_timestamp() + { + debug!("Bailing out from try_frontend_peek_inner, because of real time recency"); + return Ok(None); + } + let real_time_recency_ts: Option = None; + + // # From peek_timestamp_read_hold + + let dataflow_builder = DataflowBuilder::new(catalog.state(), compute_instance_snapshot); + let input_id_bundle = dataflow_builder.sufficient_collections(source_ids.clone()); + + // ## From sequence_peek_timestamp + + let in_immediate_multi_stmt_txn = session + .transaction() + .in_immediate_multi_stmt_txn(&select_plan.when); + + // Fetch or generate a timestamp for this query and fetch or acquire read holds. + // We need to keep these read holds until `implement_fast_path_peek_plan`, which acquires + // its own read holds. + let (determination, read_holds) = match session.get_transaction_timestamp_determination() { + // Use the transaction's timestamp if it exists and this isn't an AS OF query. + Some( + determination @ TimestampDetermination { + timestamp_context: TimestampContext::TimelineTimestamp { .. }, + .. + }, + ) if in_immediate_multi_stmt_txn => (determination, None), + _ => { + let determine_bundle = if in_immediate_multi_stmt_txn { + // TODO: handle multi-statement transactions + // needs timedomain_for, which needs DataflowBuilder / index oracle / sufficient_collections + debug!( + "Bailing out from try_frontend_peek_inner, because of in_immediate_multi_stmt_txn" + ); + return Ok(None); + } else { + // If not in a transaction, use the source. + &input_id_bundle + }; + let (determination, read_holds) = self + .frontend_determine_timestamp( + catalog.state(), + session, + determine_bundle, + &select_plan.when, + target_cluster_id, + &timeline_context, + oracle_read_ts, + real_time_recency_ts, + ) + .await?; + // We only need read holds if the read depends on a timestamp. + let read_holds = match determination.timestamp_context.timestamp() { + Some(_ts) => Some(read_holds), + None => { + // We don't need the read holds and shouldn't add them + // to the txn. + // + // TODO: Handle this within determine_timestamp. + drop(read_holds); + None + } + }; + (determination, read_holds) + } + }; + + // The old peek sequencing's sequence_peek_timestamp does two more things here: + // the txn_read_holds stuff and session.add_transaction_ops. We do these later in the new + // peek sequencing code, because at this point we might still bail out from the new peek + // sequencing, in which case we don't want the mentioned side effects to happen. + + // # From peek_optimize + + // Generate data structures that can be moved to another task where we will perform possibly + // expensive optimizations. + let timestamp_context = determination.timestamp_context.clone(); + if session.vars().enable_session_cardinality_estimates() { + debug!( + "Bailing out from try_frontend_peek_inner, because of enable_session_cardinality_estimates" + ); + return Ok(None); + } + let stats = Box::new(EmptyStatisticsOracle); + let session_meta = session.meta(); + let now = catalog.config().now.clone(); + let select_plan = select_plan.clone(); + + // TODO: if explain_ctx.needs_plan_insights() ... + + let span = Span::current(); + + let (global_lir_plan, _optimization_finished_at) = match mz_ore::task::spawn_blocking( + || "optimize peek", + move || { + span.in_scope(|| { + let raw_expr = select_plan.source.clone(); + + // HIR ⇒ MIR lowering and MIR optimization (local) + let local_mir_plan = optimizer.catch_unwind_optimize(raw_expr)?; + // Attach resolved context required to continue the pipeline. + let local_mir_plan = + local_mir_plan.resolve(timestamp_context.clone(), &session_meta, stats); + // MIR optimization (global), MIR ⇒ LIR lowering, and LIR optimization (global) + let global_lir_plan = optimizer.catch_unwind_optimize(local_mir_plan)?; + + let optimization_finished_at = now(); + + // TODO: plan_insights stuff + + Ok::<_, AdapterError>((global_lir_plan, optimization_finished_at)) + }) + }, + ) + .await + { + Ok(Ok(r)) => r, + Ok(Err(adapter_error)) => { + return Err(adapter_error); + } + Err(_join_error) => { + // Should only happen if the runtime is shutting down, because we + // - never call `abort`; + // - catch panics with `catch_unwind_optimize`. + return Err(AdapterError::Unstructured(anyhow::anyhow!( + "peek optimization aborted, because the system is shutting down" + ))); + } + }; + + // # From peek_finish + + // TODO: statement logging + + let (peek_plan, df_meta, typ) = global_lir_plan.unapply(); + + // TODO: plan_insights stuff + + // This match is based on what `implement_fast_path_peek_plan` supports. + let fast_path_plan = match peek_plan { + PeekPlan::SlowPath(_) => { + debug!("Bailing out from try_frontend_peek_inner, because it's a slow-path peek"); + return Ok(None); + } + PeekPlan::FastPath(p @ FastPathPlan::Constant(_, _)) + | PeekPlan::FastPath(p @ FastPathPlan::PeekExisting(_, _, _, _)) => p, + PeekPlan::FastPath(FastPathPlan::PeekPersist(_, _, _)) => { + debug!( + "Bailing out from try_frontend_peek_inner, because it's a Persist fast path peek" + ); + return Ok(None); + } + }; + + // Warning: Do not bail out from the new peek sequencing after this point, because the + // following has side effects. + + coord::sequencer::emit_optimizer_notices(&*catalog, session, &df_meta.optimizer_notices); + + // # We do the second half of sequence_peek_timestamp, as mentioned above. + + // TODO: txn_read_holds stuff. Add SessionClient::txn_read_holds. + + // (This TODO is copied from the old peek sequencing.) + // TODO: Checking for only `InTransaction` and not `Implied` (also `Started`?) seems + // arbitrary and we don't recall why we did it (possibly an error!). Change this to always + // set the transaction ops. Decide and document what our policy should be on AS OF queries. + // Maybe they shouldn't be allowed in transactions at all because it's hard to explain + // what's going on there. This should probably get a small design document. + + // We only track the peeks in the session if the query doesn't use AS + // OF or we're inside an explicit transaction. The latter case is + // necessary to support PG's `BEGIN` semantics, whose behavior can + // depend on whether or not reads have occurred in the txn. + let mut transaction_determination = determination.clone(); + let requires_linearization = (&explain_ctx).into(); + if select_plan.when.is_transactional() { + session.add_transaction_ops(TransactionOps::Peeks { + determination: transaction_determination, + cluster_id: target_cluster_id, + requires_linearization, + })?; + } else if matches!(session.transaction(), &TransactionStatus::InTransaction(_)) { + // If the query uses AS OF, then ignore the timestamp. + transaction_determination.timestamp_context = TimestampContext::NoTimestamp; + session.add_transaction_ops(TransactionOps::Peeks { + determination: transaction_determination, + cluster_id: target_cluster_id, + requires_linearization, + })?; + }; + + // TODO: move this up to the beginning of the function when we have eliminated all the + // fallbacks to the old peek sequencing. Currently, it has to be here to avoid + // double-counting a fallback situation, but this has the drawback that if we error out + // from this function then we don't count the peek at all. + session + .metrics() + .query_total() + .with_label_values(&[session_type, stmt_type]) + .inc(); + + // # Now back to peek_finish + + // TODO: statement logging + + let max_result_size = catalog.system_config().max_result_size(); + + let row_set_finishing_seconds = session.metrics().row_set_finishing_seconds().clone(); + + // Implement the peek, and capture the response. + let resp = self + .peek_client_mut() + .implement_fast_path_peek_plan( + fast_path_plan, + determination.timestamp_context.timestamp_or_default(), + select_plan.finishing, + target_cluster_id, + target_replica, + typ, + max_result_size, + max_query_result_size, + row_set_finishing_seconds, + read_holds, + ) + .await?; + + // Ok to lose `_read_holds` at this point, because `implement_fast_path_peek_plan` acquired + // its own read holds before returning. + + Ok(Some(resp)) + } + + /// (Similar to Coordinator::determine_timestamp) + /// Determines the timestamp for a query, acquires read holds that ensure the + /// query remains executable at that time, and returns those. + /// The caller is responsible for eventually dropping those read holds. + /// + /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. + pub(crate) async fn frontend_determine_timestamp( + &mut self, + catalog_state: &CatalogState, + session: &Session, + id_bundle: &CollectionIdBundle, + when: &QueryWhen, + compute_instance: ComputeInstanceId, + timeline_context: &TimelineContext, + oracle_read_ts: Option, + real_time_recency_ts: Option, + ) -> Result<(TimestampDetermination, ReadHolds), AdapterError> { + // this is copy-pasted from Coordinator + + let constraint_based = ConstraintBasedTimestampSelection::from_str( + &CONSTRAINT_BASED_TIMESTAMP_SELECTION.get(catalog_state.system_config().dyncfgs()), + ); + + let isolation_level = session.vars().transaction_isolation(); + + let (read_holds, upper) = self + .peek_client_mut() + .acquire_read_holds_and_collection_write_frontiers(id_bundle) + .await + .expect("missing collection"); + let (det, read_holds) = ::determine_timestamp_for_inner( + session, + id_bundle, + when, + compute_instance, + timeline_context, + oracle_read_ts, + real_time_recency_ts, + isolation_level, + &constraint_based, + read_holds, + upper.clone(), + )?; + + session + .metrics() + .determine_timestamp() + .with_label_values(&[ + match det.respond_immediately() { + true => "true", + false => "false", + }, + isolation_level.as_str(), + &compute_instance.to_string(), + constraint_based.as_str(), + ]) + .inc(); + if !det.respond_immediately() + && isolation_level == &IsolationLevel::StrictSerializable + && real_time_recency_ts.is_none() + { + // Note down the difference between StrictSerializable and Serializable into a metric. + if let Some(strict) = det.timestamp_context.timestamp() { + let (serializable_det, _tmp_read_holds) = + ::determine_timestamp_for_inner( + session, + id_bundle, + when, + compute_instance, + timeline_context, + oracle_read_ts, + real_time_recency_ts, + isolation_level, + &constraint_based, + read_holds.clone(), + upper, + )?; + if let Some(serializable) = serializable_det.timestamp_context.timestamp() { + session + .metrics() + .timestamp_difference_for_strict_serializable_ms() + .with_label_values(&[ + compute_instance.to_string().as_ref(), + constraint_based.as_str(), + ]) + .observe(f64::cast_lossy(u64::from( + strict.saturating_sub(*serializable), + ))); + } + } + } + + Ok((det, read_holds)) + } +} diff --git a/src/adapter/src/lib.rs b/src/adapter/src/lib.rs index 9bf1107a610cf..d9253e0df1657 100644 --- a/src/adapter/src/lib.rs +++ b/src/adapter/src/lib.rs @@ -43,6 +43,7 @@ mod command; mod coord; mod error; mod explain; +mod frontend_peek; mod notice; mod optimize; mod util; @@ -53,11 +54,14 @@ pub mod config; pub mod continual_task; pub mod flags; pub mod metrics; +pub mod peek_client; pub mod session; pub mod statement_logging; pub mod telemetry; pub mod webhook; +pub use crate::peek_client::PeekClient; + pub use crate::client::{Client, Handle, SessionClient}; pub use crate::command::{ExecuteResponse, ExecuteResponseKind, StartupResponse}; pub use crate::coord::ExecuteContext; diff --git a/src/adapter/src/metrics.rs b/src/adapter/src/metrics.rs index 0a773e0d9ab7e..b9824a79e4e34 100644 --- a/src/adapter/src/metrics.rs +++ b/src/adapter/src/metrics.rs @@ -232,6 +232,12 @@ impl Metrics { SessionMetrics { row_set_finishing_seconds: self.row_set_finishing_seconds(), session_startup_table_writes_seconds: self.session_startup_table_writes_seconds.clone(), + query_total: self.query_total.clone(), + determine_timestamp: self.determine_timestamp.clone(), + timestamp_difference_for_strict_serializable_ms: self + .timestamp_difference_for_strict_serializable_ms + .clone(), + optimization_notices: self.optimization_notices.clone(), } } } @@ -241,6 +247,10 @@ impl Metrics { pub struct SessionMetrics { row_set_finishing_seconds: Histogram, session_startup_table_writes_seconds: Histogram, + query_total: IntCounterVec, + determine_timestamp: IntCounterVec, + timestamp_difference_for_strict_serializable_ms: HistogramVec, + optimization_notices: IntCounterVec, } impl SessionMetrics { @@ -251,6 +261,22 @@ impl SessionMetrics { pub(crate) fn session_startup_table_writes_seconds(&self) -> &Histogram { &self.session_startup_table_writes_seconds } + + pub(crate) fn query_total(&self) -> &IntCounterVec { + &self.query_total + } + + pub(crate) fn determine_timestamp(&self) -> &IntCounterVec { + &self.determine_timestamp + } + + pub(crate) fn timestamp_difference_for_strict_serializable_ms(&self) -> &HistogramVec { + &self.timestamp_difference_for_strict_serializable_ms + } + + pub(crate) fn optimization_notices(&self) -> &IntCounterVec { + &self.optimization_notices + } } pub(crate) fn session_type_label_value(user: &User) -> &'static str { diff --git a/src/adapter/src/notice.rs b/src/adapter/src/notice.rs index 5a06f46f3b842..7609a6d549e6b 100644 --- a/src/adapter/src/notice.rs +++ b/src/adapter/src/notice.rs @@ -87,6 +87,10 @@ pub enum AdapterNotice { QueryTrace { trace_id: opentelemetry::trace::TraceId, }, + /// Like QueryTrace, but for the frontend peek sequencing. + FrontendQueryTrace { + trace_id: opentelemetry::trace::TraceId, + }, UnimplementedIsolationLevel { isolation_level: String, }, @@ -177,6 +181,7 @@ impl AdapterNotice { AdapterNotice::QueryTimestamp { .. } => Severity::Notice, AdapterNotice::EqualSubscribeBounds { .. } => Severity::Notice, AdapterNotice::QueryTrace { .. } => Severity::Notice, + AdapterNotice::FrontendQueryTrace { .. } => Severity::Notice, AdapterNotice::UnimplementedIsolationLevel { .. } => Severity::Notice, AdapterNotice::StrongSessionSerializable => Severity::Notice, AdapterNotice::BadStartupSetting { .. } => Severity::Notice, @@ -279,6 +284,7 @@ impl AdapterNotice { AdapterNotice::QueryTimestamp { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::EqualSubscribeBounds { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::QueryTrace { .. } => SqlState::SUCCESSFUL_COMPLETION, + AdapterNotice::FrontendQueryTrace { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::UnimplementedIsolationLevel { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::StrongSessionSerializable => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::BadStartupSetting { .. } => SqlState::SUCCESSFUL_COMPLETION, @@ -389,6 +395,9 @@ impl fmt::Display for AdapterNotice { AdapterNotice::QueryTrace { trace_id } => { write!(f, "trace id: {}", trace_id) } + AdapterNotice::FrontendQueryTrace { trace_id } => { + write!(f, "frontend trace id: {}", trace_id) + } AdapterNotice::UnimplementedIsolationLevel { isolation_level } => { write!( f, diff --git a/src/adapter/src/optimize/copy_to.rs b/src/adapter/src/optimize/copy_to.rs index 35532b8075e23..5733068c52e12 100644 --- a/src/adapter/src/optimize/copy_to.rs +++ b/src/adapter/src/optimize/copy_to.rs @@ -168,7 +168,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), Some(self.select_id), ); let expr = optimize_mir_local(expr, &mut transform_ctx)?.into_inner(); @@ -345,7 +345,7 @@ impl<'s> Optimize>> for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), ); // Run global optimization. mz_transform::optimize_dataflow(&mut df_desc, &mut transform_ctx, false)?; diff --git a/src/adapter/src/optimize/dataflows.rs b/src/adapter/src/optimize/dataflows.rs index b5f3a19b80fe8..7ebb0c302127e 100644 --- a/src/adapter/src/optimize/dataflows.rs +++ b/src/adapter/src/optimize/dataflows.rs @@ -54,7 +54,9 @@ use crate::util::viewable_variables; #[derive(Debug, Clone)] pub struct ComputeInstanceSnapshot { instance_id: ComputeInstanceId, - collections: BTreeSet, + /// The collections that exist on this compute instance. If it's None, then any collection that + /// a caller asks us about is considered to exist. + collections: Option>, } impl ComputeInstanceSnapshot { @@ -64,23 +66,43 @@ impl ComputeInstanceSnapshot { .collection_ids(id) .map(|collection_ids| Self { instance_id: id, - collections: collection_ids.collect(), + collections: Some(collection_ids.collect()), }) } + pub fn new_from_parts(instance_id: ComputeInstanceId, collections: BTreeSet) -> Self { + Self { + instance_id, + collections: Some(collections), + } + } + + pub fn new_without_collections(instance_id: ComputeInstanceId) -> Self { + Self { + instance_id, + collections: None, + } + } + /// Return the ID of this compute instance. pub fn instance_id(&self) -> ComputeInstanceId { self.instance_id } - /// Reports whether the instance contains the indicated collection. + /// Reports whether the instance contains the indicated collection. If the snapshot doesn't + /// track collections, then it returns true. pub fn contains_collection(&self, id: &GlobalId) -> bool { - self.collections.contains(id) + self.collections + .as_ref() + .map_or(true, |collections| collections.contains(id)) } /// Inserts the given collection into the snapshot. pub fn insert_collection(&mut self, id: GlobalId) { - self.collections.insert(id); + self.collections + .as_mut() + .expect("insert_collection called on snapshot with None collections") + .insert(id); } } diff --git a/src/adapter/src/optimize/index.rs b/src/adapter/src/optimize/index.rs index 848b312d0d955..1d1a9353a9bc5 100644 --- a/src/adapter/src/optimize/index.rs +++ b/src/adapter/src/optimize/index.rs @@ -178,7 +178,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), ); // Run global optimization. mz_transform::optimize_dataflow(&mut df_desc, &mut transform_ctx, false)?; diff --git a/src/adapter/src/optimize/materialized_view.rs b/src/adapter/src/optimize/materialized_view.rs index 5f92cb3c0a8da..0a60a1362d364 100644 --- a/src/adapter/src/optimize/materialized_view.rs +++ b/src/adapter/src/optimize/materialized_view.rs @@ -198,7 +198,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), Some(self.view_id), ); let expr = optimize_mir_local(expr, &mut transform_ctx)?.into_inner(); @@ -287,7 +287,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), ); // Apply source monotonicity overrides. for id in self.force_source_non_monotonic.iter() { diff --git a/src/adapter/src/optimize/peek.rs b/src/adapter/src/optimize/peek.rs index 416a2c9a5fd85..63456042b5296 100644 --- a/src/adapter/src/optimize/peek.rs +++ b/src/adapter/src/optimize/peek.rs @@ -182,7 +182,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), Some(self.select_id), ); let expr = optimize_mir_local(expr, &mut transform_ctx)?.into_inner(); @@ -337,7 +337,7 @@ impl<'s> Optimize>> for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), ); // Let's already try creating a fast path plan. If successful, we don't need to run the diff --git a/src/adapter/src/optimize/subscribe.rs b/src/adapter/src/optimize/subscribe.rs index 8c06e0d08d9fa..56c35c3634511 100644 --- a/src/adapter/src/optimize/subscribe.rs +++ b/src/adapter/src/optimize/subscribe.rs @@ -230,7 +230,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), Some(self.view_id), ); let expr = optimize_mir_local(expr, &mut transform_ctx)?; @@ -273,7 +273,7 @@ impl Optimize for Optimizer { &self.config.features, &self.typecheck_ctx, &mut df_meta, - Some(&self.metrics), + Some(&mut self.metrics), ); // Run global optimization. mz_transform::optimize_dataflow(&mut df_desc, &mut transform_ctx, false)?; diff --git a/src/adapter/src/optimize/view.rs b/src/adapter/src/optimize/view.rs index c9096fc6eda26..70339585db778 100644 --- a/src/adapter/src/optimize/view.rs +++ b/src/adapter/src/optimize/view.rs @@ -96,7 +96,7 @@ impl Optimize for Optimizer<'_> { &self.config.features, &self.typecheck_ctx, &mut df_meta, - self.metrics.as_ref(), + self.metrics.as_mut(), None, ); diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs new file mode 100644 index 0000000000000..cf8c87583d657 --- /dev/null +++ b/src/adapter/src/peek_client.rs @@ -0,0 +1,362 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use std::collections::BTreeMap; +use std::sync::Arc; + +use differential_dataflow::consolidation::consolidate; +use mz_compute_client::controller::error::CollectionMissing; +use mz_compute_client::protocol::command::PeekTarget; +use mz_compute_client::protocol::response::PeekResponse; +use mz_compute_types::ComputeInstanceId; +use mz_expr::row::RowCollection; +use mz_ore::cast::CastFrom; +use mz_repr::Timestamp; +use mz_repr::global_id::TransientIdGen; +use mz_repr::{RelationDesc, Row}; +use mz_sql::optimizer_metrics::OptimizerMetrics; +use mz_storage_types::sources::Timeline; +use mz_timestamp_oracle::TimestampOracle; +use prometheus::Histogram; +use timely::progress::Antichain; +use tokio::sync::oneshot; +use uuid::Uuid; + +use crate::command::Command; +use crate::coord; +use crate::coord::Coordinator; +use crate::coord::peek::FastPathPlan; +use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds}; + +/// Storage collections trait alias we need to consult for since/frontiers. +pub type StorageCollectionsHandle = Arc< + dyn mz_storage_client::storage_collections::StorageCollections + + Send + + Sync, +>; + +/// Clients needed for peek sequencing from the Adapter Frontend. +#[derive(Debug)] +pub struct PeekClient { + coordinator_client: Client, + /// Channels to talk to each compute Instance task directly. Lazily populated. + compute_instances: + BTreeMap>, + /// Handle to storage collections for reading frontiers and policies. + pub storage_collections: StorageCollectionsHandle, + /// A generator for transient `GlobalId`s, shared with Coordinator. + pub transient_id_gen: Arc, + pub optimizer_metrics: OptimizerMetrics, + /// Per-timeline oracles from the coordinator. Lazily populated. + oracles: BTreeMap + Send + Sync>>, + // TODO: This is initialized only at session startup. We'll be able to properly check + // the actual feature flag value (without a Coordinator call) once we'll always have a catalog + // snapshot at hand. + pub enable_frontend_peek_sequencing: bool, +} + +impl PeekClient { + /// Creates a PeekClient. Leaves `enable_frontend_peek_sequencing` false! This should be filled + /// in later. + pub fn new( + coordinator_client: Client, + storage_collections: StorageCollectionsHandle, + transient_id_gen: Arc, + optimizer_metrics: OptimizerMetrics, + ) -> Self { + Self { + coordinator_client, + compute_instances: Default::default(), // lazily populated + storage_collections, + transient_id_gen, + optimizer_metrics, + oracles: Default::default(), // lazily populated + enable_frontend_peek_sequencing: false, // should be filled in later! + } + } + + pub async fn get_compute_instance_client( + &mut self, + compute_instance: ComputeInstanceId, + ) -> Result<&mut mz_compute_client::controller::instance::Client, AdapterError> { + if !self.compute_instances.contains_key(&compute_instance) { + let client = self + .send_to_coordinator(|tx| Command::GetComputeInstanceClient { + instance_id: compute_instance, + tx, + }) + .await?; + self.compute_instances.insert(compute_instance, client); + } + Ok(self + .compute_instances + .get_mut(&compute_instance) + .expect("ensured above")) + } + + pub async fn get_oracle( + &mut self, + timeline: Timeline, + ) -> Result<&mut Arc + Send + Sync>, AdapterError> { + if !self.oracles.contains_key(&timeline) { + let oracle = self + .send_to_coordinator(|tx| Command::GetOracle { + timeline: timeline.clone(), + tx, + }) + .await?; + self.oracles.insert(timeline.clone(), oracle); + } + Ok(self.oracles.get_mut(&timeline).expect("ensured above")) + } + + async fn send_to_coordinator(&self, f: F) -> T + where + F: FnOnce(oneshot::Sender) -> Command, + { + let (tx, rx) = oneshot::channel(); + self.coordinator_client.send(f(tx)); + rx.await.expect("sender dropped") + } + + /// Acquire read holds on the given compute/storage collections, and + /// determine the smallest common valid write frontier among the specified collections. + /// + /// Similar to `Coordinator::acquire_read_holds` and `TimestampProvider::least_valid_write` + /// combined. + /// + /// Note: Unlike the Coordinator/StorageController's `least_valid_write` that treats sinks + /// specially when fetching storage frontiers (see `mz_storage_controller::collections_frontiers`), + /// we intentionally do not special‑case sinks here because peeks never read from sinks. + /// Therefore, using `StorageCollections::collections_frontiers` is sufficient. + /// + /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. + pub async fn acquire_read_holds_and_collection_write_frontiers( + &mut self, + id_bundle: &CollectionIdBundle, + ) -> Result<(ReadHolds, Antichain), CollectionMissing> { + let mut read_holds = ReadHolds::new(); + let mut upper = Antichain::new(); + + if !id_bundle.storage_ids.is_empty() { + let desired_storage: Vec<_> = id_bundle.storage_ids.iter().copied().collect(); + let storage_read_holds = self + .storage_collections + .acquire_read_holds(desired_storage) + .expect("missing storage collections"); + read_holds.storage_holds = storage_read_holds + .into_iter() + .map(|hold| (hold.id(), hold)) + .collect(); + + let storage_ids: Vec<_> = id_bundle.storage_ids.iter().copied().collect(); + for f in self + .storage_collections + .collections_frontiers(storage_ids) + .expect("missing collections") + { + upper.extend(f.write_frontier); + } + } + + for (&instance_id, collection_ids) in &id_bundle.compute_ids { + let client = self + .get_compute_instance_client(instance_id) + .await + .expect("missing compute instance client"); + + for (id, read_hold, write_frontier) in client + .acquire_read_holds_and_collection_write_frontiers( + collection_ids.iter().copied().collect(), + ) + .await? + { + let prev = read_holds + .compute_holds + .insert((instance_id, id), read_hold); + assert!( + prev.is_none(), + "duplicate compute ID in id_bundle {id_bundle:?}" + ); + + upper.extend(write_frontier); + } + } + + Ok((read_holds, upper)) + } + + /// Implement a fast-path peek plan. + /// This is similar to `Coordinator::implement_peek_plan`, but only for fast path peeks. + /// + /// Supported variants: + /// - FastPathPlan::Constant + /// - FastPathPlan::PeekExisting (PeekTarget::Index only) + /// + /// This fn assumes that the caller has already acquired read holds for the peek at the + /// appropriate timestamp. Before this fn returns, it passes on the responsibility of holding + /// back the frontiers to Compute, so then the caller can forget its own read holds. + /// + /// Note: FastPathPlan::PeekPersist is not yet supported here; we may add this later. + /// + /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. + /// + /// TODO: add statement logging + /// TODO: cancellation (see pending_peeks/client_pending_peeks wiring in the old sequencing) + pub async fn implement_fast_path_peek_plan( + &mut self, + fast_path: FastPathPlan, + timestamp: Timestamp, + finishing: mz_expr::RowSetFinishing, + compute_instance: ComputeInstanceId, + target_replica: Option, + intermediate_result_type: mz_repr::SqlRelationType, + max_result_size: u64, + max_returned_query_size: Option, + row_set_finishing_seconds: Histogram, + _read_holds: Option>, + ) -> Result { + match fast_path { + // If the dataflow optimizes to a constant expression, we can immediately return the result. + FastPathPlan::Constant(rows_res, _) => { + let mut rows = match rows_res { + Ok(rows) => rows, + Err(e) => return Err(e.into()), + }; + consolidate(&mut rows); + + let mut results = Vec::new(); + for (row, count) in rows { + if count.is_negative() { + return Err(AdapterError::Unstructured(anyhow::anyhow!( + "Negative multiplicity in constant result: {}", + count + ))); + } + if count.is_positive() { + let count = usize::cast_from( + u64::try_from(count.into_inner()) + .expect("known to be positive from check above"), + ); + results.push(( + row, + std::num::NonZeroUsize::new(count) + .expect("known to be non-zero from check above"), + )); + } + } + let row_collection = RowCollection::new(results, &finishing.order_by); + match finishing.finish( + row_collection, + max_result_size, + max_returned_query_size, + &row_set_finishing_seconds, + ) { + Ok((rows, _bytes)) => Ok(Coordinator::send_immediate_rows(rows)), + // TODO: make this a structured error. (also in the old sequencing) + Err(e) => Err(AdapterError::ResultSize(e)), + } + } + FastPathPlan::PeekExisting(_coll_id, idx_id, literal_constraints, mfp) => { + let (rows_tx, rows_rx) = oneshot::channel(); + let uuid = Uuid::new_v4(); + + // At this stage we don't know column names for the result because we + // only know the peek's result type as a bare SqlRelationType. + let cols = (0..intermediate_result_type.arity()).map(|i| format!("peek_{i}")); + let result_desc = RelationDesc::new(intermediate_result_type.clone(), cols); + + // Issue peek to the instance + let client = self + .get_compute_instance_client(compute_instance) + .await + .expect("missing compute instance client"); + let peek_target = PeekTarget::Index { id: idx_id }; + let literal_vec: Option> = literal_constraints; + let map_filter_project = mfp; + let finishing_for_instance = finishing.clone(); + client + .peek_call_sync( + peek_target, + literal_vec, + uuid, + timestamp, + result_desc, + finishing_for_instance, + map_filter_project, + // We let `Instance::peek` acquire read holds instead of passing them in. + // This is different from the old peek sequencing: That code is able to + // acquire read holds in `ComputeController::peek` at no significant cost, + // because that is a thick client to the controller. Here, we only have a + // thin client, which would need to do a roundtrip to the controller task + // for acquiring read holds, so it's better to let the instance do it in the + // same call that kicks off the peek itself. (Another difference is that + // the old peek sequencing's `ComputeController::peek` does not wait for + // `Instance::peek` to finish before returning, which means that if it were + // to let `Instance::peek` acquire the read holds, then we might acquire + // read holds too late, leading to panic if the Coordinator drops its own + // read holds in the meantime, e.g., due to cancelling the peek.) + None, + target_replica, + rows_tx, + ) + .await; + + // TODO: call `create_peek_response_stream` instead. For that, we'll need to pass in + // a PersistClient from afar. + let peek_response_stream = async_stream::stream!({ + match rows_rx.await { + Ok(PeekResponse::Rows(rows)) => { + match finishing.finish( + rows, + max_result_size, + max_returned_query_size, + &row_set_finishing_seconds, + ) { + Ok((rows, _size_bytes)) => { + yield coord::peek::PeekResponseUnary::Rows(Box::new(rows)) + } + Err(e) => yield coord::peek::PeekResponseUnary::Error(e), + } + } + Ok(PeekResponse::Stashed(_response)) => { + // TODO: support this (through `create_peek_response_stream`) + yield coord::peek::PeekResponseUnary::Error("stashed peek responses not yet supported in frontend peek sequencing".into()); + } + Ok(PeekResponse::Error(err)) => { + yield coord::peek::PeekResponseUnary::Error(err); + } + Ok(PeekResponse::Canceled) => { + yield coord::peek::PeekResponseUnary::Canceled; + } + Err(e) => { + yield coord::peek::PeekResponseUnary::Error(e.to_string()); + } + } + }); + + Ok(crate::ExecuteResponse::SendingRowsStreaming { + rows: Box::pin(peek_response_stream), + instance_id: compute_instance, + strategy: crate::statement_logging::StatementExecutionStrategy::FastPath, + }) + } + FastPathPlan::PeekPersist(..) => { + // TODO: Implement this. (We currently bail out in `try_frontend_peek_inner`, + // similarly to slow-path peeks.) + // Note that `Instance::peek` has the following comment: + // "For persist peeks, the controller should provide a storage read hold. + // We don't support acquiring it here." + // Can we do this from here? + // (Note that if we want to bail out for this case, we need to bail out earlier!) + unimplemented!("PeekPersist not yet supported in frontend peek sequencing") + } + } + } +} diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 532075a6e609b..9321789c04c14 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -76,12 +76,12 @@ use crate::metrics::ComputeControllerMetrics; use crate::protocol::command::{ComputeParameters, PeekTarget}; use crate::protocol::response::{PeekResponse, SubscribeBatch}; -mod instance; mod introspection; mod replica; mod sequential_hydration; pub mod error; +pub mod instance; pub(crate) type StorageCollections = Arc< dyn mz_storage_client::storage_collections::StorageCollections + Send + Sync, @@ -334,6 +334,14 @@ impl ComputeController { self.instances.get(&id).ok_or(InstanceMissing(id)) } + /// Return an `instance::Client` for the indicated compute instance. + pub fn thin_instance_client( + &self, + id: ComputeInstanceId, + ) -> Result, InstanceMissing> { + self.instance(id).map(|instance| instance.client.clone()) + } + /// Return a mutable reference to the indicated compute instance. fn instance_mut( &mut self, @@ -900,7 +908,7 @@ where result_desc, finishing, map_filter_project, - read_hold, + Some(read_hold), target_replica, peek_response_tx, ) @@ -1083,19 +1091,7 @@ impl InstanceState { F: FnOnce(&mut Instance) -> R + Send + 'static, R: Send + 'static, { - let (tx, rx) = oneshot::channel(); - let otel_ctx = OpenTelemetryContext::obtain(); - self.client - .send(Box::new(move |instance| { - let _span = debug_span!("instance::call_sync").entered(); - otel_ctx.attach_as_parent(); - - let result = f(instance); - let _ = tx.send(result); - })) - .expect("instance not dropped"); - - rx.await.expect("instance not dropped") + self.client.call_sync(f).await } /// Acquires a [`ReadHold`] for the identified compute collection. diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 1feb4a340e50c..1ca33b637404d 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -127,12 +127,12 @@ impl From for ReadPolicyError { } /// A command sent to an [`Instance`] task. -pub type Command = Box) + Send>; +pub(super) type Command = Box) + Send>; -/// A client for an [`Instance`] task. +/// A client for an `Instance` task. #[derive(Clone, derivative::Derivative)] #[derivative(Debug)] -pub(super) struct Client { +pub struct Client { /// A sender for commands for the instance. command_tx: mpsc::UnboundedSender>, /// A sender for read hold changes for collections installed on the instance. @@ -141,20 +141,42 @@ pub(super) struct Client { } impl Client { - pub fn send(&self, command: Command) -> Result<(), SendError>> { + pub(super) fn send(&self, command: Command) -> Result<(), SendError>> { self.command_tx.send(command) } - pub fn read_hold_tx(&self) -> read_holds::ChangeTx { + pub(super) fn read_hold_tx(&self) -> read_holds::ChangeTx { Arc::clone(&self.read_hold_tx) } + + /// Call a method to be run on the instance task, by sending a message to the instance and + /// waiting for a response message. + pub(super) async fn call_sync(&self, f: F) -> R + where + F: FnOnce(&mut Instance) -> R + Send + 'static, + R: Send + 'static, + { + use tokio::sync::oneshot; + use tracing::debug_span; + let (tx, rx) = oneshot::channel(); + let otel_ctx = OpenTelemetryContext::obtain(); + self.send(Box::new(move |instance| { + let _span = debug_span!("instance::call_sync").entered(); + otel_ctx.attach_as_parent(); + let result = f(instance); + let _ = tx.send(result); + })) + .expect("instance not dropped"); + + rx.await.expect("instance not dropped") + } } impl Client where T: ComputeControllerTimestamp, { - pub fn spawn( + pub(super) fn spawn( id: ComputeInstanceId, build_info: &'static BuildInfo, storage: StorageCollections, @@ -204,6 +226,50 @@ where read_hold_tx, } } + + /// Acquires a `ReadHold` and collection write frontier for each of the identified compute + /// collections. + pub async fn acquire_read_holds_and_collection_write_frontiers( + &self, + ids: Vec, + ) -> Result, Antichain)>, CollectionMissing> { + self.call_sync(move |i| i.acquire_read_holds_and_collection_write_frontiers(ids)) + .await + } + + /// Issue a peek by calling into the instance task synchronously, letting the instance acquire + /// read holds if none are provided. This ensures the read holds are established before + /// returning to the caller. + pub async fn peek_call_sync( + &self, + peek_target: PeekTarget, + literal_constraints: Option>, + uuid: Uuid, + timestamp: T, + result_desc: RelationDesc, + finishing: RowSetFinishing, + map_filter_project: mz_expr::SafeMfpPlan, + read_hold: Option>, + target_replica: Option, + peek_response_tx: oneshot::Sender, + ) { + self.call_sync(move |i| { + i.peek( + peek_target, + literal_constraints, + uuid, + timestamp, + result_desc, + finishing, + map_filter_project, + read_hold, + target_replica, + peek_response_tx, + ) + .expect("validated by instance"); + }) + .await + } } /// A response from a replica, composed of a replica ID, the replica's current epoch, and the @@ -966,6 +1032,11 @@ impl Instance { ]); Ok(serde_json::Value::Object(map)) } + + /// Reports the current write frontier for the identified compute collection. + fn collection_write_frontier(&self, id: GlobalId) -> Result, CollectionMissing> { + Ok(self.collection(id)?.write_frontier()) + } } impl Instance @@ -1640,14 +1711,29 @@ where result_desc: RelationDesc, finishing: RowSetFinishing, map_filter_project: mz_expr::SafeMfpPlan, - mut read_hold: ReadHold, + read_hold: Option>, target_replica: Option, peek_response_tx: oneshot::Sender, ) -> Result<(), PeekError> { use PeekError::*; - // Downgrade the provided read hold to the peek time. + // Acquire a read hold if one was not provided. let target_id = peek_target.id(); + let mut read_hold = match read_hold { + Some(h) => h, + None => match &peek_target { + PeekTarget::Index { id } => self + .acquire_read_hold(*id) + .map_err(|_| ReadHoldInsufficient(target_id))?, + PeekTarget::Persist { .. } => { + // For persist peeks, the controller should provide a storage read hold. + // We don't support acquiring it here. + return Err(ReadHoldInsufficient(target_id)); + } + }, + }; + + // Downgrade the provided (or acquired) read hold to the peek time. if read_hold.id() != target_id { return Err(ReadHoldIdMismatch(read_hold.id())); } @@ -2321,6 +2407,45 @@ where } } + /// Acquires a `ReadHold` and collection write frontier for each of the identified compute + /// collections. + fn acquire_read_holds_and_collection_write_frontiers( + &self, + ids: Vec, + ) -> Result, Antichain)>, CollectionMissing> { + let mut result = Vec::new(); + for id in ids.into_iter() { + // TODO: This takes locks separately for each id. We should change these methods to be + // able to take a collection of ids. + result.push(( + id, + self.acquire_read_hold(id)?, + self.collection_write_frontier(id)?, + )); + } + Ok(result) + } + + /// Acquires a `ReadHold` for the identified compute collection. + /// + /// This mirrors the logic used by the controller-side `InstanceState::acquire_read_hold`, + /// but executes on the instance task itself. + fn acquire_read_hold(&self, id: GlobalId) -> Result, CollectionMissing> { + // Similarly to InstanceState::acquire_read_hold and StorageCollections::acquire_read_holds, + // we acquire read holds at the earliest possible time rather than returning a copy + // of the implied read hold. This is so that dependents can acquire read holds on + // compute dependencies at frontiers that are held back by other read holds the caller + // has previously taken. + let collection = self.collection(id)?; + let since = collection.shared.lock_read_capabilities(|caps| { + let since = caps.frontier().to_owned(); + caps.update_iter(since.iter().map(|t| (t.clone(), 1))); + since + }); + let hold = ReadHold::new(id, since, Arc::clone(&self.read_hold_tx)); + Ok(hold) + } + /// Process pending maintenance work. /// /// This method is invoked periodically by the global controller. diff --git a/src/sql/src/optimizer_metrics.rs b/src/sql/src/optimizer_metrics.rs index 7f00c4cc901d4..318748411fb32 100644 --- a/src/sql/src/optimizer_metrics.rs +++ b/src/sql/src/optimizer_metrics.rs @@ -9,7 +9,6 @@ //! Metrics collected by the optimizer. -use std::cell::RefCell; use std::time::Duration; use mz_ore::metric; @@ -27,7 +26,7 @@ pub struct OptimizerMetrics { transform_total: IntCounterVec, /// Local storage of transform times; these are emitted as part of the /// log-line when end-to-end optimization times exceed the configured threshold. - transform_time_seconds: RefCell>>, + transform_time_seconds: std::collections::BTreeMap>, } impl OptimizerMetrics { @@ -58,7 +57,7 @@ impl OptimizerMetrics { help: "How many times a given transform was applied.", var_labels: ["transform"], )), - transform_time_seconds: RefCell::new(std::collections::BTreeMap::new()), + transform_time_seconds: std::collections::BTreeMap::new(), } } @@ -81,8 +80,7 @@ impl OptimizerMetrics { if duration > threshold { let transform_times = self .transform_time_seconds - .take() - .into_iter() + .iter() .map(|(k, v)| { ( k, @@ -121,8 +119,8 @@ impl OptimizerMetrics { self.transform_total.with_label_values(&[transform]).inc(); } - pub fn observe_transform_time(&self, transform: &str, duration: Duration) { - let mut transform_time_seconds = self.transform_time_seconds.borrow_mut(); + pub fn observe_transform_time(&mut self, transform: &str, duration: Duration) { + let transform_time_seconds = &mut self.transform_time_seconds; if let Some(times) = transform_time_seconds.get_mut(transform) { times.push(duration); } else { diff --git a/src/sql/src/session/vars/definitions.rs b/src/sql/src/session/vars/definitions.rs index 71c7b84e26b58..c2de13b29bf7b 100644 --- a/src/sql/src/session/vars/definitions.rs +++ b/src/sql/src/session/vars/definitions.rs @@ -2207,6 +2207,12 @@ feature_flags!( default: false, enable_for_item_parsing: true, }, + { + name: enable_frontend_peek_sequencing, // currently, changes only take effect for new sessions + desc: "Enables the new peek sequencing code, which does most of its work in the Adapter Frontend instead of the Coordinator main task.", + default: true, ////////// todo: default to false before merging the PR. (but figure out what to do in CI) + enable_for_item_parsing: false, + }, ); impl From<&super::SystemVars> for OptimizerFeatures { diff --git a/src/transform/src/lib.rs b/src/transform/src/lib.rs index 1a89315de4090..f2746e08f4dc1 100644 --- a/src/transform/src/lib.rs +++ b/src/transform/src/lib.rs @@ -125,7 +125,7 @@ pub struct TransformCtx<'a> { /// Transforms can use this field to communicate information outside the result plans. pub df_meta: &'a mut DataflowMetainfo, /// Metrics for the optimizer. - pub metrics: Option<&'a OptimizerMetrics>, + pub metrics: Option<&'a mut OptimizerMetrics>, /// The last hash of the query, if known. pub last_hash: BTreeMap, } @@ -143,7 +143,7 @@ impl<'a> TransformCtx<'a> { features: &'a OptimizerFeatures, typecheck_ctx: &'a SharedContext, df_meta: &'a mut DataflowMetainfo, - metrics: Option<&'a OptimizerMetrics>, + metrics: Option<&'a mut OptimizerMetrics>, global_id: Option, ) -> Self { Self { @@ -168,7 +168,7 @@ impl<'a> TransformCtx<'a> { features: &'a OptimizerFeatures, typecheck_ctx: &'a SharedContext, df_meta: &'a mut DataflowMetainfo, - metrics: Option<&'a OptimizerMetrics>, + metrics: Option<&'a mut OptimizerMetrics>, ) -> Self { Self { indexes, @@ -229,7 +229,7 @@ pub trait Transform: fmt::Debug { let duration = start.elapsed(); let hash_after = args.update_last_hash(relation); - if let Some(metrics) = args.metrics { + if let Some(metrics) = &mut args.metrics { let transform_name = self.name(); metrics.observe_transform_time(transform_name, duration); metrics.inc_transform(hash_before != hash_after, transform_name); diff --git a/test/cluster/mzcompose.py b/test/cluster/mzcompose.py index 3a0987c0060f8..66a96c586991a 100644 --- a/test/cluster/mzcompose.py +++ b/test/cluster/mzcompose.py @@ -3688,10 +3688,12 @@ def workflow_statement_logging(c: Composition, parser: WorkflowArgumentParser) - ): c.up("materialized") + # TODO: enable_frontend_peek_sequencing when it supports statement logging. c.sql( """ ALTER SYSTEM SET statement_logging_max_sample_rate = 1.0; ALTER SYSTEM SET statement_logging_default_sample_rate = 1.0; + ALTER SYSTEM SET enable_frontend_peek_sequencing = false; """, port=6877, user="mz_system", From 88c08cd8a70053f034a3388a38a2b6aca5d02dff Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Thu, 25 Sep 2025 18:53:44 +0200 Subject: [PATCH 02/55] Tweak transient id generation and update tests --- src/adapter/src/coord/sequencer/inner/peek.rs | 2 +- src/environmentd/tests/testdata/http/ws | 16 ++++++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/adapter/src/coord/sequencer/inner/peek.rs b/src/adapter/src/coord/sequencer/inner/peek.rs index 98c204356e7c4..4616cd440e1a1 100644 --- a/src/adapter/src/coord/sequencer/inner/peek.rs +++ b/src/adapter/src/coord/sequencer/inner/peek.rs @@ -283,7 +283,6 @@ impl Coordinator { let compute_instance = self .instance_snapshot(cluster.id()) .expect("compute instance does not exist"); - let (_, view_id) = self.allocate_transient_id(); let optimizer_config = optimize::OptimizerConfig::from(self.catalog().system_config()) .override_from(&self.catalog.get_cluster(cluster.id()).config.features()) .override_from(&explain_ctx); @@ -330,6 +329,7 @@ impl Coordinator { } }; copy_to_ctx.output_batch_count = Some(max_worker_count); + let (_, view_id) = self.allocate_transient_id(); // Build an optimizer for this COPY TO. Either::Right(optimize::copy_to::Optimizer::new( Arc::clone(&catalog), diff --git a/src/environmentd/tests/testdata/http/ws b/src/environmentd/tests/testdata/http/ws index eebfe76558b9d..84324745008d9 100644 --- a/src/environmentd/tests/testdata/http/ws +++ b/src/environmentd/tests/testdata/http/ws @@ -7,6 +7,14 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. +# Disabling temporarily only because the transient IDs are annoyingly changing. +ws-text +{"query": "ALTER SYSTEM SET enable_frontend_peek_sequencing = fase"} +---- +{"type":"CommandStarting","payload":{"has_rows":false,"is_streaming":false}} +{"type":"Error","payload":{"message":"permission denied to alter system","code":"42501","detail":"You must be the 'mz_system' role"}} +{"type":"ReadyForQuery","payload":"I"} + ws-text {"query": "bad sql"} ---- @@ -384,7 +392,7 @@ ws-text ws-text {"query": "SELECT 1"} ---- -{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish output=[#0]\\n Map (1)\\n Constant\\n - ()\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"Map\": {\n \"input\": {\n \"Constant\": {\n \"rows\": [\n {\n \"data\": []\n }\n ],\n \"typ\": {\n \"column_types\": [],\n \"keys\": []\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n }\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t74:\\n Finish output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t73\\n\\nt73:\\n Constant\\n - (1)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t74\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 73\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": [\n []\n ]\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t73\",\n \"plan\": {\n \"Constant\": {\n \"rows\": {\n \"Ok\": [\n [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n 1\n ]\n ]\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish output=[#0]\\n →Constant (1 rows)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"Constant\": [\n {\n \"Ok\": [\n [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n 1\n ]\n ]\n },\n {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {},\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"mz_catalog_server\",\n \"id\": {\n \"System\": 2\n }\n },\n \"redacted_sql\": \"SELECT ''\"\n}","code":"MZ001","severity":"notice"}} +{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish output=[#0]\\n Map (1)\\n Constant\\n - ()\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"Map\": {\n \"input\": {\n \"Constant\": {\n \"rows\": [\n {\n \"data\": []\n }\n ],\n \"typ\": {\n \"column_types\": [],\n \"keys\": []\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n }\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t58:\\n Finish output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t57\\n\\nt57:\\n Constant\\n - (1)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t58\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 57\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": [\n []\n ]\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t57\",\n \"plan\": {\n \"Constant\": {\n \"rows\": {\n \"Ok\": [\n [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n 1\n ]\n ]\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish output=[#0]\\n →Constant (1 rows)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"Constant\": [\n {\n \"Ok\": [\n [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n 1\n ]\n ]\n },\n {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {},\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"mz_catalog_server\",\n \"id\": {\n \"System\": 2\n }\n },\n \"redacted_sql\": \"SELECT ''\"\n}","code":"MZ001","severity":"notice"}} {"type":"CommandStarting","payload":{"has_rows":true,"is_streaming":false}} {"type":"Rows","payload":{"columns":[{"name":"?column?","type_oid":23,"type_len":4,"type_mod":-1}]}} {"type":"Row","payload":["1"]} @@ -394,7 +402,7 @@ ws-text ws-text {"query": "SELECT 1 / 0"} ---- -{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish output=[#0]\\n Map ((1 / 0))\\n Constant\\n - ()\\n\\nTarget cluster: quickstart\\n\",\n \"json\": {\n \"Map\": {\n \"input\": {\n \"Constant\": {\n \"rows\": [\n {\n \"data\": []\n }\n ],\n \"typ\": {\n \"column_types\": [],\n \"keys\": []\n }\n }\n },\n \"scalars\": [\n {\n \"CallBinary\": {\n \"func\": {\n \"DivInt32\": null\n },\n \"expr1\": {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"expr2\": {\n \"Literal\": [\n {\n \"data\": [\n 44\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"name\": null\n }\n }\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t77:\\n Finish output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t76\\n\\nt76:\\n Error \\\"division by zero\\\"\\n\\nTarget cluster: quickstart\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t77\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 76\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t76\",\n \"plan\": {\n \"Constant\": {\n \"rows\": {\n \"Err\": \"DivisionByZero\"\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish output=[#0]\\n →Constant (error: \\\"division by zero\\\")\\n\\nTarget cluster: quickstart\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"Constant\": [\n {\n \"Err\": \"DivisionByZero\"\n },\n {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {},\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"quickstart\",\n \"id\": {\n \"User\": 1\n }\n },\n \"redacted_sql\": \"SELECT '' / ''\"\n}","code":"MZ001","severity":"notice"}} +{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish output=[#0]\\n Map ((1 / 0))\\n Constant\\n - ()\\n\\nTarget cluster: quickstart\\n\",\n \"json\": {\n \"Map\": {\n \"input\": {\n \"Constant\": {\n \"rows\": [\n {\n \"data\": []\n }\n ],\n \"typ\": {\n \"column_types\": [],\n \"keys\": []\n }\n }\n },\n \"scalars\": [\n {\n \"CallBinary\": {\n \"func\": {\n \"DivInt32\": null\n },\n \"expr1\": {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"expr2\": {\n \"Literal\": [\n {\n \"data\": [\n 44\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"name\": null\n }\n }\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t60:\\n Finish output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t59\\n\\nt59:\\n Error \\\"division by zero\\\"\\n\\nTarget cluster: quickstart\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t60\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 59\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t59\",\n \"plan\": {\n \"Constant\": {\n \"rows\": {\n \"Err\": \"DivisionByZero\"\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish output=[#0]\\n →Constant (error: \\\"division by zero\\\")\\n\\nTarget cluster: quickstart\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"Constant\": [\n {\n \"Err\": \"DivisionByZero\"\n },\n {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {},\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"quickstart\",\n \"id\": {\n \"User\": 1\n }\n },\n \"redacted_sql\": \"SELECT '' / ''\"\n}","code":"MZ001","severity":"notice"}} {"type":"CommandStarting","payload":{"has_rows":false,"is_streaming":false}} {"type":"Error","payload":{"message":"division by zero","code":"XX000"}} {"type":"ReadyForQuery","payload":"I"} @@ -402,7 +410,7 @@ ws-text ws-text {"query": "SELECT 1 FROM mz_sources LIMIT 1"} ---- -{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish limit=1 output=[#0]\\n Project (#15)\\n Map (1)\\n Get mz_catalog.mz_sources\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t80:\\n Finish limit=1 output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t79\\n\\nt79:\\n Project (#15)\\n Map (1)\\n ReadIndex on=mz_sources mz_sources_ind=[*** full scan ***]\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t80\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 79\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t79\",\n \"plan\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n },\n \"access_strategy\": {\n \"Index\": [\n [\n {\n \"System\": 751\n },\n \"FullScan\"\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"Ok\": {\n \"data\": [\n 45,\n 1\n ]\n }\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish limit=1 output=[#0]\\n →Map/Filter/Project\\n Project: #15\\n Map: 1\\n →Indexed mz_catalog.mz_sources (using mz_catalog.mz_sources_ind)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"PeekExisting\": [\n {\n \"System\": 469\n },\n {\n \"System\": 751\n },\n null,\n {\n \"mfp\": {\n \"expressions\": [\n {\n \"Literal\": [\n {\n \"Ok\": {\n \"data\": [\n 45,\n 1\n ]\n }\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ],\n \"predicates\": [],\n \"projection\": [\n 15\n ],\n \"input_arity\": 15\n }\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {\n \"s751\": {\n \"name\": {\n \"schema\": \"mz_catalog\",\n \"item\": \"mz_sources_ind\"\n },\n \"type\": \"compute\"\n }\n },\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"mz_catalog_server\",\n \"id\": {\n \"System\": 2\n }\n },\n \"redacted_sql\": \"SELECT '' FROM [s469 AS mz_catalog.mz_sources] LIMIT ''\"\n}","code":"MZ001","severity":"notice"}} +{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish limit=1 output=[#0]\\n Project (#15)\\n Map (1)\\n Get mz_catalog.mz_sources\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t62:\\n Finish limit=1 output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t61\\n\\nt61:\\n Project (#15)\\n Map (1)\\n ReadIndex on=mz_sources mz_sources_ind=[*** full scan ***]\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t62\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 61\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t61\",\n \"plan\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n },\n \"access_strategy\": {\n \"Index\": [\n [\n {\n \"System\": 751\n },\n \"FullScan\"\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"Ok\": {\n \"data\": [\n 45,\n 1\n ]\n }\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish limit=1 output=[#0]\\n →Map/Filter/Project\\n Project: #15\\n Map: 1\\n →Indexed mz_catalog.mz_sources (using mz_catalog.mz_sources_ind)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"PeekExisting\": [\n {\n \"System\": 469\n },\n {\n \"System\": 751\n },\n null,\n {\n \"mfp\": {\n \"expressions\": [\n {\n \"Literal\": [\n {\n \"Ok\": {\n \"data\": [\n 45,\n 1\n ]\n }\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ],\n \"predicates\": [],\n \"projection\": [\n 15\n ],\n \"input_arity\": 15\n }\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {\n \"s751\": {\n \"name\": {\n \"schema\": \"mz_catalog\",\n \"item\": \"mz_sources_ind\"\n },\n \"type\": \"compute\"\n }\n },\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"mz_catalog_server\",\n \"id\": {\n \"System\": 2\n }\n },\n \"redacted_sql\": \"SELECT '' FROM [s469 AS mz_catalog.mz_sources] LIMIT ''\"\n}","code":"MZ001","severity":"notice"}} {"type":"CommandStarting","payload":{"has_rows":true,"is_streaming":false}} {"type":"Rows","payload":{"columns":[{"name":"?column?","type_oid":23,"type_len":4,"type_mod":-1}]}} {"type":"Row","payload":["1"]} @@ -412,7 +420,7 @@ ws-text ws-text {"query": "SELECT 1 / 0 FROM mz_sources LIMIT 1"} ---- -{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish limit=1 output=[#0]\\n Project (#15)\\n Map ((1 / 0))\\n Get mz_catalog.mz_sources\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"CallBinary\": {\n \"func\": {\n \"DivInt32\": null\n },\n \"expr1\": {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"expr2\": {\n \"Literal\": [\n {\n \"data\": [\n 44\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"name\": null\n }\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t83:\\n Finish limit=1 output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t82\\n\\nt82:\\n Project (#15)\\n Map (error(\\\"division by zero\\\"))\\n ReadIndex on=mz_sources mz_sources_ind=[*** full scan ***]\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t83\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 82\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t82\",\n \"plan\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n },\n \"access_strategy\": {\n \"Index\": [\n [\n {\n \"System\": 751\n },\n \"FullScan\"\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"Err\": \"DivisionByZero\"\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish limit=1 output=[#0]\\n →Map/Filter/Project\\n Project: #15\\n Map: error(\\\"division by zero\\\")\\n →Indexed mz_catalog.mz_sources (using mz_catalog.mz_sources_ind)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"PeekExisting\": [\n {\n \"System\": 469\n },\n {\n \"System\": 751\n },\n null,\n {\n \"mfp\": {\n \"expressions\": [\n {\n \"Literal\": [\n {\n \"Err\": \"DivisionByZero\"\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ],\n \"predicates\": [],\n \"projection\": [\n 15\n ],\n \"input_arity\": 15\n }\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {\n \"s751\": {\n \"name\": {\n \"schema\": \"mz_catalog\",\n \"item\": \"mz_sources_ind\"\n },\n \"type\": \"compute\"\n }\n },\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"mz_catalog_server\",\n \"id\": {\n \"System\": 2\n }\n },\n \"redacted_sql\": \"SELECT '' / '' FROM [s469 AS mz_catalog.mz_sources] LIMIT ''\"\n}","code":"MZ001","severity":"notice"}} +{"type":"Notice","payload":{"message":"{\n \"plans\": {\n \"raw\": {\n \"text\": \"Finish limit=1 output=[#0]\\n Project (#15)\\n Map ((1 / 0))\\n Get mz_catalog.mz_sources\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"CallBinary\": {\n \"func\": {\n \"DivInt32\": null\n },\n \"expr1\": {\n \"Literal\": [\n {\n \"data\": [\n 45,\n 1\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"expr2\": {\n \"Literal\": [\n {\n \"data\": [\n 44\n ]\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n },\n null\n ]\n },\n \"name\": null\n }\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n },\n \"optimized\": {\n \"global\": {\n \"text\": \"t64:\\n Finish limit=1 output=[#0]\\n ArrangeBy keys=[[#0]]\\n ReadGlobalFromSameDataflow t63\\n\\nt63:\\n Project (#15)\\n Map (error(\\\"division by zero\\\"))\\n ReadIndex on=mz_sources mz_sources_ind=[*** full scan ***]\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"t64\",\n \"plan\": {\n \"ArrangeBy\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"Transient\": 63\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ],\n \"keys\": []\n },\n \"access_strategy\": \"SameDataflow\"\n }\n },\n \"keys\": [\n [\n {\n \"Column\": [\n 0,\n null\n ]\n }\n ]\n ]\n }\n }\n },\n {\n \"id\": \"t63\",\n \"plan\": {\n \"Project\": {\n \"input\": {\n \"Map\": {\n \"input\": {\n \"Get\": {\n \"id\": {\n \"Global\": {\n \"System\": 469\n }\n },\n \"typ\": {\n \"column_types\": [\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"Oid\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": false\n },\n {\n \"scalar_type\": {\n \"Array\": \"MzAclItem\"\n },\n \"nullable\": false\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n },\n {\n \"scalar_type\": \"String\",\n \"nullable\": true\n }\n ],\n \"keys\": [\n [\n 0\n ],\n [\n 1\n ]\n ]\n },\n \"access_strategy\": {\n \"Index\": [\n [\n {\n \"System\": 751\n },\n \"FullScan\"\n ]\n ]\n }\n }\n },\n \"scalars\": [\n {\n \"Literal\": [\n {\n \"Err\": \"DivisionByZero\"\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ]\n }\n },\n \"outputs\": [\n 15\n ]\n }\n }\n }\n ],\n \"sources\": []\n }\n },\n \"fast_path\": {\n \"text\": \"Explained Query (fast path):\\n Finish limit=1 output=[#0]\\n →Map/Filter/Project\\n Project: #15\\n Map: error(\\\"division by zero\\\")\\n →Indexed mz_catalog.mz_sources (using mz_catalog.mz_sources_ind)\\n\\nTarget cluster: mz_catalog_server\\n\",\n \"json\": {\n \"plans\": [\n {\n \"id\": \"Explained Query (fast path)\",\n \"plan\": {\n \"PeekExisting\": [\n {\n \"System\": 469\n },\n {\n \"System\": 751\n },\n null,\n {\n \"mfp\": {\n \"expressions\": [\n {\n \"Literal\": [\n {\n \"Err\": \"DivisionByZero\"\n },\n {\n \"scalar_type\": \"Int32\",\n \"nullable\": false\n }\n ]\n }\n ],\n \"predicates\": [],\n \"projection\": [\n 15\n ],\n \"input_arity\": 15\n }\n }\n ]\n }\n }\n ],\n \"sources\": []\n }\n }\n }\n },\n \"insights\": {\n \"imports\": {\n \"s751\": {\n \"name\": {\n \"schema\": \"mz_catalog\",\n \"item\": \"mz_sources_ind\"\n },\n \"type\": \"compute\"\n }\n },\n \"fast_path_clusters\": {},\n \"fast_path_limit\": null,\n \"persist_count\": []\n },\n \"cluster\": {\n \"name\": \"mz_catalog_server\",\n \"id\": {\n \"System\": 2\n }\n },\n \"redacted_sql\": \"SELECT '' / '' FROM [s469 AS mz_catalog.mz_sources] LIMIT ''\"\n}","code":"MZ001","severity":"notice"}} {"type":"CommandStarting","payload":{"has_rows":false,"is_streaming":false}} {"type":"Error","payload":{"message":"division by zero","code":"XX000"}} {"type":"ReadyForQuery","payload":"I"} From cc5a49f4069d8b8c957eff438be932e570e2affe Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Thu, 25 Sep 2025 20:32:00 +0200 Subject: [PATCH 03/55] More test tweaks --- src/environmentd/src/test_util.rs | 11 +++++++++++ src/environmentd/tests/server.rs | 7 +++++++ test/sqllogictest/recursion_limit.slt | 8 ++++++++ 3 files changed, 26 insertions(+) diff --git a/src/environmentd/src/test_util.rs b/src/environmentd/src/test_util.rs index d9a2fe26dd8e7..3a446e2b8b2fa 100644 --- a/src/environmentd/src/test_util.rs +++ b/src/environmentd/src/test_util.rs @@ -831,6 +831,17 @@ impl TestServer { } } + pub async fn disable_feature_flags(&self, flags: &[&'static str]) { + let internal_client = self.connect().internal().await.unwrap(); + + for flag in flags { + internal_client + .batch_execute(&format!("ALTER SYSTEM SET {} = false;", flag)) + .await + .unwrap(); + } + } + pub fn ws_addr(&self) -> Uri { format!( "ws://{}/api/experimental/sql", diff --git a/src/environmentd/tests/server.rs b/src/environmentd/tests/server.rs index 7a5efafc41c1e..cf14fbfd32046 100644 --- a/src/environmentd/tests/server.rs +++ b/src/environmentd/tests/server.rs @@ -183,6 +183,10 @@ fn setup_statement_logging_core( "statement_logging_use_reproducible_rng".to_string(), "true".to_string(), ) + .with_system_parameter_default( + "enable_frontend_peek_sequencing".to_string(), + "false".to_string(), + ) .start_blocking(); let client = server.connect(postgres::NoTls).unwrap(); (server, client) @@ -721,6 +725,9 @@ fn test_statement_logging_sampling_constrained() { #[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] async fn test_statement_logging_unsampled_metrics() { let server = test_util::TestHarness::default().start().await; + server + .disable_feature_flags(&["enable_frontend_peek_sequencing"]) + .await; let client = server.connect().await.unwrap(); // TODO[btv] diff --git a/test/sqllogictest/recursion_limit.slt b/test/sqllogictest/recursion_limit.slt index ac4e083c83996..d90e87ca126cc 100644 --- a/test/sqllogictest/recursion_limit.slt +++ b/test/sqllogictest/recursion_limit.slt @@ -10,6 +10,14 @@ # Tests to exercise recursion limits and stack overflow protection in # the query planner and optimizer. +# TODO: We should instead either figure out why we take more stack space, and/or adjust the test to be slightly smaller. +# I'm guessing that the difference must be small, and the test was just already very near the stack limit, so this +# is probably a non-issue. +simple conn=mz_system,user=mz_system +ALTER SYSTEM SET enable_frontend_peek_sequencing = false; +---- +COMPLETE 0 + query I SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1; ---- From daa00a53645b5a91443b2d52ecccb436b9cbf598 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 27 Sep 2025 13:27:27 +0200 Subject: [PATCH 04/55] Address some comments --- src/adapter/src/client.rs | 2 +- src/adapter/src/coord/command_handler.rs | 2 +- src/adapter/src/coord/sequencer.rs | 3 +-- src/adapter/src/coord/sequencer/inner.rs | 2 +- src/adapter/src/frontend_peek.rs | 16 ++++++---------- src/adapter/src/metrics.rs | 21 +++++++++++++-------- src/adapter/src/notice.rs | 9 --------- src/adapter/src/peek_client.rs | 10 +++++----- src/compute-client/src/controller.rs | 2 +- 9 files changed, 29 insertions(+), 38 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 5ee4dbf28f559..771c36732f4be 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -543,7 +543,7 @@ pub struct SessionClient { timeouts: Timeout, segment_client: Option, environment_id: EnvironmentId, - /// Thin client for frontend peek sequencing; populated at connection startup. + /// Client for frontend peek sequencing; populated at connection startup. peek_client: PeekClient, } diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index 9f80420974c64..a665ee17f53ca 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -275,7 +275,7 @@ impl Coordinator { let _ = tx.send( self.controller .compute - .thin_instance_client(instance_id) + .instance_client(instance_id) .map_err(|_| AdapterError::ConcurrentClusterDrop), ); } diff --git a/src/adapter/src/coord/sequencer.rs b/src/adapter/src/coord/sequencer.rs index a7be9afd751c9..383643ecd6114 100644 --- a/src/adapter/src/coord/sequencer.rs +++ b/src/adapter/src/coord/sequencer.rs @@ -993,8 +993,7 @@ pub(crate) fn emit_optimizer_notices( } session .metrics() - .optimization_notices() - .with_label_values(&[kind.metric_label()]) + .optimization_notices(&[kind.metric_label()]) .inc_by(1); } } diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index a1b9370bac6d0..ba99fb7b8bc78 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -100,6 +100,7 @@ use tracing::{Instrument, Span, info, warn}; use crate::catalog::{self, Catalog, ConnCatalog, DropObjectInfo, UpdatePrivilegeVariant}; use crate::command::{ExecuteResponse, Response}; use crate::coord::appends::{BuiltinTableAppendNotify, DeferredOp, DeferredPlan, PendingWriteTxn}; +use crate::coord::sequencer::emit_optimizer_notices; use crate::coord::{ AlterConnectionValidationReady, AlterSinkReadyContext, Coordinator, CreateConnectionValidationReady, DeferredPlanStatement, ExecuteContext, ExplainContext, @@ -140,7 +141,6 @@ macro_rules! return_if_err { }; } -use crate::coord::sequencer::emit_optimizer_notices; pub(super) use return_if_err; struct DropOps { diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 6f6e143091aa3..41abb7c9dde02 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -69,8 +69,7 @@ impl SessionClient { .span_context() .clone(); if span_context.is_valid() { - // We emit a slightly different notice here than in handle_execute. - session.add_notice(AdapterNotice::FrontendQueryTrace { + session.add_notice(AdapterNotice::QueryTrace { trace_id: span_context.trace_id(), }); } @@ -275,7 +274,7 @@ impl SessionClient { let oracle_read_ts = match timeline { Some(timeline) if needs_linearized_read_ts => { - let oracle = self.peek_client_mut().get_oracle(timeline).await?; + let oracle = self.peek_client_mut().ensure_oracle(timeline).await?; let oracle_read_ts = oracle.read_ts().await; Some(oracle_read_ts) } @@ -489,8 +488,7 @@ impl SessionClient { // from this function then we don't count the peek at all. session .metrics() - .query_total() - .with_label_values(&[session_type, stmt_type]) + .query_total(&[session_type, stmt_type]) .inc(); // # Now back to peek_finish @@ -570,8 +568,7 @@ impl SessionClient { session .metrics() - .determine_timestamp() - .with_label_values(&[ + .determine_timestamp(&[ match det.respond_immediately() { true => "true", false => "false", @@ -604,9 +601,8 @@ impl SessionClient { if let Some(serializable) = serializable_det.timestamp_context.timestamp() { session .metrics() - .timestamp_difference_for_strict_serializable_ms() - .with_label_values(&[ - compute_instance.to_string().as_ref(), + .timestamp_difference_for_strict_serializable_ms(&[ + &compute_instance.to_string().as_ref(), constraint_based.as_str(), ]) .observe(f64::cast_lossy(u64::from( diff --git a/src/adapter/src/metrics.rs b/src/adapter/src/metrics.rs index b9824a79e4e34..6ace2d9104ed3 100644 --- a/src/adapter/src/metrics.rs +++ b/src/adapter/src/metrics.rs @@ -13,6 +13,7 @@ use mz_ore::stats::{histogram_milliseconds_buckets, histogram_seconds_buckets}; use mz_sql::ast::{AstInfo, Statement, StatementKind, SubscribeOutput}; use mz_sql::session::user::User; use mz_sql_parser::ast::statement_kind_label_value; +use prometheus::core::{AtomicU64, GenericCounter}; use prometheus::{Histogram, HistogramVec, IntCounter, IntCounterVec, IntGaugeVec}; #[derive(Debug, Clone)] @@ -262,20 +263,24 @@ impl SessionMetrics { &self.session_startup_table_writes_seconds } - pub(crate) fn query_total(&self) -> &IntCounterVec { - &self.query_total + pub(crate) fn query_total(&self, label_values: &[&str]) -> GenericCounter { + self.query_total.with_label_values(label_values) } - pub(crate) fn determine_timestamp(&self) -> &IntCounterVec { - &self.determine_timestamp + pub(crate) fn determine_timestamp(&self, label_values: &[&str]) -> GenericCounter { + self.determine_timestamp.with_label_values(label_values) } - pub(crate) fn timestamp_difference_for_strict_serializable_ms(&self) -> &HistogramVec { - &self.timestamp_difference_for_strict_serializable_ms + pub(crate) fn timestamp_difference_for_strict_serializable_ms( + &self, + label_values: &[&str], + ) -> Histogram { + self.timestamp_difference_for_strict_serializable_ms + .with_label_values(label_values) } - pub(crate) fn optimization_notices(&self) -> &IntCounterVec { - &self.optimization_notices + pub(crate) fn optimization_notices(&self, label_values: &[&str]) -> GenericCounter { + self.optimization_notices.with_label_values(label_values) } } diff --git a/src/adapter/src/notice.rs b/src/adapter/src/notice.rs index 7609a6d549e6b..5a06f46f3b842 100644 --- a/src/adapter/src/notice.rs +++ b/src/adapter/src/notice.rs @@ -87,10 +87,6 @@ pub enum AdapterNotice { QueryTrace { trace_id: opentelemetry::trace::TraceId, }, - /// Like QueryTrace, but for the frontend peek sequencing. - FrontendQueryTrace { - trace_id: opentelemetry::trace::TraceId, - }, UnimplementedIsolationLevel { isolation_level: String, }, @@ -181,7 +177,6 @@ impl AdapterNotice { AdapterNotice::QueryTimestamp { .. } => Severity::Notice, AdapterNotice::EqualSubscribeBounds { .. } => Severity::Notice, AdapterNotice::QueryTrace { .. } => Severity::Notice, - AdapterNotice::FrontendQueryTrace { .. } => Severity::Notice, AdapterNotice::UnimplementedIsolationLevel { .. } => Severity::Notice, AdapterNotice::StrongSessionSerializable => Severity::Notice, AdapterNotice::BadStartupSetting { .. } => Severity::Notice, @@ -284,7 +279,6 @@ impl AdapterNotice { AdapterNotice::QueryTimestamp { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::EqualSubscribeBounds { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::QueryTrace { .. } => SqlState::SUCCESSFUL_COMPLETION, - AdapterNotice::FrontendQueryTrace { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::UnimplementedIsolationLevel { .. } => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::StrongSessionSerializable => SqlState::SUCCESSFUL_COMPLETION, AdapterNotice::BadStartupSetting { .. } => SqlState::SUCCESSFUL_COMPLETION, @@ -395,9 +389,6 @@ impl fmt::Display for AdapterNotice { AdapterNotice::QueryTrace { trace_id } => { write!(f, "trace id: {}", trace_id) } - AdapterNotice::FrontendQueryTrace { trace_id } => { - write!(f, "frontend trace id: {}", trace_id) - } AdapterNotice::UnimplementedIsolationLevel { isolation_level } => { write!( f, diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index cf8c87583d657..92dfc5fa626ad 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -41,7 +41,7 @@ pub type StorageCollectionsHandle = Arc< + Sync, >; -/// Clients needed for peek sequencing from the Adapter Frontend. +/// Clients needed for peek sequencing in the Adapter Frontend. #[derive(Debug)] pub struct PeekClient { coordinator_client: Client, @@ -81,7 +81,7 @@ impl PeekClient { } } - pub async fn get_compute_instance_client( + pub async fn ensure_compute_instance_client( &mut self, compute_instance: ComputeInstanceId, ) -> Result<&mut mz_compute_client::controller::instance::Client, AdapterError> { @@ -100,7 +100,7 @@ impl PeekClient { .expect("ensured above")) } - pub async fn get_oracle( + pub async fn ensure_oracle( &mut self, timeline: Timeline, ) -> Result<&mut Arc + Send + Sync>, AdapterError> { @@ -167,7 +167,7 @@ impl PeekClient { for (&instance_id, collection_ids) in &id_bundle.compute_ids { let client = self - .get_compute_instance_client(instance_id) + .ensure_compute_instance_client(instance_id) .await .expect("missing compute instance client"); @@ -274,7 +274,7 @@ impl PeekClient { // Issue peek to the instance let client = self - .get_compute_instance_client(compute_instance) + .ensure_compute_instance_client(compute_instance) .await .expect("missing compute instance client"); let peek_target = PeekTarget::Index { id: idx_id }; diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 9321789c04c14..338c4f87802b2 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -335,7 +335,7 @@ impl ComputeController { } /// Return an `instance::Client` for the indicated compute instance. - pub fn thin_instance_client( + pub fn instance_client( &self, id: ComputeInstanceId, ) -> Result, InstanceMissing> { From 938c1361bf81a4bf5550eb9468a3b7937d28c188 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 6 Oct 2025 13:29:44 +0200 Subject: [PATCH 05/55] Change TODOs to TODO(peek-seq) and minor things --- .../checks/all_checks/statement_logging.py | 2 +- src/adapter/src/client.rs | 2 +- src/adapter/src/frontend_peek.rs | 49 ++++++++++--------- src/adapter/src/peek_client.rs | 19 +++---- src/compute-client/src/controller/instance.rs | 4 +- test/cluster/mzcompose.py | 2 +- test/sqllogictest/recursion_limit.slt | 4 +- 7 files changed, 41 insertions(+), 41 deletions(-) diff --git a/misc/python/materialize/checks/all_checks/statement_logging.py b/misc/python/materialize/checks/all_checks/statement_logging.py index 85b7fa08f4a65..478de63fbe0b0 100644 --- a/misc/python/materialize/checks/all_checks/statement_logging.py +++ b/misc/python/materialize/checks/all_checks/statement_logging.py @@ -16,7 +16,7 @@ class StatementLogging(Check): def initialize(self) -> Testdrive: return Testdrive( dedent( - # TODO: enable_frontend_peek_sequencing when it supports statement logging. + # TODO(peek-seq): enable_frontend_peek_sequencing when it supports statement logging. """ $ postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET statement_logging_max_sample_rate TO 1.0 diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 771c36732f4be..7ab3996d6266a 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -696,7 +696,7 @@ impl SessionClient { // Attempt peek sequencing in the session task. // If unsupported, fall back to the Coordinator path. - // TODO: wire up cancel_future + // TODO(peek-seq): wire up cancel_future if let Some(resp) = self.try_frontend_peek(&portal_name).await? { debug!("frontend peek succeeded"); return Ok((resp, execute_started)); diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 41abb7c9dde02..45fc110bb17c2 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -47,7 +47,7 @@ impl SessionClient { session: &mut Session, ) -> Result, AdapterError> { if session.transaction().is_in_multi_statement_transaction() { - // TODO: handle multi-statement transactions + // TODO(peek-seq): handle multi-statement transactions debug!( "Bailing out from try_frontend_peek_inner, because is_in_multi_statement_transaction" ); @@ -55,7 +55,7 @@ impl SessionClient { } if session.vars().emit_timestamp_notice() { - // TODO: implement this. See end of peek_finish + // TODO(peek-seq): implement this. See end of peek_finish debug!("Bailing out from try_frontend_peek_inner, because emit_timestamp_notice"); return Ok(None); } @@ -78,21 +78,21 @@ impl SessionClient { // This is from handle_execute_inner, but we do it already here because of lifetime issues, // and also to be able to give a catalog to `verify_portal`. // - // TODO: This snapshot is wasted when we end up bailing out from the frontend peek + // TODO(peek-seq): This snapshot is wasted when we end up bailing out from the frontend peek // sequencing. I think the best way to solve this is with that optimization where we // continuously keep a catalog snapshot in the session, and only get a new one when the // catalog revision has changed, which we could see with an atomic read. let catalog = self.catalog_snapshot("try_frontend_peek").await; if let Err(_) = Coordinator::verify_portal(&*catalog, session, portal_name) { - // TODO: Don't fall back to the coordinator's peek sequencing here, but retire already. + // TODO(peek-seq): Don't fall back to the coordinator's peek sequencing here, but retire already. debug!( "Bailing out from try_frontend_peek_inner, because verify_portal returned an error" ); return Ok(None); } - // TODO: statement logging (and then re-enable it in workflow_statement_logging) + // TODO(peek-seq): statement logging (and then enable it in various tests) let (stmt, params) = { let portal = session .get_portal_unverified(portal_name) @@ -154,7 +154,7 @@ impl SessionClient { (cluster, cluster.id, &cluster.name) }; - // TODO: statement logging: set_statement_execution_cluster + // TODO(peek-seq): statement logging: set_statement_execution_cluster if let Err(e) = coord::catalog_serving::check_cluster_restrictions( target_cluster_name.as_str(), @@ -168,7 +168,7 @@ impl SessionClient { &conn_catalog, |_id| { // This is only used by `Plan::SideEffectingFunc`, so it is irrelevant for us here - // TODO: refactor `check_plan` to make this nicer + // TODO(peek-seq): refactor `check_plan` to make this nicer unreachable!() }, session, @@ -185,7 +185,8 @@ impl SessionClient { // (This is done slightly earlier in the normal peek sequencing, but we have to be past the // last use of `conn_catalog` here.) if let Some(_) = coord::appends::waiting_on_startup_appends(&*catalog, session, &plan) { - // TODO: Don't fall back to the coordinator's peek sequencing here, but call `defer_op`. + // TODO(peek-seq): Don't fall back to the coordinator's peek sequencing here, but call + // `defer_op`. debug!("Bailing out from try_frontend_peek_inner, because waiting_on_startup_appends"); return Ok(None); } @@ -195,8 +196,8 @@ impl SessionClient { // # From sequence_peek if session.vars().emit_plan_insights_notice() { - // TODO: We'll need to do this when we want the frontend peek sequencing to take over - // from the old sequencing code. + // TODO(peek-seq): We'll need to do this when we want the frontend peek sequencing to + // take over from the old sequencing code. debug!("Bailing out from try_frontend_peek_inner, because emit_plan_insights_notice"); return Ok(None); } @@ -243,8 +244,8 @@ impl SessionClient { .transpose()?; let source_ids = select_plan.source.depends_on(); - // TODO: validate_timeline_context can be expensive in real scenarios (not in simple - // benchmarks), because it traverses transitive dependencies even of indexed views and + // TODO(peek-seq): validate_timeline_context can be expensive in real scenarios (not in + // simple benchmarks), because it traverses transitive dependencies even of indexed views and // materialized views (also traversing their MIR plans). let mut timeline_context = catalog.validate_timeline_context(source_ids.iter().copied())?; if matches!(timeline_context, TimelineContext::TimestampIndependent) @@ -283,8 +284,8 @@ impl SessionClient { // # From peek_real_time_recency - // TODO: Real-time recency is slow anyhow, so we don't handle it in frontend peek sequencing - // for now. + // TODO(peek-seq): Real-time recency is slow anyhow, so we don't handle it in frontend peek + // sequencing for now. let vars = session.vars(); if vars.real_time_recency() && vars.transaction_isolation() == &IsolationLevel::StrictSerializable @@ -319,7 +320,7 @@ impl SessionClient { ) if in_immediate_multi_stmt_txn => (determination, None), _ => { let determine_bundle = if in_immediate_multi_stmt_txn { - // TODO: handle multi-statement transactions + // TODO(peek-seq): handle multi-statement transactions // needs timedomain_for, which needs DataflowBuilder / index oracle / sufficient_collections debug!( "Bailing out from try_frontend_peek_inner, because of in_immediate_multi_stmt_txn" @@ -348,7 +349,7 @@ impl SessionClient { // We don't need the read holds and shouldn't add them // to the txn. // - // TODO: Handle this within determine_timestamp. + // TODO(peek-seq): Handle this within determine_timestamp? drop(read_holds); None } @@ -378,7 +379,7 @@ impl SessionClient { let now = catalog.config().now.clone(); let select_plan = select_plan.clone(); - // TODO: if explain_ctx.needs_plan_insights() ... + // TODO(peek-seq): if explain_ctx.needs_plan_insights() ... let span = Span::current(); @@ -398,7 +399,7 @@ impl SessionClient { let optimization_finished_at = now(); - // TODO: plan_insights stuff + // TODO(peek-seq): plan_insights stuff Ok::<_, AdapterError>((global_lir_plan, optimization_finished_at)) }) @@ -422,11 +423,11 @@ impl SessionClient { // # From peek_finish - // TODO: statement logging + // TODO(peek-seq): statement logging let (peek_plan, df_meta, typ) = global_lir_plan.unapply(); - // TODO: plan_insights stuff + // TODO(peek-seq): plan_insights stuff // This match is based on what `implement_fast_path_peek_plan` supports. let fast_path_plan = match peek_plan { @@ -451,7 +452,7 @@ impl SessionClient { // # We do the second half of sequence_peek_timestamp, as mentioned above. - // TODO: txn_read_holds stuff. Add SessionClient::txn_read_holds. + // TODO(peek-seq): txn_read_holds stuff. Add SessionClient::txn_read_holds. // (This TODO is copied from the old peek sequencing.) // TODO: Checking for only `InTransaction` and not `Implied` (also `Started`?) seems @@ -482,8 +483,8 @@ impl SessionClient { })?; }; - // TODO: move this up to the beginning of the function when we have eliminated all the - // fallbacks to the old peek sequencing. Currently, it has to be here to avoid + // TODO(peek-seq): move this up to the beginning of the function when we have eliminated all + // the fallbacks to the old peek sequencing. Currently, it has to be here to avoid // double-counting a fallback situation, but this has the drawback that if we error out // from this function then we don't count the peek at all. session @@ -493,7 +494,7 @@ impl SessionClient { // # Now back to peek_finish - // TODO: statement logging + // TODO(peek-seq): statement logging let max_result_size = catalog.system_config().max_result_size(); diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 92dfc5fa626ad..8e86f6f24f651 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -55,7 +55,7 @@ pub struct PeekClient { pub optimizer_metrics: OptimizerMetrics, /// Per-timeline oracles from the coordinator. Lazily populated. oracles: BTreeMap + Send + Sync>>, - // TODO: This is initialized only at session startup. We'll be able to properly check + // TODO(peek-seq): This is initialized only at session startup. We'll be able to properly check // the actual feature flag value (without a Coordinator call) once we'll always have a catalog // snapshot at hand. pub enable_frontend_peek_sequencing: bool, @@ -207,8 +207,9 @@ impl PeekClient { /// /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. /// - /// TODO: add statement logging - /// TODO: cancellation (see pending_peeks/client_pending_peeks wiring in the old sequencing) + /// TODO(peek-seq): add statement logging + /// TODO(peek-seq): cancellation (see pending_peeks/client_pending_peeks wiring in the old + /// sequencing) pub async fn implement_fast_path_peek_plan( &mut self, fast_path: FastPathPlan, @@ -259,7 +260,7 @@ impl PeekClient { &row_set_finishing_seconds, ) { Ok((rows, _bytes)) => Ok(Coordinator::send_immediate_rows(rows)), - // TODO: make this a structured error. (also in the old sequencing) + // TODO(peek-seq): make this a structured error. (also in the old sequencing) Err(e) => Err(AdapterError::ResultSize(e)), } } @@ -308,8 +309,8 @@ impl PeekClient { ) .await; - // TODO: call `create_peek_response_stream` instead. For that, we'll need to pass in - // a PersistClient from afar. + // TODO(peek-seq): call `create_peek_response_stream` instead. For that, we'll need + // to pass in a PersistClient from afar. let peek_response_stream = async_stream::stream!({ match rows_rx.await { Ok(PeekResponse::Rows(rows)) => { @@ -326,7 +327,7 @@ impl PeekClient { } } Ok(PeekResponse::Stashed(_response)) => { - // TODO: support this (through `create_peek_response_stream`) + // TODO(peek-seq): support this (through `create_peek_response_stream`) yield coord::peek::PeekResponseUnary::Error("stashed peek responses not yet supported in frontend peek sequencing".into()); } Ok(PeekResponse::Error(err)) => { @@ -348,8 +349,8 @@ impl PeekClient { }) } FastPathPlan::PeekPersist(..) => { - // TODO: Implement this. (We currently bail out in `try_frontend_peek_inner`, - // similarly to slow-path peeks.) + // TODO(peek-seq): Implement this. (We currently bail out in + // `try_frontend_peek_inner`, similarly to slow-path peeks.) // Note that `Instance::peek` has the following comment: // "For persist peeks, the controller should provide a storage read hold. // We don't support acquiring it here." diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 1ca33b637404d..ff115783b5325 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -2415,8 +2415,8 @@ where ) -> Result, Antichain)>, CollectionMissing> { let mut result = Vec::new(); for id in ids.into_iter() { - // TODO: This takes locks separately for each id. We should change these methods to be - // able to take a collection of ids. + // TODO(peek-seq): This takes locks separately for each id. We should change these + // methods to be able to take a collection of ids. result.push(( id, self.acquire_read_hold(id)?, diff --git a/test/cluster/mzcompose.py b/test/cluster/mzcompose.py index 66a96c586991a..085bf49e161a4 100644 --- a/test/cluster/mzcompose.py +++ b/test/cluster/mzcompose.py @@ -3688,7 +3688,7 @@ def workflow_statement_logging(c: Composition, parser: WorkflowArgumentParser) - ): c.up("materialized") - # TODO: enable_frontend_peek_sequencing when it supports statement logging. + # TODO(peek-seq): enable_frontend_peek_sequencing when it supports statement logging. c.sql( """ ALTER SYSTEM SET statement_logging_max_sample_rate = 1.0; diff --git a/test/sqllogictest/recursion_limit.slt b/test/sqllogictest/recursion_limit.slt index d90e87ca126cc..3caaef398668a 100644 --- a/test/sqllogictest/recursion_limit.slt +++ b/test/sqllogictest/recursion_limit.slt @@ -10,9 +10,7 @@ # Tests to exercise recursion limits and stack overflow protection in # the query planner and optimizer. -# TODO: We should instead either figure out why we take more stack space, and/or adjust the test to be slightly smaller. -# I'm guessing that the difference must be small, and the test was just already very near the stack limit, so this -# is probably a non-issue. +# TODO(peek-seq): This is not working, because we look at this flag only at session start, which already happened! simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_frontend_peek_sequencing = false; ---- From 86806212c5a9d90b9aa7258efe185bbe546e9cfd Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 6 Oct 2025 14:14:25 +0200 Subject: [PATCH 06/55] Move the call logic into instance::Client and delete instance::Client::send --- src/compute-client/src/controller.rs | 11 +--------- src/compute-client/src/controller/instance.rs | 21 ++++++++++++++----- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 338c4f87802b2..e650fe0641b64 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -60,7 +60,6 @@ use timely::PartialOrder; use timely::progress::{Antichain, Timestamp}; use tokio::sync::{mpsc, oneshot}; use tokio::time::{self, MissedTickBehavior}; -use tracing::debug_span; use uuid::Uuid; use crate::controller::error::{ @@ -1075,15 +1074,7 @@ impl InstanceState { where F: FnOnce(&mut Instance) + Send + 'static, { - let otel_ctx = OpenTelemetryContext::obtain(); - self.client - .send(Box::new(move |instance| { - let _span = debug_span!("instance::call").entered(); - otel_ctx.attach_as_parent(); - - f(instance) - })) - .expect("instance not dropped"); + self.client.call(f) } async fn call_sync(&self, f: F) -> R diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index ff115783b5325..8898b42cded4b 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -48,6 +48,7 @@ use timely::progress::frontier::MutableAntichain; use timely::progress::{Antichain, ChangeBatch, Timestamp}; use tokio::sync::mpsc::error::SendError; use tokio::sync::{mpsc, oneshot}; +use tracing::debug_span; use uuid::Uuid; use crate::controller::error::{ @@ -141,14 +142,24 @@ pub struct Client { } impl Client { - pub(super) fn send(&self, command: Command) -> Result<(), SendError>> { - self.command_tx.send(command) - } - pub(super) fn read_hold_tx(&self) -> read_holds::ChangeTx { Arc::clone(&self.read_hold_tx) } + pub(super) fn call(&self, f: F) + where + F: FnOnce(&mut Instance) + Send + 'static, + { + let otel_ctx = OpenTelemetryContext::obtain(); + self.command_tx.send(Box::new(move |instance| { + let _span = debug_span!("instance::call").entered(); + otel_ctx.attach_as_parent(); + + f(instance) + })) + .expect("instance not dropped"); + } + /// Call a method to be run on the instance task, by sending a message to the instance and /// waiting for a response message. pub(super) async fn call_sync(&self, f: F) -> R @@ -160,7 +171,7 @@ impl Client { use tracing::debug_span; let (tx, rx) = oneshot::channel(); let otel_ctx = OpenTelemetryContext::obtain(); - self.send(Box::new(move |instance| { + self.command_tx.send(Box::new(move |instance| { let _span = debug_span!("instance::call_sync").entered(); otel_ctx.attach_as_parent(); let result = f(instance); From 459ecfdaf60bd7ba27d2f7b781f0bd6173539b45 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 6 Oct 2025 14:16:34 +0200 Subject: [PATCH 07/55] Address various minor comments --- src/adapter/src/optimize/dataflows.rs | 2 + src/adapter/src/peek_client.rs | 12 +++-- src/compute-client/src/controller/instance.rs | 52 +++++++++---------- 3 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/adapter/src/optimize/dataflows.rs b/src/adapter/src/optimize/dataflows.rs index 7ebb0c302127e..afe1429f28dfc 100644 --- a/src/adapter/src/optimize/dataflows.rs +++ b/src/adapter/src/optimize/dataflows.rs @@ -56,6 +56,8 @@ pub struct ComputeInstanceSnapshot { instance_id: ComputeInstanceId, /// The collections that exist on this compute instance. If it's None, then any collection that /// a caller asks us about is considered to exist. + /// TODO: Remove this completely once all callers are able to handle suddenly missing + /// collections, in which case we won't need a `ComputeInstanceSnapshot` at all. collections: Option>, } diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 8e86f6f24f651..2ae3b8c41552f 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -46,6 +46,9 @@ pub type StorageCollectionsHandle = Arc< pub struct PeekClient { coordinator_client: Client, /// Channels to talk to each compute Instance task directly. Lazily populated. + /// Note that these are never cleaned up. In theory, this could lead to a very slow memory leak + /// if a long-running user session keeps peeking on clusters that are being created and dropped + /// in a hot loop. Hopefully this won't occur any time soon. compute_instances: BTreeMap>, /// Handle to storage collections for reading frontiers and policies. @@ -87,7 +90,7 @@ impl PeekClient { ) -> Result<&mut mz_compute_client::controller::instance::Client, AdapterError> { if !self.compute_instances.contains_key(&compute_instance) { let client = self - .send_to_coordinator(|tx| Command::GetComputeInstanceClient { + .call_coordinator(|tx| Command::GetComputeInstanceClient { instance_id: compute_instance, tx, }) @@ -106,7 +109,7 @@ impl PeekClient { ) -> Result<&mut Arc + Send + Sync>, AdapterError> { if !self.oracles.contains_key(&timeline) { let oracle = self - .send_to_coordinator(|tx| Command::GetOracle { + .call_coordinator(|tx| Command::GetOracle { timeline: timeline.clone(), tx, }) @@ -116,7 +119,7 @@ impl PeekClient { Ok(self.oracles.get_mut(&timeline).expect("ensured above")) } - async fn send_to_coordinator(&self, f: F) -> T + async fn call_coordinator(&self, f: F) -> T where F: FnOnce(oneshot::Sender) -> Command, { @@ -283,7 +286,7 @@ impl PeekClient { let map_filter_project = mfp; let finishing_for_instance = finishing.clone(); client - .peek_call_sync( + .peek( peek_target, literal_vec, uuid, @@ -303,7 +306,6 @@ impl PeekClient { // to let `Instance::peek` acquire the read holds, then we might acquire // read holds too late, leading to panic if the Coordinator drops its own // read holds in the meantime, e.g., due to cancelling the peek.) - None, target_replica, rows_tx, ) diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 8898b42cded4b..5d7b96b339bb3 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -128,7 +128,7 @@ impl From for ReadPolicyError { } /// A command sent to an [`Instance`] task. -pub(super) type Command = Box) + Send>; +type Command = Box) + Send>; /// A client for an `Instance` task. #[derive(Clone, derivative::Derivative)] @@ -151,12 +151,13 @@ impl Client { F: FnOnce(&mut Instance) + Send + 'static, { let otel_ctx = OpenTelemetryContext::obtain(); - self.command_tx.send(Box::new(move |instance| { - let _span = debug_span!("instance::call").entered(); - otel_ctx.attach_as_parent(); + self.command_tx + .send(Box::new(move |instance| { + let _span = debug_span!("instance::call").entered(); + otel_ctx.attach_as_parent(); - f(instance) - })) + f(instance) + })) .expect("instance not dropped"); } @@ -167,17 +168,16 @@ impl Client { F: FnOnce(&mut Instance) -> R + Send + 'static, R: Send + 'static, { - use tokio::sync::oneshot; - use tracing::debug_span; let (tx, rx) = oneshot::channel(); let otel_ctx = OpenTelemetryContext::obtain(); - self.command_tx.send(Box::new(move |instance| { - let _span = debug_span!("instance::call_sync").entered(); - otel_ctx.attach_as_parent(); - let result = f(instance); - let _ = tx.send(result); - })) - .expect("instance not dropped"); + self.command_tx + .send(Box::new(move |instance| { + let _span = debug_span!("instance::call_sync").entered(); + otel_ctx.attach_as_parent(); + let result = f(instance); + let _ = tx.send(result); + })) + .expect("instance not dropped"); rx.await.expect("instance not dropped") } @@ -248,10 +248,10 @@ where .await } - /// Issue a peek by calling into the instance task synchronously, letting the instance acquire - /// read holds if none are provided. This ensures the read holds are established before - /// returning to the caller. - pub async fn peek_call_sync( + /// Issue a peek by calling into the instance task, letting the instance acquire read holds if + /// none are provided. This ensures the read holds are established before returning to the + /// caller. + pub async fn peek( &self, peek_target: PeekTarget, literal_constraints: Option>, @@ -260,7 +260,6 @@ where result_desc: RelationDesc, finishing: RowSetFinishing, map_filter_project: mz_expr::SafeMfpPlan, - read_hold: Option>, target_replica: Option, peek_response_tx: oneshot::Sender, ) { @@ -273,7 +272,7 @@ where result_desc, finishing, map_filter_project, - read_hold, + None, target_replica, peek_response_tx, ) @@ -2426,8 +2425,6 @@ where ) -> Result, Antichain)>, CollectionMissing> { let mut result = Vec::new(); for id in ids.into_iter() { - // TODO(peek-seq): This takes locks separately for each id. We should change these - // methods to be able to take a collection of ids. result.push(( id, self.acquire_read_hold(id)?, @@ -2671,10 +2668,11 @@ pub(super) struct SharedCollectionState { /// This accumulation contains the capabilities held by all [`ReadHold`]s given out for the /// collection, including `implied_read_hold` and `warmup_read_hold`. /// - /// NOTE: This field may only be modified by [`Instance::apply_read_hold_change`] and - /// `ComputeController::acquire_read_hold`. Nobody else should modify read capabilities - /// directly. Instead, collection users should manage read holds through [`ReadHold`] objects - /// acquired through `ComputeController::acquire_read_hold`. + /// NOTE: This field may only be modified by [`Instance::apply_read_hold_change`], + /// [`Instance::acquire_read_hold`], and `ComputeController::acquire_read_hold`. + /// Nobody else should modify read capabilities directly. Instead, collection users should + /// manage read holds through [`ReadHold`] objects acquired through + /// `ComputeController::acquire_read_hold`. /// /// TODO(teskje): Restructure the code to enforce the above in the type system. read_capabilities: Arc>>, From 8a25df94f21141417d6c86c0215b9ebfe2cfbc0f Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Wed, 8 Oct 2025 15:23:25 +0200 Subject: [PATCH 08/55] Always pass read holds in to Instance::peek --- src/adapter/src/frontend_peek.rs | 5 ---- src/adapter/src/peek_client.rs | 26 +++++++++---------- src/compute-client/src/controller.rs | 2 +- src/compute-client/src/controller/instance.rs | 18 +++---------- 4 files changed, 17 insertions(+), 34 deletions(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 45fc110bb17c2..c54fb65ed7e88 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -308,8 +308,6 @@ impl SessionClient { .in_immediate_multi_stmt_txn(&select_plan.when); // Fetch or generate a timestamp for this query and fetch or acquire read holds. - // We need to keep these read holds until `implement_fast_path_peek_plan`, which acquires - // its own read holds. let (determination, read_holds) = match session.get_transaction_timestamp_determination() { // Use the transaction's timestamp if it exists and this isn't an AS OF query. Some( @@ -517,9 +515,6 @@ impl SessionClient { ) .await?; - // Ok to lose `_read_holds` at this point, because `implement_fast_path_peek_plan` acquired - // its own read holds before returning. - Ok(Some(resp)) } diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 2ae3b8c41552f..f0d39452d97bd 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -210,6 +210,11 @@ impl PeekClient { /// /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. /// + /// Note: `input_read_holds` is an Option, because we don't need it for constant queries. + /// `input_read_holds` has holds for all inputs. For fast-path peeks, this includes the peek. + /// target For slow-path peeks (to be implemented later), we'll need to additionally call into + /// the Controller to acquire a hold on the peek target after we create the dataflow. + /// /// TODO(peek-seq): add statement logging /// TODO(peek-seq): cancellation (see pending_peeks/client_pending_peeks wiring in the old /// sequencing) @@ -224,7 +229,7 @@ impl PeekClient { max_result_size: u64, max_returned_query_size: Option, row_set_finishing_seconds: Histogram, - _read_holds: Option>, + input_read_holds: Option>, ) -> Result { match fast_path { // If the dataflow optimizes to a constant expression, we can immediately return the result. @@ -285,6 +290,12 @@ impl PeekClient { let literal_vec: Option> = literal_constraints; let map_filter_project = mfp; let finishing_for_instance = finishing.clone(); + let target_read_hold = input_read_holds + .expect("need a read hold for non-constant peeks") + .compute_holds + .get(&(compute_instance, idx_id)) + .expect("missing compute read hold on peek target") + .clone(); client .peek( peek_target, @@ -294,18 +305,7 @@ impl PeekClient { result_desc, finishing_for_instance, map_filter_project, - // We let `Instance::peek` acquire read holds instead of passing them in. - // This is different from the old peek sequencing: That code is able to - // acquire read holds in `ComputeController::peek` at no significant cost, - // because that is a thick client to the controller. Here, we only have a - // thin client, which would need to do a roundtrip to the controller task - // for acquiring read holds, so it's better to let the instance do it in the - // same call that kicks off the peek itself. (Another difference is that - // the old peek sequencing's `ComputeController::peek` does not wait for - // `Instance::peek` to finish before returning, which means that if it were - // to let `Instance::peek` acquire the read holds, then we might acquire - // read holds too late, leading to panic if the Coordinator drops its own - // read holds in the meantime, e.g., due to cancelling the peek.) + target_read_hold, target_replica, rows_tx, ) diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index e650fe0641b64..2447daebcd249 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -907,7 +907,7 @@ where result_desc, finishing, map_filter_project, - Some(read_hold), + read_hold, target_replica, peek_response_tx, ) diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 5d7b96b339bb3..8706ca1cbebee 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -260,6 +260,7 @@ where result_desc: RelationDesc, finishing: RowSetFinishing, map_filter_project: mz_expr::SafeMfpPlan, + target_read_hold: ReadHold, target_replica: Option, peek_response_tx: oneshot::Sender, ) { @@ -272,7 +273,7 @@ where result_desc, finishing, map_filter_project, - None, + target_read_hold, target_replica, peek_response_tx, ) @@ -1721,7 +1722,7 @@ where result_desc: RelationDesc, finishing: RowSetFinishing, map_filter_project: mz_expr::SafeMfpPlan, - read_hold: Option>, + mut read_hold: ReadHold, target_replica: Option, peek_response_tx: oneshot::Sender, ) -> Result<(), PeekError> { @@ -1729,19 +1730,6 @@ where // Acquire a read hold if one was not provided. let target_id = peek_target.id(); - let mut read_hold = match read_hold { - Some(h) => h, - None => match &peek_target { - PeekTarget::Index { id } => self - .acquire_read_hold(*id) - .map_err(|_| ReadHoldInsufficient(target_id))?, - PeekTarget::Persist { .. } => { - // For persist peeks, the controller should provide a storage read hold. - // We don't support acquiring it here. - return Err(ReadHoldInsufficient(target_id)); - } - }, - }; // Downgrade the provided (or acquired) read hold to the peek time. if read_hold.id() != target_id { From 2670d2158177c57113559dc079b9ac26be98aa75 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Wed, 8 Oct 2025 18:04:19 +0200 Subject: [PATCH 09/55] Fix read holds for constant indexes --- src/adapter/src/coord/timestamp_selection.rs | 2 +- src/adapter/src/frontend_peek.rs | 20 ++++++-------------- src/adapter/src/peek_client.rs | 3 +-- test/sqllogictest/indexes.slt | 12 ++++++++++++ 4 files changed, 20 insertions(+), 17 deletions(-) diff --git a/src/adapter/src/coord/timestamp_selection.rs b/src/adapter/src/coord/timestamp_selection.rs index 808baf6ee442e..2b2b8f2a5993b 100644 --- a/src/adapter/src/coord/timestamp_selection.rs +++ b/src/adapter/src/coord/timestamp_selection.rs @@ -56,7 +56,7 @@ pub enum TimestampContext { /// the `chosen_ts`. oracle_ts: Option, }, - /// Read is execute without a timeline or timestamp. + /// Read is executed without a timeline or timestamp. NoTimestamp, } diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index c54fb65ed7e88..ab4daa4c1c5d3 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -311,11 +311,15 @@ impl SessionClient { let (determination, read_holds) = match session.get_transaction_timestamp_determination() { // Use the transaction's timestamp if it exists and this isn't an AS OF query. Some( - determination @ TimestampDetermination { + _determination @ TimestampDetermination { timestamp_context: TimestampContext::TimelineTimestamp { .. }, .. }, - ) if in_immediate_multi_stmt_txn => (determination, None), + ) if in_immediate_multi_stmt_txn => { + // TODO(peek-seq): handle multi-statement transactions: return the above + // `determination` and the holds from txn_read_holds. + return Ok(None); + } _ => { let determine_bundle = if in_immediate_multi_stmt_txn { // TODO(peek-seq): handle multi-statement transactions @@ -340,18 +344,6 @@ impl SessionClient { real_time_recency_ts, ) .await?; - // We only need read holds if the read depends on a timestamp. - let read_holds = match determination.timestamp_context.timestamp() { - Some(_ts) => Some(read_holds), - None => { - // We don't need the read holds and shouldn't add them - // to the txn. - // - // TODO(peek-seq): Handle this within determine_timestamp? - drop(read_holds); - None - } - }; (determination, read_holds) } }; diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index f0d39452d97bd..2263b086768a3 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -229,7 +229,7 @@ impl PeekClient { max_result_size: u64, max_returned_query_size: Option, row_set_finishing_seconds: Histogram, - input_read_holds: Option>, + input_read_holds: ReadHolds, ) -> Result { match fast_path { // If the dataflow optimizes to a constant expression, we can immediately return the result. @@ -291,7 +291,6 @@ impl PeekClient { let map_filter_project = mfp; let finishing_for_instance = finishing.clone(); let target_read_hold = input_read_holds - .expect("need a read hold for non-constant peeks") .compute_holds .get(&(compute_instance, idx_id)) .expect("missing compute read hold on peek target") diff --git a/test/sqllogictest/indexes.slt b/test/sqllogictest/indexes.slt index 272f7f8cdd817..c3c3d65e94ab6 100644 --- a/test/sqllogictest/indexes.slt +++ b/test/sqllogictest/indexes.slt @@ -27,3 +27,15 @@ CREATE INDEX illegal_sys_ind ON mz_internal.mz_comments (id) db error: ERROR: cannot create index with unstable dependencies DETAIL: The object depends on the following unstable objects: mz_comments + +# Test an index on a constant view +statement ok +CREATE VIEW c1 AS SELECT 1; + +statement ok +CREATE INDEX i1 ON c1(); + +query I +SELECT * FROM c1; +---- +1 From 6647062e138c76b3ab527fbae556c4edca9021d8 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Thu, 9 Oct 2025 17:42:08 +0200 Subject: [PATCH 10/55] Temporarily opt out of the frontend peek sequencing in tests that rely on waiting_on_startup_appends --- src/adapter/src/coord/appends.rs | 2 ++ src/adapter/src/frontend_peek.rs | 8 +++++++- src/environmentd/src/test_util.rs | 11 +++++++++++ src/environmentd/tests/sql.rs | 12 ++++++++++++ 4 files changed, 32 insertions(+), 1 deletion(-) diff --git a/src/adapter/src/coord/appends.rs b/src/adapter/src/coord/appends.rs index 048c06bfd4faf..7adc4be79f9ec 100644 --- a/src/adapter/src/coord/appends.rs +++ b/src/adapter/src/coord/appends.rs @@ -926,6 +926,8 @@ pub struct GroupCommitPermit { /// So as an optimization we do not wait for these writes to complete. But if a [`Session`] tries /// to query any of these builtin objects, we need to block that query on the writes completing to /// maintain linearizability. +/// +/// Warning: this already clears the wait flag (i.e., it calls `clear_builtin_table_updates`). pub(crate) fn waiting_on_startup_appends( catalog: &Catalog, session: &mut Session, diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index ab4daa4c1c5d3..8b0a4f0b2bb4f 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -186,7 +186,13 @@ impl SessionClient { // last use of `conn_catalog` here.) if let Some(_) = coord::appends::waiting_on_startup_appends(&*catalog, session, &plan) { // TODO(peek-seq): Don't fall back to the coordinator's peek sequencing here, but call - // `defer_op`. + // `defer_op`. Needs `ExecuteContext`. + // This fallback is currently causing a bug: `waiting_on_startup_appends` has the + // side effect that it already clears the wait flag, and therefore the old peek + // sequencing that we fall back to here won't do waiting. This is tested by + // `test_mz_sessions` and `test_pg_cancel_dropped_role`, where I've disabled the + // frontend peek sequencing for now. This bug will just go away once we don't fall back + // to the old peek sequencing here, but properly call `defer_op` instead. debug!("Bailing out from try_frontend_peek_inner, because waiting_on_startup_appends"); return Ok(None); } diff --git a/src/environmentd/src/test_util.rs b/src/environmentd/src/test_util.rs index 3a446e2b8b2fa..282e414459a03 100644 --- a/src/environmentd/src/test_util.rs +++ b/src/environmentd/src/test_util.rs @@ -1173,6 +1173,17 @@ impl TestServerWithRuntime { } } + /// Disable LaunchDarkly feature flags. + pub fn disable_feature_flags(&self, flags: &[&'static str]) { + let mut internal_client = self.connect_internal(postgres::NoTls).unwrap(); + + for flag in flags { + internal_client + .batch_execute(&format!("ALTER SYSTEM SET {} = false;", flag)) + .unwrap(); + } + } + /// Return a [`postgres::Config`] for connecting to the __public__ SQL port of the running /// `environmentd` server. pub fn pg_config(&self) -> postgres::Config { diff --git a/src/environmentd/tests/sql.rs b/src/environmentd/tests/sql.rs index b59b0b7b5b388..11c95e20cbc9a 100644 --- a/src/environmentd/tests/sql.rs +++ b/src/environmentd/tests/sql.rs @@ -2591,6 +2591,11 @@ fn test_dont_drop_sinks_twice() { #[mz_ore::test] fn test_mz_sessions() { let server = test_util::TestHarness::default().start_blocking(); + // TODO(peek-seq): This would currently fail with the flag on, indicating a bug in the frontend + // peek sequencing. Re-enable this once we never fall back from the frontend peek sequencing to + // the old peek sequencing. See comment on the call to `waiting_on_startup_appends` in + // `frontend_peek.rs`. + server.disable_feature_flags(&["enable_frontend_peek_sequencing"]); let mut foo_client = server .pg_config() @@ -3145,6 +3150,13 @@ fn test_params() { #[mz_ore::test] fn test_pg_cancel_dropped_role() { let server = test_util::TestHarness::default().start_blocking(); + + // TODO(peek-seq): This would currently fail with the flag on, indicating a bug in the frontend + // peek sequencing. Re-enable this once we never fall back from the frontend peek sequencing to + // the old peek sequencing. See comment on the call to `waiting_on_startup_appends` in + // `frontend_peek.rs`. + server.disable_feature_flags(&["enable_frontend_peek_sequencing"]); + let dropped_role = "r1"; let mut query_client = server.connect(postgres::NoTls).unwrap(); From 84e1f13db7815e8fb98c3d0d78d581c4d2b7fbda Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Thu, 9 Oct 2025 17:59:10 +0200 Subject: [PATCH 11/55] Opt out of the frontend peek sequencing in test_expected_spans --- src/environmentd/tests/tracing.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/environmentd/tests/tracing.rs b/src/environmentd/tests/tracing.rs index a0d6ddea0d9ea..defa7ba60f21a 100644 --- a/src/environmentd/tests/tracing.rs +++ b/src/environmentd/tests/tracing.rs @@ -39,6 +39,13 @@ async fn test_expected_spans() { .start() .await; + // This test checks for specific functions of the old peek sequencing, so we disable the new + // peek sequencing for now. + // TODO(peek-seq): Modify the test to check for the new peek sequencing instead of the old one. + server + .disable_feature_flags(&["enable_frontend_peek_sequencing"]) + .await; + let client = server.connect().await.unwrap(); // Assert that there are no expected spans. From e6ea0593f8f34e085196e2592e36e9b1544fd71a Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Fri, 10 Oct 2025 16:09:11 +0200 Subject: [PATCH 12/55] Opt out of the frontend peek sequencing in test_peek_on_dropped_indexed_view --- src/environmentd/tests/sql.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/environmentd/tests/sql.rs b/src/environmentd/tests/sql.rs index 11c95e20cbc9a..f14fe5d981a86 100644 --- a/src/environmentd/tests/sql.rs +++ b/src/environmentd/tests/sql.rs @@ -3208,6 +3208,10 @@ fn test_pg_cancel_dropped_role() { fn test_peek_on_dropped_indexed_view() { let server = test_util::TestHarness::default().start_blocking(); + // TODO(peek-seq): This needs peek cancellation to work, which is not yet implemented in the + // new peek sequencing. + server.disable_feature_flags(&["enable_frontend_peek_sequencing"]); + let mut ddl_client = server.connect(postgres::NoTls).unwrap(); let mut peek_client = server.connect(postgres::NoTls).unwrap(); From 965c140603e97f3887d1dd385e96c6a00c2f46b8 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 10:19:40 +0200 Subject: [PATCH 13/55] Unsafe session invariant fix --- src/adapter/src/client.rs | 66 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 62 insertions(+), 4 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 7ab3996d6266a..a8551ce25be9f 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -1100,13 +1100,71 @@ impl SessionClient { ) -> Result, AdapterError> { if self.peek_client().enable_frontend_peek_sequencing { // We need to snatch the session out of self here, because if we just had a reference to it, - // then we couldn't do other things with self. - let mut session = self.session.take().expect("SessionClient invariant"); + // then we couldn't do other things with self. However, if this future is dropped before we + // reinsert the session, we must put it back to avoid "losing" the session. + // + // To make this cancel-safe, use a small RAII guard that owns the Session and reinserts it + // into self.session in its Drop implementation if we haven't already put it back. + struct SessionReinserter { + target: *mut Option, + session: Option, + } + impl SessionReinserter { + fn new(target: &mut Option, session: Session) -> Self { + SessionReinserter { + // Store a raw pointer to avoid borrow conflicts while we also borrow &mut self + // during try_frontend_peek_inner. We only ever touch this pointer in Drop or + // in reinsert_now after await completes. + target: target as *mut Option, + session: Some(session), + } + } + fn session_mut(&mut self) -> &mut Session { + self.session.as_mut().expect("session present") + } + fn reinsert_now(&mut self) { + if let Some(session) = self.session.take() { + // SAFETY: While this guard is alive we never access self.session through + // &mut self elsewhere. try_frontend_peek_inner takes a &mut Session directly, + // so no code reads self.session. Therefore writing back here is exclusive. + unsafe { + debug_assert!((*self.target).is_none(), "session slot not empty"); + *self.target = Some(session); + } + } + } + } + impl Drop for SessionReinserter { + fn drop(&mut self) { + // If the future was dropped before we manually reinserted the session, put it back now. + if let Some(session) = self.session.take() { + unsafe { + // See safety comment in reinsert_now. + // If the slot already contains a session (should not happen), prefer not to overwrite. + if (*self.target).is_none() { + *self.target = Some(session); + } else { + // Drop the session to avoid duplicating it. In practice this path should not occur. + // (We intentionally do nothing.) + } + } + } + } + } + // SAFETY: SessionReinserter contains a raw pointer to `self.session`. The future that + // holds this guard also holds `&mut self`, so `self` (and thus the pointer target) lives + // for the entire lifetime of the future. The future may move across threads, but the + // pointer remains valid and is only accessed from that future; there is no concurrent + // access. Therefore it is safe to mark the guard as Send. + unsafe impl Send for SessionReinserter {} + + let session = self.session.take().expect("SessionClient invariant"); + let mut reinserter = SessionReinserter::new(&mut self.session, session); let r = self - .try_frontend_peek_inner(portal_name, &mut session) + .try_frontend_peek_inner(portal_name, reinserter.session_mut()) .await; // Always put it back, even if we got an error. - self.session = Some(session); + reinserter.reinsert_now(); r } else { Ok(None) From 04101713b9d30e14581a4dabc4c3a21aadbf3d66 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 10:43:45 +0200 Subject: [PATCH 14/55] Avoid unsafe by moving try_frontend_peek_inner from SessionClient to PeekClient --- src/adapter/src/client.rs | 91 +++++++++++++++----------------- src/adapter/src/frontend_peek.rs | 19 ++++--- src/adapter/src/peek_client.rs | 2 +- 3 files changed, 53 insertions(+), 59 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index a8551ce25be9f..29e81987c8ff5 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -1103,68 +1103,63 @@ impl SessionClient { // then we couldn't do other things with self. However, if this future is dropped before we // reinsert the session, we must put it back to avoid "losing" the session. // - // To make this cancel-safe, use a small RAII guard that owns the Session and reinserts it - // into self.session in its Drop implementation if we haven't already put it back. - struct SessionReinserter { - target: *mut Option, + // To make this cancel-safe without unsafe code, reuse the oneshot guard pattern used in + // send_with_cancel: create a channel to return the Session, guard the receiver so dropping + // it after a send reinserts the Session into self.session, and ensure the sender sends the + // Session back in all paths (normal completion or drop). + + // Take ownership of the Session now. + let session = self.session.take().expect("SessionClient invariant"); + + // Destructure self to get mutable references to the session slot and the PeekClient, + // so we can operate on them independently without borrowing all of `self`. + let Self { session: client_session, peek_client, .. } = self; + + // Channel used to return the Session back into client_session. + let (tx, rx) = oneshot::channel::(); + + // Guarded receiver: if dropped after the Session has been sent, reinsert it. + let guarded_rx = rx.with_guard(|returned: Session| { + *client_session = Some(returned); + }); + + // RAII helper that ensures the Session is sent back via the channel. + struct SessionReturner { + tx: Option>, session: Option, } - impl SessionReinserter { - fn new(target: &mut Option, session: Session) -> Self { - SessionReinserter { - // Store a raw pointer to avoid borrow conflicts while we also borrow &mut self - // during try_frontend_peek_inner. We only ever touch this pointer in Drop or - // in reinsert_now after await completes. - target: target as *mut Option, - session: Some(session), - } - } + impl SessionReturner { fn session_mut(&mut self) -> &mut Session { self.session.as_mut().expect("session present") } - fn reinsert_now(&mut self) { - if let Some(session) = self.session.take() { - // SAFETY: While this guard is alive we never access self.session through - // &mut self elsewhere. try_frontend_peek_inner takes a &mut Session directly, - // so no code reads self.session. Therefore writing back here is exclusive. - unsafe { - debug_assert!((*self.target).is_none(), "session slot not empty"); - *self.target = Some(session); - } + fn send_now(&mut self) { + if let (Some(tx), Some(session)) = (self.tx.take(), self.session.take()) { + // Ignore send errors: if the receiver was already dropped, its guard will + // have attempted reinsertion if a value was present, and if it's still alive + // it will observe the value when we drop guarded_rx below. + let _ = tx.send(session); } } } - impl Drop for SessionReinserter { + impl Drop for SessionReturner { fn drop(&mut self) { - // If the future was dropped before we manually reinserted the session, put it back now. - if let Some(session) = self.session.take() { - unsafe { - // See safety comment in reinsert_now. - // If the slot already contains a session (should not happen), prefer not to overwrite. - if (*self.target).is_none() { - *self.target = Some(session); - } else { - // Drop the session to avoid duplicating it. In practice this path should not occur. - // (We intentionally do nothing.) - } - } + if let (Some(tx), Some(session)) = (self.tx.take(), self.session.take()) { + let _ = tx.send(session); } } } - // SAFETY: SessionReinserter contains a raw pointer to `self.session`. The future that - // holds this guard also holds `&mut self`, so `self` (and thus the pointer target) lives - // for the entire lifetime of the future. The future may move across threads, but the - // pointer remains valid and is only accessed from that future; there is no concurrent - // access. Therefore it is safe to mark the guard as Send. - unsafe impl Send for SessionReinserter {} - let session = self.session.take().expect("SessionClient invariant"); - let mut reinserter = SessionReinserter::new(&mut self.session, session); - let r = self - .try_frontend_peek_inner(portal_name, reinserter.session_mut()) + let mut returner = SessionReturner { tx: Some(tx), session: Some(session) }; + + let r = peek_client + .try_frontend_peek_inner(portal_name, returner.session_mut()) .await; - // Always put it back, even if we got an error. - reinserter.reinsert_now(); + + // Always send the Session back, even if we got an error. + returner.send_now(); + // Drop the guarded receiver now to immediately reinsert into self.session. + drop(guarded_rx); + r } else { Ok(None) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 8b0a4f0b2bb4f..0615935b231e5 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -35,12 +35,13 @@ use crate::optimize::Optimize; use crate::optimize::dataflows::{ComputeInstanceSnapshot, DataflowBuilder}; use crate::session::{Session, TransactionOps, TransactionStatus}; use crate::{ - AdapterError, AdapterNotice, CollectionIdBundle, ExecuteResponse, ReadHolds, SessionClient, - TimelineContext, TimestampContext, TimestampProvider, optimize, + AdapterError, AdapterNotice, CollectionIdBundle, ExecuteResponse, ReadHolds, + TimelineContext, TimestampContext, TimestampProvider, optimize, PeekClient, }; use crate::{coord, metrics}; +use crate::command::{Command, CatalogSnapshot}; -impl SessionClient { +impl PeekClient { pub(crate) async fn try_frontend_peek_inner( &mut self, portal_name: &str, @@ -82,7 +83,7 @@ impl SessionClient { // sequencing. I think the best way to solve this is with that optimization where we // continuously keep a catalog snapshot in the session, and only get a new one when the // catalog revision has changed, which we could see with an atomic read. - let catalog = self.catalog_snapshot("try_frontend_peek").await; + let CatalogSnapshot { catalog } = self.call_coordinator(|tx| Command::CatalogSnapshot { tx }).await; if let Err(_) = Coordinator::verify_portal(&*catalog, session, portal_name) { // TODO(peek-seq): Don't fall back to the coordinator's peek sequencing here, but retire already. @@ -214,8 +215,8 @@ impl SessionClient { let compute_instance_snapshot = ComputeInstanceSnapshot::new_without_collections(cluster.id()); - let (_, view_id) = self.peek_client().transient_id_gen.allocate_id(); - let (_, index_id) = self.peek_client().transient_id_gen.allocate_id(); + let (_, view_id) = self.transient_id_gen.allocate_id(); + let (_, index_id) = self.transient_id_gen.allocate_id(); let optimizer_config = optimize::OptimizerConfig::from(catalog.system_config()) .override_from(&catalog.get_cluster(cluster.id()).config.features()); @@ -234,7 +235,7 @@ impl SessionClient { view_id, index_id, optimizer_config, - self.peek_client().optimizer_metrics.clone(), + self.optimizer_metrics.clone(), ); let target_replica_name = session.vars().cluster_replica(); @@ -281,7 +282,7 @@ impl SessionClient { let oracle_read_ts = match timeline { Some(timeline) if needs_linearized_read_ts => { - let oracle = self.peek_client_mut().ensure_oracle(timeline).await?; + let oracle = self.ensure_oracle(timeline).await?; let oracle_read_ts = oracle.read_ts().await; Some(oracle_read_ts) } @@ -498,7 +499,6 @@ impl SessionClient { // Implement the peek, and capture the response. let resp = self - .peek_client_mut() .implement_fast_path_peek_plan( fast_path_plan, determination.timestamp_context.timestamp_or_default(), @@ -542,7 +542,6 @@ impl SessionClient { let isolation_level = session.vars().transaction_isolation(); let (read_holds, upper) = self - .peek_client_mut() .acquire_read_holds_and_collection_write_frontiers(id_bundle) .await .expect("missing collection"); diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 2263b086768a3..c140e593c870a 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -119,7 +119,7 @@ impl PeekClient { Ok(self.oracles.get_mut(&timeline).expect("ensured above")) } - async fn call_coordinator(&self, f: F) -> T + pub(crate) async fn call_coordinator(&self, f: F) -> T where F: FnOnce(oneshot::Sender) -> Command, { From cc7e9d2e51b661ecc5dc626f6b11c28f381f2ea4 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 10:52:43 +0200 Subject: [PATCH 15/55] Avoid involving GuardedReceiver --- src/adapter/src/client.rs | 66 +++++++++++++-------------------------- 1 file changed, 22 insertions(+), 44 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 29e81987c8ff5..c5cf47efae452 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -1099,68 +1099,46 @@ impl SessionClient { portal_name: &str, ) -> Result, AdapterError> { if self.peek_client().enable_frontend_peek_sequencing { - // We need to snatch the session out of self here, because if we just had a reference to it, - // then we couldn't do other things with self. However, if this future is dropped before we - // reinsert the session, we must put it back to avoid "losing" the session. - // - // To make this cancel-safe without unsafe code, reuse the oneshot guard pattern used in - // send_with_cancel: create a channel to return the Session, guard the receiver so dropping - // it after a send reinserts the Session into self.session, and ensure the sender sends the - // Session back in all paths (normal completion or drop). - - // Take ownership of the Session now. - let session = self.session.take().expect("SessionClient invariant"); - - // Destructure self to get mutable references to the session slot and the PeekClient, - // so we can operate on them independently without borrowing all of `self`. - let Self { session: client_session, peek_client, .. } = self; - - // Channel used to return the Session back into client_session. - let (tx, rx) = oneshot::channel::(); - - // Guarded receiver: if dropped after the Session has been sent, reinsert it. - let guarded_rx = rx.with_guard(|returned: Session| { - *client_session = Some(returned); - }); + // Take ownership of the session and split-borrow the pieces we need. + let Self { session: slot, peek_client, .. } = self; + let session = slot.take().expect("SessionClient invariant"); - // RAII helper that ensures the Session is sent back via the channel. - struct SessionReturner { - tx: Option>, + // RAII guard that always puts the session back into the slot. + struct SessionReinserter<'a> { + slot: &'a mut Option, session: Option, } - impl SessionReturner { + impl<'a> SessionReinserter<'a> { fn session_mut(&mut self) -> &mut Session { self.session.as_mut().expect("session present") } - fn send_now(&mut self) { - if let (Some(tx), Some(session)) = (self.tx.take(), self.session.take()) { - // Ignore send errors: if the receiver was already dropped, its guard will - // have attempted reinsertion if a value was present, and if it's still alive - // it will observe the value when we drop guarded_rx below. - let _ = tx.send(session); + fn put_back(&mut self) { + if let Some(sess) = self.session.take() { + debug_assert!(self.slot.is_none()); + *self.slot = Some(sess); } } } - impl Drop for SessionReturner { + impl<'a> Drop for SessionReinserter<'a> { fn drop(&mut self) { - if let (Some(tx), Some(session)) = (self.tx.take(), self.session.take()) { - let _ = tx.send(session); + if self.slot.is_none() { + if let Some(sess) = self.session.take() { + *self.slot = Some(sess); + } } } } - let mut returner = SessionReturner { tx: Some(tx), session: Some(session) }; + let mut guard = SessionReinserter { slot, session: Some(session) }; - let r = peek_client - .try_frontend_peek_inner(portal_name, returner.session_mut()) + let res = peek_client + .try_frontend_peek_inner(portal_name, guard.session_mut()) .await; - // Always send the Session back, even if we got an error. - returner.send_now(); - // Drop the guarded receiver now to immediately reinsert into self.session. - drop(guarded_rx); + // Ensure reinsertion on the normal path, too. + guard.put_back(); - r + res } else { Ok(None) } From bdc25a276ae53618453b4d7d47623d3717b79fff Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 10:55:35 +0200 Subject: [PATCH 16/55] minor --- src/adapter/src/client.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index c5cf47efae452..f36ce94dfc090 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -27,7 +27,7 @@ use mz_compute_types::ComputeInstanceId; use mz_ore::channel::OneshotReceiverExt; use mz_ore::collections::CollectionExt; use mz_ore::id_gen::{IdAllocator, IdAllocatorInnerBitSet, MAX_ORG_ID, org_id_conn_bits}; -use mz_ore::instrument; +use mz_ore::{instrument, soft_assert_or_log}; use mz_ore::now::{EpochMillis, NowFn, to_datetime}; use mz_ore::result::ResultExt; use mz_ore::task::AbortOnDropHandle; @@ -1103,7 +1103,7 @@ impl SessionClient { let Self { session: slot, peek_client, .. } = self; let session = slot.take().expect("SessionClient invariant"); - // RAII guard that always puts the session back into the slot. + // RAII guard that always puts the session back into `slot` (`SessionClient::session`). struct SessionReinserter<'a> { slot: &'a mut Option, session: Option, @@ -1114,7 +1114,7 @@ impl SessionClient { } fn put_back(&mut self) { if let Some(sess) = self.session.take() { - debug_assert!(self.slot.is_none()); + soft_assert_or_log!(self.slot.is_none(), "the `take` above removed it"); *self.slot = Some(sess); } } From ce9c8b0e24c3fad60167e789f23b5b0c4926ec54 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 11:01:25 +0200 Subject: [PATCH 17/55] Encapsulate catalog_snapshot --- src/adapter/src/frontend_peek.rs | 3 +-- src/adapter/src/peek_client.rs | 20 ++++++++++++++++++-- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 0615935b231e5..b24511e0d393c 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -39,7 +39,6 @@ use crate::{ TimelineContext, TimestampContext, TimestampProvider, optimize, PeekClient, }; use crate::{coord, metrics}; -use crate::command::{Command, CatalogSnapshot}; impl PeekClient { pub(crate) async fn try_frontend_peek_inner( @@ -83,7 +82,7 @@ impl PeekClient { // sequencing. I think the best way to solve this is with that optimization where we // continuously keep a catalog snapshot in the session, and only get a new one when the // catalog revision has changed, which we could see with an atomic read. - let CatalogSnapshot { catalog } = self.call_coordinator(|tx| Command::CatalogSnapshot { tx }).await; + let catalog = self.catalog_snapshot("try_frontend_peek_inner").await; if let Err(_) = Coordinator::verify_portal(&*catalog, session, portal_name) { // TODO(peek-seq): Don't fall back to the coordinator's peek sequencing here, but retire already. diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index c140e593c870a..cc45f834a522d 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -28,7 +28,8 @@ use timely::progress::Antichain; use tokio::sync::oneshot; use uuid::Uuid; -use crate::command::Command; +use crate::command::{Command, CatalogSnapshot}; +use crate::catalog::Catalog; use crate::coord; use crate::coord::Coordinator; use crate::coord::peek::FastPathPlan; @@ -119,7 +120,22 @@ impl PeekClient { Ok(self.oracles.get_mut(&timeline).expect("ensured above")) } - pub(crate) async fn call_coordinator(&self, f: F) -> T + /// Fetch a snapshot of the catalog for use in frontend peek sequencing. + /// Records the time taken in the adapter metrics, labeled by `context`. + pub async fn catalog_snapshot(&self, context: &str) -> Arc { + let start = std::time::Instant::now(); + let CatalogSnapshot { catalog } = self + .call_coordinator(|tx| Command::CatalogSnapshot { tx }) + .await; + self.coordinator_client + .metrics() + .catalog_snapshot_seconds + .with_label_values(&[context]) + .observe(start.elapsed().as_secs_f64()); + catalog + } + + async fn call_coordinator(&self, f: F) -> T where F: FnOnce(oneshot::Sender) -> Command, { From 99761e4ccea2164dcbfc028064145c6a614388c5 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 11:10:26 +0200 Subject: [PATCH 18/55] cargo fmt --- src/adapter/src/client.rs | 13 ++++++++++--- src/adapter/src/frontend_peek.rs | 4 ++-- src/adapter/src/peek_client.rs | 2 +- 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index f36ce94dfc090..40397bb6c2738 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -27,12 +27,12 @@ use mz_compute_types::ComputeInstanceId; use mz_ore::channel::OneshotReceiverExt; use mz_ore::collections::CollectionExt; use mz_ore::id_gen::{IdAllocator, IdAllocatorInnerBitSet, MAX_ORG_ID, org_id_conn_bits}; -use mz_ore::{instrument, soft_assert_or_log}; use mz_ore::now::{EpochMillis, NowFn, to_datetime}; use mz_ore::result::ResultExt; use mz_ore::task::AbortOnDropHandle; use mz_ore::thread::JoinOnDropHandle; use mz_ore::tracing::OpenTelemetryContext; +use mz_ore::{instrument, soft_assert_or_log}; use mz_repr::{CatalogItemId, ColumnIndex, Row, SqlScalarType}; use mz_sql::ast::{Raw, Statement}; use mz_sql::catalog::{EnvironmentId, SessionCatalog}; @@ -1100,7 +1100,11 @@ impl SessionClient { ) -> Result, AdapterError> { if self.peek_client().enable_frontend_peek_sequencing { // Take ownership of the session and split-borrow the pieces we need. - let Self { session: slot, peek_client, .. } = self; + let Self { + session: slot, + peek_client, + .. + } = self; let session = slot.take().expect("SessionClient invariant"); // RAII guard that always puts the session back into `slot` (`SessionClient::session`). @@ -1129,7 +1133,10 @@ impl SessionClient { } } - let mut guard = SessionReinserter { slot, session: Some(session) }; + let mut guard = SessionReinserter { + slot, + session: Some(session), + }; let res = peek_client .try_frontend_peek_inner(portal_name, guard.session_mut()) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index b24511e0d393c..dd6c1c10ac6d8 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -35,8 +35,8 @@ use crate::optimize::Optimize; use crate::optimize::dataflows::{ComputeInstanceSnapshot, DataflowBuilder}; use crate::session::{Session, TransactionOps, TransactionStatus}; use crate::{ - AdapterError, AdapterNotice, CollectionIdBundle, ExecuteResponse, ReadHolds, - TimelineContext, TimestampContext, TimestampProvider, optimize, PeekClient, + AdapterError, AdapterNotice, CollectionIdBundle, ExecuteResponse, PeekClient, ReadHolds, + TimelineContext, TimestampContext, TimestampProvider, optimize, }; use crate::{coord, metrics}; diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index cc45f834a522d..b71eed61c93d7 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -28,8 +28,8 @@ use timely::progress::Antichain; use tokio::sync::oneshot; use uuid::Uuid; -use crate::command::{Command, CatalogSnapshot}; use crate::catalog::Catalog; +use crate::command::{CatalogSnapshot, Command}; use crate::coord; use crate::coord::Coordinator; use crate::coord::peek::FastPathPlan; From 162a3a2c5249597b0b5d691f51f8fd76ef8bd904 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 14:10:09 +0200 Subject: [PATCH 19/55] Make recursion_limit.slt easier --- test/sqllogictest/recursion_limit.slt | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/test/sqllogictest/recursion_limit.slt b/test/sqllogictest/recursion_limit.slt index 3caaef398668a..bd30e817e1344 100644 --- a/test/sqllogictest/recursion_limit.slt +++ b/test/sqllogictest/recursion_limit.slt @@ -10,13 +10,8 @@ # Tests to exercise recursion limits and stack overflow protection in # the query planner and optimizer. -# TODO(peek-seq): This is not working, because we look at this flag only at session start, which already happened! -simple conn=mz_system,user=mz_system -ALTER SYSTEM SET enable_frontend_peek_sequencing = false; ----- -COMPLETE 0 - +# 475 UNIONs query I -SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1; +SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1 UNION SELECT 1; ---- 1 From 8439295d731966b2606d07a11358578761334406 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 14:23:14 +0200 Subject: [PATCH 20/55] Simplify diff checking in implement_fast_path_peek_plan --- src/adapter/src/peek_client.rs | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index b71eed61c93d7..4cbc002558fd4 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -258,23 +258,23 @@ impl PeekClient { let mut results = Vec::new(); for (row, count) in rows { - if count.is_negative() { - return Err(AdapterError::Unstructured(anyhow::anyhow!( - "Negative multiplicity in constant result: {}", - count - ))); - } - if count.is_positive() { - let count = usize::cast_from( - u64::try_from(count.into_inner()) - .expect("known to be positive from check above"), - ); - results.push(( - row, - std::num::NonZeroUsize::new(count) - .expect("known to be non-zero from check above"), - )); - } + let count = match u64::try_from(count.into_inner()) { + Ok(u) => usize::cast_from(u), + Err(_) => { + return Err(AdapterError::Unstructured(anyhow::anyhow!( + "Negative multiplicity in constant result: {}", + count + ))); + } + }; + match std::num::NonZeroUsize::new(count) { + Some(nzu) => { + results.push((row, nzu)); + } + None => { + // No need to retain 0 diffs. + } + }; } let row_collection = RowCollection::new(results, &finishing.order_by); match finishing.finish( From 67218ba46cbc1c2de5f2df8888310787cfe082f3 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 14:50:45 +0200 Subject: [PATCH 21/55] Refactor acquire_read_holds_and_collection_write_frontiers --- src/compute-client/src/controller/instance.rs | 31 +++++++------------ 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index 8706ca1cbebee..b0ab7f88f4de8 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -244,8 +244,18 @@ where &self, ids: Vec, ) -> Result, Antichain)>, CollectionMissing> { - self.call_sync(move |i| i.acquire_read_holds_and_collection_write_frontiers(ids)) - .await + self.call_sync(move |i| { + let mut result = Vec::new(); + for id in ids.into_iter() { + result.push(( + id, + i.acquire_read_hold(id)?, + i.collection_write_frontier(id)?, + )); + } + Ok(result) + }) + .await } /// Issue a peek by calling into the instance task, letting the instance acquire read holds if @@ -2405,23 +2415,6 @@ where } } - /// Acquires a `ReadHold` and collection write frontier for each of the identified compute - /// collections. - fn acquire_read_holds_and_collection_write_frontiers( - &self, - ids: Vec, - ) -> Result, Antichain)>, CollectionMissing> { - let mut result = Vec::new(); - for id in ids.into_iter() { - result.push(( - id, - self.acquire_read_hold(id)?, - self.collection_write_frontier(id)?, - )); - } - Ok(result) - } - /// Acquires a `ReadHold` for the identified compute collection. /// /// This mirrors the logic used by the controller-side `InstanceState::acquire_read_hold`, From a4848d6ab6dcab01e0d174acfb0091b98c144a9b Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 15:56:50 +0200 Subject: [PATCH 22/55] minor --- src/adapter/src/peek_client.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 4cbc002558fd4..a241c1accd5a2 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -226,10 +226,9 @@ impl PeekClient { /// /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. /// - /// Note: `input_read_holds` is an Option, because we don't need it for constant queries. - /// `input_read_holds` has holds for all inputs. For fast-path peeks, this includes the peek. - /// target For slow-path peeks (to be implemented later), we'll need to additionally call into - /// the Controller to acquire a hold on the peek target after we create the dataflow. + /// Note: `input_read_holds` has holds for all inputs. For fast-path peeks, this includes the + /// peek target. For slow-path peeks (to be implemented later), we'll need to additionally call + /// into the Controller to acquire a hold on the peek target after we create the dataflow. /// /// TODO(peek-seq): add statement logging /// TODO(peek-seq): cancellation (see pending_peeks/client_pending_peeks wiring in the old From 1a63da082e77bf3f741b93cbae0cb58bc7e1d0ec Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 16:15:30 +0200 Subject: [PATCH 23/55] Move enable_frontend_peek_sequencing from PeekClient to SessionClient --- src/adapter/src/client.rs | 11 ++++++++--- src/adapter/src/peek_client.rs | 10 ++-------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 40397bb6c2738..4b2208ecc46b8 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -270,7 +270,6 @@ impl Client { storage_collections, transient_id_gen, optimizer_metrics, - // enable_frontend_peek_sequencing is initialized below, once we have a ConnCatalog ); let mut client = SessionClient { @@ -280,6 +279,7 @@ impl Client { environment_id: self.environment_id.clone(), segment_client: self.segment_client.clone(), peek_client, + enable_frontend_peek_sequencing: false, // initialized below, once we have a ConnCatalog }; let session = client.session(); @@ -411,7 +411,7 @@ Issue a SQL query to get started. Need help? } } - client.peek_client.enable_frontend_peek_sequencing = ENABLE_FRONTEND_PEEK_SEQUENCING + client.enable_frontend_peek_sequencing = ENABLE_FRONTEND_PEEK_SEQUENCING .require(catalog.system_vars()) .is_ok(); @@ -545,6 +545,11 @@ pub struct SessionClient { environment_id: EnvironmentId, /// Client for frontend peek sequencing; populated at connection startup. peek_client: PeekClient, + /// Whether frontend peek sequencing is enabled; initialized at connection startup. + // TODO(peek-seq): Currently, this is initialized only at session startup. We'll be able to + // check the actual feature flag value at every peek (without a Coordinator call) once we'll + // always have a catalog snapshot at hand. + pub enable_frontend_peek_sequencing: bool, } impl SessionClient { @@ -1098,7 +1103,7 @@ impl SessionClient { &mut self, portal_name: &str, ) -> Result, AdapterError> { - if self.peek_client().enable_frontend_peek_sequencing { + if self.enable_frontend_peek_sequencing { // Take ownership of the session and split-borrow the pieces we need. let Self { session: slot, diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index a241c1accd5a2..b6722407e5363 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -59,15 +59,10 @@ pub struct PeekClient { pub optimizer_metrics: OptimizerMetrics, /// Per-timeline oracles from the coordinator. Lazily populated. oracles: BTreeMap + Send + Sync>>, - // TODO(peek-seq): This is initialized only at session startup. We'll be able to properly check - // the actual feature flag value (without a Coordinator call) once we'll always have a catalog - // snapshot at hand. - pub enable_frontend_peek_sequencing: bool, } impl PeekClient { - /// Creates a PeekClient. Leaves `enable_frontend_peek_sequencing` false! This should be filled - /// in later. + /// Creates a PeekClient. pub fn new( coordinator_client: Client, storage_collections: StorageCollectionsHandle, @@ -80,8 +75,7 @@ impl PeekClient { storage_collections, transient_id_gen, optimizer_metrics, - oracles: Default::default(), // lazily populated - enable_frontend_peek_sequencing: false, // should be filled in later! + oracles: Default::default(), // lazily populated } } From 6e53e90815fddf2081798f6298885df9c37c7255 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Mon, 13 Oct 2025 16:28:25 +0200 Subject: [PATCH 24/55] Remove the SessionReinserter stuff --- src/adapter/src/client.rs | 53 ++++----------------------------------- 1 file changed, 5 insertions(+), 48 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 4b2208ecc46b8..4b39ce61a1794 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -27,12 +27,12 @@ use mz_compute_types::ComputeInstanceId; use mz_ore::channel::OneshotReceiverExt; use mz_ore::collections::CollectionExt; use mz_ore::id_gen::{IdAllocator, IdAllocatorInnerBitSet, MAX_ORG_ID, org_id_conn_bits}; +use mz_ore::instrument; use mz_ore::now::{EpochMillis, NowFn, to_datetime}; use mz_ore::result::ResultExt; use mz_ore::task::AbortOnDropHandle; use mz_ore::thread::JoinOnDropHandle; use mz_ore::tracing::OpenTelemetryContext; -use mz_ore::{instrument, soft_assert_or_log}; use mz_repr::{CatalogItemId, ColumnIndex, Row, SqlScalarType}; use mz_sql::ast::{Raw, Statement}; use mz_sql::catalog::{EnvironmentId, SessionCatalog}; @@ -1104,53 +1104,10 @@ impl SessionClient { portal_name: &str, ) -> Result, AdapterError> { if self.enable_frontend_peek_sequencing { - // Take ownership of the session and split-borrow the pieces we need. - let Self { - session: slot, - peek_client, - .. - } = self; - let session = slot.take().expect("SessionClient invariant"); - - // RAII guard that always puts the session back into `slot` (`SessionClient::session`). - struct SessionReinserter<'a> { - slot: &'a mut Option, - session: Option, - } - impl<'a> SessionReinserter<'a> { - fn session_mut(&mut self) -> &mut Session { - self.session.as_mut().expect("session present") - } - fn put_back(&mut self) { - if let Some(sess) = self.session.take() { - soft_assert_or_log!(self.slot.is_none(), "the `take` above removed it"); - *self.slot = Some(sess); - } - } - } - impl<'a> Drop for SessionReinserter<'a> { - fn drop(&mut self) { - if self.slot.is_none() { - if let Some(sess) = self.session.take() { - *self.slot = Some(sess); - } - } - } - } - - let mut guard = SessionReinserter { - slot, - session: Some(session), - }; - - let res = peek_client - .try_frontend_peek_inner(portal_name, guard.session_mut()) - .await; - - // Ensure reinsertion on the normal path, too. - guard.put_back(); - - res + let session = self.session.as_mut().expect("SessionClient invariant"); + self.peek_client + .try_frontend_peek_inner(portal_name, session) + .await } else { Ok(None) } From 5ea723dc527d2462de82cc3497f77cbd6f7e6ec6 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Tue, 14 Oct 2025 13:28:41 +0200 Subject: [PATCH 25/55] Add support for PeekResponse::Stashed --- src/adapter/src/client.rs | 2 + src/adapter/src/command.rs | 2 + src/adapter/src/coord/command_handler.rs | 1 + src/adapter/src/coord/peek.rs | 2 +- src/adapter/src/frontend_peek.rs | 9 +++++ src/adapter/src/peek_client.rs | 51 ++++++++---------------- 6 files changed, 31 insertions(+), 36 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 4b39ce61a1794..5e0777f3fa2a7 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -263,6 +263,7 @@ impl Client { storage_collections, transient_id_gen, optimizer_metrics, + persist_client, } = response; let peek_client = PeekClient::new( @@ -270,6 +271,7 @@ impl Client { storage_collections, transient_id_gen, optimizer_metrics, + persist_client, ); let mut client = SessionClient { diff --git a/src/adapter/src/command.rs b/src/adapter/src/command.rs index 2bb4e6fa672b7..612d136b1ce5a 100644 --- a/src/adapter/src/command.rs +++ b/src/adapter/src/command.rs @@ -21,6 +21,7 @@ use mz_compute_types::ComputeInstanceId; use mz_ore::collections::CollectionExt; use mz_ore::soft_assert_no_log; use mz_ore::tracing::OpenTelemetryContext; +use mz_persist_client::PersistClient; use mz_pgcopy::CopyFormatParams; use mz_repr::global_id::TransientIdGen; use mz_repr::role_id::RoleId; @@ -249,6 +250,7 @@ pub struct StartupResponse { >, pub transient_id_gen: Arc, pub optimizer_metrics: OptimizerMetrics, + pub persist_client: PersistClient, } /// The response to [`Client::authenticate`](crate::Client::authenticate). diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index a665ee17f53ca..11a56d337d9f9 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -551,6 +551,7 @@ impl Coordinator { storage_collections: Arc::clone(&self.controller.storage_collections), transient_id_gen: Arc::clone(&self.transient_id_gen), optimizer_metrics: self.optimizer_metrics.clone(), + persist_client: self.persist_client.clone(), }); if tx.send(resp).is_err() { // Failed to send to adapter, but everything is setup so we can terminate diff --git a/src/adapter/src/coord/peek.rs b/src/adapter/src/coord/peek.rs index fee595d17a7ce..c3d2736c5dfe2 100644 --- a/src/adapter/src/coord/peek.rs +++ b/src/adapter/src/coord/peek.rs @@ -885,7 +885,7 @@ impl crate::coord::Coordinator { /// Creates an async stream that processes peek responses and yields rows. #[mz_ore::instrument(level = "debug")] - fn create_peek_response_stream( + pub(crate) fn create_peek_response_stream( rows_rx: tokio::sync::oneshot::Receiver, finishing: RowSetFinishing, max_result_size: u64, diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index dd6c1c10ac6d8..64ba58bfc85a9 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -496,6 +496,13 @@ impl PeekClient { let row_set_finishing_seconds = session.metrics().row_set_finishing_seconds().clone(); + let peek_stash_read_batch_size_bytes = + mz_compute_types::dyncfgs::PEEK_RESPONSE_STASH_READ_BATCH_SIZE_BYTES + .get(catalog.system_config().dyncfgs()); + let peek_stash_read_memory_budget_bytes = + mz_compute_types::dyncfgs::PEEK_RESPONSE_STASH_READ_MEMORY_BUDGET_BYTES + .get(catalog.system_config().dyncfgs()); + // Implement the peek, and capture the response. let resp = self .implement_fast_path_peek_plan( @@ -509,6 +516,8 @@ impl PeekClient { max_query_result_size, row_set_finishing_seconds, read_holds, + peek_stash_read_batch_size_bytes, + peek_stash_read_memory_budget_bytes, ) .await?; diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index b6722407e5363..e6f9cf94e5024 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -13,10 +13,10 @@ use std::sync::Arc; use differential_dataflow::consolidation::consolidate; use mz_compute_client::controller::error::CollectionMissing; use mz_compute_client::protocol::command::PeekTarget; -use mz_compute_client::protocol::response::PeekResponse; use mz_compute_types::ComputeInstanceId; use mz_expr::row::RowCollection; use mz_ore::cast::CastFrom; +use mz_persist_client::PersistClient; use mz_repr::Timestamp; use mz_repr::global_id::TransientIdGen; use mz_repr::{RelationDesc, Row}; @@ -30,7 +30,6 @@ use uuid::Uuid; use crate::catalog::Catalog; use crate::command::{CatalogSnapshot, Command}; -use crate::coord; use crate::coord::Coordinator; use crate::coord::peek::FastPathPlan; use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds}; @@ -59,6 +58,7 @@ pub struct PeekClient { pub optimizer_metrics: OptimizerMetrics, /// Per-timeline oracles from the coordinator. Lazily populated. oracles: BTreeMap + Send + Sync>>, + persist_client: PersistClient, } impl PeekClient { @@ -68,6 +68,7 @@ impl PeekClient { storage_collections: StorageCollectionsHandle, transient_id_gen: Arc, optimizer_metrics: OptimizerMetrics, + persist_client: PersistClient, ) -> Self { Self { coordinator_client, @@ -76,6 +77,7 @@ impl PeekClient { transient_id_gen, optimizer_metrics, oracles: Default::default(), // lazily populated + persist_client, } } @@ -239,6 +241,8 @@ impl PeekClient { max_returned_query_size: Option, row_set_finishing_seconds: Histogram, input_read_holds: ReadHolds, + peek_stash_read_batch_size_bytes: usize, + peek_stash_read_memory_budget_bytes: usize, ) -> Result { match fast_path { // If the dataflow optimizes to a constant expression, we can immediately return the result. @@ -319,39 +323,16 @@ impl PeekClient { ) .await; - // TODO(peek-seq): call `create_peek_response_stream` instead. For that, we'll need - // to pass in a PersistClient from afar. - let peek_response_stream = async_stream::stream!({ - match rows_rx.await { - Ok(PeekResponse::Rows(rows)) => { - match finishing.finish( - rows, - max_result_size, - max_returned_query_size, - &row_set_finishing_seconds, - ) { - Ok((rows, _size_bytes)) => { - yield coord::peek::PeekResponseUnary::Rows(Box::new(rows)) - } - Err(e) => yield coord::peek::PeekResponseUnary::Error(e), - } - } - Ok(PeekResponse::Stashed(_response)) => { - // TODO(peek-seq): support this (through `create_peek_response_stream`) - yield coord::peek::PeekResponseUnary::Error("stashed peek responses not yet supported in frontend peek sequencing".into()); - } - Ok(PeekResponse::Error(err)) => { - yield coord::peek::PeekResponseUnary::Error(err); - } - Ok(PeekResponse::Canceled) => { - yield coord::peek::PeekResponseUnary::Canceled; - } - Err(e) => { - yield coord::peek::PeekResponseUnary::Error(e.to_string()); - } - } - }); - + let peek_response_stream = Coordinator::create_peek_response_stream( + rows_rx, + finishing, + max_result_size, + max_returned_query_size, + row_set_finishing_seconds, + self.persist_client.clone(), + peek_stash_read_batch_size_bytes, + peek_stash_read_memory_budget_bytes, + ); Ok(crate::ExecuteResponse::SendingRowsStreaming { rows: Box::pin(peek_response_stream), instance_id: compute_instance, From a5bd4c093050fe1da30c4ef46d83f3f4469825ea Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Tue, 14 Oct 2025 13:40:49 +0200 Subject: [PATCH 26/55] Add support for FastPathPlan::PeekPersist --- src/adapter/src/frontend_peek.rs | 9 +--- src/adapter/src/peek_client.rs | 79 ++++++++++++++++++++++++++------ 2 files changed, 67 insertions(+), 21 deletions(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 64ba58bfc85a9..f1204ea51ffbb 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -432,13 +432,8 @@ impl PeekClient { return Ok(None); } PeekPlan::FastPath(p @ FastPathPlan::Constant(_, _)) - | PeekPlan::FastPath(p @ FastPathPlan::PeekExisting(_, _, _, _)) => p, - PeekPlan::FastPath(FastPathPlan::PeekPersist(_, _, _)) => { - debug!( - "Bailing out from try_frontend_peek_inner, because it's a Persist fast path peek" - ); - return Ok(None); - } + | PeekPlan::FastPath(p @ FastPathPlan::PeekExisting(_, _, _, _)) + | PeekPlan::FastPath(p @ FastPathPlan::PeekPersist(_, _, _)) => p, }; // Warning: Do not bail out from the new peek sequencing after this point, because the diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index e6f9cf94e5024..8905d4e74aef1 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -17,9 +17,9 @@ use mz_compute_types::ComputeInstanceId; use mz_expr::row::RowCollection; use mz_ore::cast::CastFrom; use mz_persist_client::PersistClient; +use mz_repr::RelationDesc; use mz_repr::Timestamp; use mz_repr::global_id::TransientIdGen; -use mz_repr::{RelationDesc, Row}; use mz_sql::optimizer_metrics::OptimizerMetrics; use mz_storage_types::sources::Timeline; use mz_timestamp_oracle::TimestampOracle; @@ -300,8 +300,6 @@ impl PeekClient { .await .expect("missing compute instance client"); let peek_target = PeekTarget::Index { id: idx_id }; - let literal_vec: Option> = literal_constraints; - let map_filter_project = mfp; let finishing_for_instance = finishing.clone(); let target_read_hold = input_read_holds .compute_holds @@ -311,12 +309,12 @@ impl PeekClient { client .peek( peek_target, - literal_vec, + literal_constraints, uuid, timestamp, result_desc, finishing_for_instance, - map_filter_project, + mfp, target_read_hold, target_replica, rows_tx, @@ -339,15 +337,68 @@ impl PeekClient { strategy: crate::statement_logging::StatementExecutionStrategy::FastPath, }) } - FastPathPlan::PeekPersist(..) => { - // TODO(peek-seq): Implement this. (We currently bail out in - // `try_frontend_peek_inner`, similarly to slow-path peeks.) - // Note that `Instance::peek` has the following comment: - // "For persist peeks, the controller should provide a storage read hold. - // We don't support acquiring it here." - // Can we do this from here? - // (Note that if we want to bail out for this case, we need to bail out earlier!) - unimplemented!("PeekPersist not yet supported in frontend peek sequencing") + FastPathPlan::PeekPersist(coll_id, literal_constraint, mfp) => { + let (rows_tx, rows_rx) = oneshot::channel(); + let uuid = Uuid::new_v4(); + + let literal_constraints = literal_constraint.map(|r| vec![r]); + let metadata = self + .storage_collections + .collection_metadata(coll_id) + .expect("storage collection for fast-path peek") + .clone(); + let peek_target = PeekTarget::Persist { + id: coll_id, + metadata, + }; + + // At this stage we don't know column names for the result because we + // only know the peek's result type as a bare SqlRelationType. + let cols = (0..intermediate_result_type.arity()).map(|i| format!("peek_{i}")); + let result_desc = RelationDesc::new(intermediate_result_type.clone(), cols); + + let finishing_for_instance = finishing.clone(); + let target_read_hold = input_read_holds + .storage_holds + .get(&coll_id) + .expect("missing storage read hold on PeekPersist peek target") + .clone(); + + // Issue peek to the instance + let client = self + .ensure_compute_instance_client(compute_instance) + .await + .expect("missing compute instance client"); + client + .peek( + peek_target, + literal_constraints, + uuid, + timestamp, + result_desc, + finishing_for_instance, + mfp, + target_read_hold, + target_replica, + rows_tx, + ) + .await; + + let peek_response_stream = Coordinator::create_peek_response_stream( + rows_rx, + finishing, + max_result_size, + max_returned_query_size, + row_set_finishing_seconds, + self.persist_client.clone(), + peek_stash_read_batch_size_bytes, + peek_stash_read_memory_budget_bytes, + ); + Ok(crate::ExecuteResponse::SendingRowsStreaming { + rows: Box::pin(peek_response_stream), + instance_id: compute_instance, + strategy: crate::statement_logging::StatementExecutionStrategy::PersistFastPath, + }) } } } From 7e0c829f372fafd505790c0b65dc17062dceb6ca Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Tue, 14 Oct 2025 13:48:04 +0200 Subject: [PATCH 27/55] Reduce duplication in implement_fast_path_peek_plan --- src/adapter/src/peek_client.rs | 238 +++++++++++++++------------------ 1 file changed, 107 insertions(+), 131 deletions(-) diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 8905d4e74aef1..d52fed34f7327 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -32,7 +32,7 @@ use crate::catalog::Catalog; use crate::command::{CatalogSnapshot, Command}; use crate::coord::Coordinator; use crate::coord::peek::FastPathPlan; -use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds}; +use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds, statement_logging}; /// Storage collections trait alias we need to consult for since/frontiers. pub type StorageCollectionsHandle = Arc< @@ -244,103 +244,65 @@ impl PeekClient { peek_stash_read_batch_size_bytes: usize, peek_stash_read_memory_budget_bytes: usize, ) -> Result { - match fast_path { - // If the dataflow optimizes to a constant expression, we can immediately return the result. - FastPathPlan::Constant(rows_res, _) => { - let mut rows = match rows_res { - Ok(rows) => rows, - Err(e) => return Err(e.into()), - }; - consolidate(&mut rows); + // If the dataflow optimizes to a constant expression, we can immediately return the result. + if let FastPathPlan::Constant(rows_res, _) = fast_path { + let mut rows = match rows_res { + Ok(rows) => rows, + Err(e) => return Err(e.into()), + }; + consolidate(&mut rows); - let mut results = Vec::new(); - for (row, count) in rows { - let count = match u64::try_from(count.into_inner()) { - Ok(u) => usize::cast_from(u), - Err(_) => { - return Err(AdapterError::Unstructured(anyhow::anyhow!( - "Negative multiplicity in constant result: {}", - count - ))); - } - }; - match std::num::NonZeroUsize::new(count) { - Some(nzu) => { - results.push((row, nzu)); - } - None => { - // No need to retain 0 diffs. - } - }; - } - let row_collection = RowCollection::new(results, &finishing.order_by); - match finishing.finish( - row_collection, - max_result_size, - max_returned_query_size, - &row_set_finishing_seconds, - ) { - Ok((rows, _bytes)) => Ok(Coordinator::send_immediate_rows(rows)), - // TODO(peek-seq): make this a structured error. (also in the old sequencing) - Err(e) => Err(AdapterError::ResultSize(e)), - } + let mut results = Vec::new(); + for (row, count) in rows { + let count = match u64::try_from(count.into_inner()) { + Ok(u) => usize::cast_from(u), + Err(_) => { + return Err(AdapterError::Unstructured(anyhow::anyhow!( + "Negative multiplicity in constant result: {}", + count + ))); + } + }; + match std::num::NonZeroUsize::new(count) { + Some(nzu) => { + results.push((row, nzu)); + } + None => { + // No need to retain 0 diffs. + } + }; } - FastPathPlan::PeekExisting(_coll_id, idx_id, literal_constraints, mfp) => { - let (rows_tx, rows_rx) = oneshot::channel(); - let uuid = Uuid::new_v4(); - - // At this stage we don't know column names for the result because we - // only know the peek's result type as a bare SqlRelationType. - let cols = (0..intermediate_result_type.arity()).map(|i| format!("peek_{i}")); - let result_desc = RelationDesc::new(intermediate_result_type.clone(), cols); + let row_collection = RowCollection::new(results, &finishing.order_by); + return match finishing.finish( + row_collection, + max_result_size, + max_returned_query_size, + &row_set_finishing_seconds, + ) { + Ok((rows, _bytes)) => Ok(Coordinator::send_immediate_rows(rows)), + // TODO(peek-seq): make this a structured error. (also in the old sequencing) + Err(e) => Err(AdapterError::ResultSize(e)), + }; + } - // Issue peek to the instance - let client = self - .ensure_compute_instance_client(compute_instance) - .await - .expect("missing compute instance client"); + let (peek_target, target_read_hold, literal_constraints, mfp, strategy) = match fast_path { + FastPathPlan::PeekExisting(_coll_id, idx_id, literal_constraints, mfp) => { let peek_target = PeekTarget::Index { id: idx_id }; - let finishing_for_instance = finishing.clone(); let target_read_hold = input_read_holds .compute_holds .get(&(compute_instance, idx_id)) - .expect("missing compute read hold on peek target") + .expect("missing compute read hold on PeekExisting peek target") .clone(); - client - .peek( - peek_target, - literal_constraints, - uuid, - timestamp, - result_desc, - finishing_for_instance, - mfp, - target_read_hold, - target_replica, - rows_tx, - ) - .await; - - let peek_response_stream = Coordinator::create_peek_response_stream( - rows_rx, - finishing, - max_result_size, - max_returned_query_size, - row_set_finishing_seconds, - self.persist_client.clone(), - peek_stash_read_batch_size_bytes, - peek_stash_read_memory_budget_bytes, - ); - Ok(crate::ExecuteResponse::SendingRowsStreaming { - rows: Box::pin(peek_response_stream), - instance_id: compute_instance, - strategy: crate::statement_logging::StatementExecutionStrategy::FastPath, - }) + let strategy = statement_logging::StatementExecutionStrategy::FastPath; + ( + peek_target, + target_read_hold, + literal_constraints, + mfp, + strategy, + ) } FastPathPlan::PeekPersist(coll_id, literal_constraint, mfp) => { - let (rows_tx, rows_rx) = oneshot::channel(); - let uuid = Uuid::new_v4(); - let literal_constraints = literal_constraint.map(|r| vec![r]); let metadata = self .storage_collections @@ -351,55 +313,69 @@ impl PeekClient { id: coll_id, metadata, }; - - // At this stage we don't know column names for the result because we - // only know the peek's result type as a bare SqlRelationType. - let cols = (0..intermediate_result_type.arity()).map(|i| format!("peek_{i}")); - let result_desc = RelationDesc::new(intermediate_result_type.clone(), cols); - - let finishing_for_instance = finishing.clone(); let target_read_hold = input_read_holds .storage_holds .get(&coll_id) .expect("missing storage read hold on PeekPersist peek target") .clone(); + let strategy = statement_logging::StatementExecutionStrategy::PersistFastPath; + ( + peek_target, + target_read_hold, + literal_constraints, + mfp, + strategy, + ) + } + _ => { + // FastPathPlan::Constant handled above. + unreachable!() + } + }; - // Issue peek to the instance - let client = self - .ensure_compute_instance_client(compute_instance) - .await - .expect("missing compute instance client"); - client - .peek( - peek_target, - literal_constraints, - uuid, - timestamp, - result_desc, - finishing_for_instance, - mfp, - target_read_hold, - target_replica, - rows_tx, - ) - .await; + let (rows_tx, rows_rx) = oneshot::channel(); + let uuid = Uuid::new_v4(); - let peek_response_stream = Coordinator::create_peek_response_stream( - rows_rx, - finishing, - max_result_size, - max_returned_query_size, - row_set_finishing_seconds, - self.persist_client.clone(), - peek_stash_read_batch_size_bytes, - peek_stash_read_memory_budget_bytes, - ); - Ok(crate::ExecuteResponse::SendingRowsStreaming { - rows: Box::pin(peek_response_stream), - instance_id: compute_instance, - strategy: crate::statement_logging::StatementExecutionStrategy::PersistFastPath, - }) - } - } + // At this stage we don't know column names for the result because we + // only know the peek's result type as a bare SqlRelationType. + let cols = (0..intermediate_result_type.arity()).map(|i| format!("peek_{i}")); + let result_desc = RelationDesc::new(intermediate_result_type.clone(), cols); + + // Issue the peek to the instance + let client = self + .ensure_compute_instance_client(compute_instance) + .await + .expect("missing compute instance client"); + let finishing_for_instance = finishing.clone(); + client + .peek( + peek_target, + literal_constraints, + uuid, + timestamp, + result_desc, + finishing_for_instance, + mfp, + target_read_hold, + target_replica, + rows_tx, + ) + .await; + + let peek_response_stream = Coordinator::create_peek_response_stream( + rows_rx, + finishing, + max_result_size, + max_returned_query_size, + row_set_finishing_seconds, + self.persist_client.clone(), + peek_stash_read_batch_size_bytes, + peek_stash_read_memory_budget_bytes, + ); + Ok(crate::ExecuteResponse::SendingRowsStreaming { + rows: Box::pin(peek_response_stream), + instance_id: compute_instance, + strategy, + }) } } From 4c53c7d3856e6c71e71a22cd679113d5d4d77251 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Wed, 15 Oct 2025 15:59:37 +0200 Subject: [PATCH 28/55] Opt out of the frontend peek sequencing in test_github_25388 --- src/environmentd/tests/server.rs | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/environmentd/tests/server.rs b/src/environmentd/tests/server.rs index cf14fbfd32046..d338afe34c5a7 100644 --- a/src/environmentd/tests/server.rs +++ b/src/environmentd/tests/server.rs @@ -3910,6 +3910,18 @@ async fn test_github_25388() { server .enable_feature_flags(&["unsafe_enable_unsafe_functions"]) .await; + + // TODO(peek-seq) The second part of this test no longer works with the new peek sequencing, + // because we no longer check the catalog after optimization whether the original dependencies + // still exist. This might be fine, because nothing bad happens: timestamp determination already + // puts a a read hold on the index, so the index doesn't actually gets dropped in the + // Controller, and therefore the peek actually succeeds. In other words, the old peek + // sequencing's dependency check was overly cautious. I'm planning to revisit this later, and + // probably delete the second part of the test. + server + .disable_feature_flags(&["enable_frontend_peek_sequencing"]) + .await; + let client1 = server.connect().await.unwrap(); client1 From d81d4c2cd38bfd1c02d6fc017d115073447c2d6d Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Thu, 16 Oct 2025 14:19:55 +0200 Subject: [PATCH 29/55] Address some minor comments --- src/adapter/src/frontend_peek.rs | 43 ++++++++----------- src/adapter/src/peek_client.rs | 12 +----- src/compute-client/src/controller/instance.rs | 7 +-- 3 files changed, 21 insertions(+), 41 deletions(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index f1204ea51ffbb..52bffa07b796e 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -28,7 +28,7 @@ use tracing::{Span, debug}; use tracing_opentelemetry::OpenTelemetrySpanExt; use crate::catalog::CatalogState; -use crate::coord::peek::{FastPathPlan, PeekPlan}; +use crate::coord::peek::PeekPlan; use crate::coord::timestamp_selection::TimestampDetermination; use crate::coord::{Coordinator, ExplainContext, TargetCluster}; use crate::optimize::Optimize; @@ -75,13 +75,12 @@ impl PeekClient { } } - // This is from handle_execute_inner, but we do it already here because of lifetime issues, - // and also to be able to give a catalog to `verify_portal`. - // // TODO(peek-seq): This snapshot is wasted when we end up bailing out from the frontend peek - // sequencing. I think the best way to solve this is with that optimization where we + // sequencing. We could solve this is with that optimization where we // continuously keep a catalog snapshot in the session, and only get a new one when the // catalog revision has changed, which we could see with an atomic read. + // But anyhow, this problem will just go away when we reach the point that we never fall + // back to the old sequencing. let catalog = self.catalog_snapshot("try_frontend_peek_inner").await; if let Err(_) = Coordinator::verify_portal(&*catalog, session, portal_name) { @@ -130,8 +129,7 @@ impl PeekClient { let select_plan = match &plan { Plan::Select(select_plan) => select_plan, _ => { - debug!("Bailing out from try_frontend_peek_inner, because it's not a SELECT"); - return Ok(None); + unreachable!("checked above at the Statement level that it's a SELECT") } }; let explain_ctx = ExplainContext::None; // EXPLAIN is not handled here for now, only SELECT @@ -156,15 +154,13 @@ impl PeekClient { // TODO(peek-seq): statement logging: set_statement_execution_cluster - if let Err(e) = coord::catalog_serving::check_cluster_restrictions( + coord::catalog_serving::check_cluster_restrictions( target_cluster_name.as_str(), &conn_catalog, &plan, - ) { - return Err(e); - } + )?; - if let Err(e) = rbac::check_plan( + rbac::check_plan( &conn_catalog, |_id| { // This is only used by `Plan::SideEffectingFunc`, so it is irrelevant for us here @@ -175,9 +171,7 @@ impl PeekClient { &plan, Some(target_cluster_id), &resolved_ids, - ) { - return Err(e.into()); - } + )?; // Check if we're still waiting for any of the builtin table appends from when we // started the Session to complete. @@ -210,13 +204,9 @@ impl PeekClient { // # From peek_validate - //let compute_instance_snapshot = self.peek_client().snapshot(cluster.id()).await.unwrap(); let compute_instance_snapshot = ComputeInstanceSnapshot::new_without_collections(cluster.id()); - let (_, view_id) = self.transient_id_gen.allocate_id(); - let (_, index_id) = self.transient_id_gen.allocate_id(); - let optimizer_config = optimize::OptimizerConfig::from(catalog.system_config()) .override_from(&catalog.get_cluster(cluster.id()).config.features()); @@ -227,6 +217,9 @@ impl PeekClient { }); } + let (_, view_id) = self.transient_id_gen.allocate_id(); + let (_, index_id) = self.transient_id_gen.allocate_id(); + let mut optimizer = optimize::peek::Optimizer::new( Arc::clone(&catalog), compute_instance_snapshot.clone(), @@ -370,7 +363,10 @@ impl PeekClient { ); return Ok(None); } + + // TODO(peek-seq): wire up statistics let stats = Box::new(EmptyStatisticsOracle); + let session_meta = session.meta(); let now = catalog.config().now.clone(); let select_plan = select_plan.clone(); @@ -425,15 +421,12 @@ impl PeekClient { // TODO(peek-seq): plan_insights stuff - // This match is based on what `implement_fast_path_peek_plan` supports. let fast_path_plan = match peek_plan { PeekPlan::SlowPath(_) => { debug!("Bailing out from try_frontend_peek_inner, because it's a slow-path peek"); return Ok(None); } - PeekPlan::FastPath(p @ FastPathPlan::Constant(_, _)) - | PeekPlan::FastPath(p @ FastPathPlan::PeekExisting(_, _, _, _)) - | PeekPlan::FastPath(p @ FastPathPlan::PeekPersist(_, _, _)) => p, + PeekPlan::FastPath(p) => p, }; // Warning: Do not bail out from the new peek sequencing after this point, because the @@ -545,7 +538,7 @@ impl PeekClient { let isolation_level = session.vars().transaction_isolation(); let (read_holds, upper) = self - .acquire_read_holds_and_collection_write_frontiers(id_bundle) + .acquire_read_holds_and_least_valid_write(id_bundle) .await .expect("missing collection"); let (det, read_holds) = ::determine_timestamp_for_inner( @@ -598,7 +591,7 @@ impl PeekClient { session .metrics() .timestamp_difference_for_strict_serializable_ms(&[ - &compute_instance.to_string().as_ref(), + compute_instance.to_string().as_ref(), constraint_based.as_str(), ]) .observe(f64::cast_lossy(u64::from( diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index d52fed34f7327..b9857cb22a2ab 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -152,7 +152,7 @@ impl PeekClient { /// Therefore, using `StorageCollections::collections_frontiers` is sufficient. /// /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. - pub async fn acquire_read_holds_and_collection_write_frontiers( + pub async fn acquire_read_holds_and_least_valid_write( &mut self, id_bundle: &CollectionIdBundle, ) -> Result<(ReadHolds, Antichain), CollectionMissing> { @@ -210,16 +210,6 @@ impl PeekClient { /// Implement a fast-path peek plan. /// This is similar to `Coordinator::implement_peek_plan`, but only for fast path peeks. /// - /// Supported variants: - /// - FastPathPlan::Constant - /// - FastPathPlan::PeekExisting (PeekTarget::Index only) - /// - /// This fn assumes that the caller has already acquired read holds for the peek at the - /// appropriate timestamp. Before this fn returns, it passes on the responsibility of holding - /// back the frontiers to Compute, so then the caller can forget its own read holds. - /// - /// Note: FastPathPlan::PeekPersist is not yet supported here; we may add this later. - /// /// Note: self is taken &mut because of the lazy fetching in `get_compute_instance_client`. /// /// Note: `input_read_holds` has holds for all inputs. For fast-path peeks, this includes the diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index b0ab7f88f4de8..d34fac0f64fef 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -258,9 +258,7 @@ where .await } - /// Issue a peek by calling into the instance task, letting the instance acquire read holds if - /// none are provided. This ensures the read holds are established before returning to the - /// caller. + /// Issue a peek by calling into the instance task. pub async fn peek( &self, peek_target: PeekTarget, @@ -1738,10 +1736,9 @@ where ) -> Result<(), PeekError> { use PeekError::*; - // Acquire a read hold if one was not provided. let target_id = peek_target.id(); - // Downgrade the provided (or acquired) read hold to the peek time. + // Downgrade the provided read hold to the peek time. if read_hold.id() != target_id { return Err(ReadHoldIdMismatch(read_hold.id())); } From dbe764ce96f977674e60f1a33ea99a19125c418f Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Fri, 17 Oct 2025 15:59:56 +0200 Subject: [PATCH 30/55] Handle Plan::SideEffectingFunc gracefeully in the new peek sequencing --- src/adapter/src/frontend_peek.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 52bffa07b796e..374072fc9dc5a 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -129,7 +129,8 @@ impl PeekClient { let select_plan = match &plan { Plan::Select(select_plan) => select_plan, _ => { - unreachable!("checked above at the Statement level that it's a SELECT") + debug!("Bailing out from try_frontend_peek_inner, because it's not a SELECT"); + return Ok(None); } }; let explain_ctx = ExplainContext::None; // EXPLAIN is not handled here for now, only SELECT From 38d9a21504cb9d4b1cdc50d4ff149c427821c6bf Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Fri, 17 Oct 2025 18:19:29 +0200 Subject: [PATCH 31/55] Error handling 1 --- src/adapter/src/error.rs | 40 ++++++++++++++++++- src/adapter/src/peek_client.rs | 6 +-- src/adapter/src/util.rs | 4 +- src/compute-client/src/controller.rs | 7 +++- src/compute-client/src/controller/error.rs | 17 ++++++++ src/compute-client/src/controller/instance.rs | 22 +++++----- 6 files changed, 79 insertions(+), 17 deletions(-) diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index 30f5eef280ffd..b54f3c2957ca4 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -33,6 +33,7 @@ use smallvec::SmallVec; use timely::progress::Antichain; use tokio::sync::oneshot; use tokio_postgres::error::SqlState; +use mz_compute_client::controller::error::PeekError; use crate::coord::NetworkPolicyError; use crate::optimize::OptimizerError; @@ -51,7 +52,10 @@ pub enum AdapterError { AmbiguousSystemColumnReference, /// An error occurred in a catalog operation. Catalog(mz_catalog::memory::error::Error), - /// The cached plan or descriptor changed. + /// 1. The cached plan or descriptor changed, + /// 2. or some dependency of a statement disappeared during sequencing. + /// TODO(ggevay): we should refactor 2. usages to use `ConcurrentDependencyDrop` instead + /// (e.g., in MV sequencing) ChangedPlan(String), /// The cursor already exists. DuplicateCursor(String), @@ -97,6 +101,11 @@ pub enum AdapterError { ConstraintViolation(NotNullViolation), /// Transaction cluster was dropped in the middle of a transaction. ConcurrentClusterDrop, + /// A dependency was dropped while sequencing a statement. + ConcurrentDependencyDrop { + dependency_kind: &'static str, + dependency_id: String + }, /// Target cluster has no replicas to service query. NoClusterReplicasAvailable { name: String, @@ -492,6 +501,7 @@ impl AdapterError { AdapterError::InvalidTableMutationSelection => SqlState::INVALID_TRANSACTION_STATE, AdapterError::ConstraintViolation(NotNullViolation(_)) => SqlState::NOT_NULL_VIOLATION, AdapterError::ConcurrentClusterDrop => SqlState::INVALID_TRANSACTION_STATE, + AdapterError::ConcurrentDependencyDrop { .. } => SqlState::UNDEFINED_OBJECT, AdapterError::NoClusterReplicasAvailable { .. } => SqlState::FEATURE_NOT_SUPPORTED, AdapterError::OperationProhibitsTransaction(_) => SqlState::ACTIVE_SQL_TRANSACTION, AdapterError::OperationRequiresTransaction(_) => SqlState::NO_ACTIVE_SQL_TRANSACTION, @@ -665,6 +675,12 @@ impl fmt::Display for AdapterError { AdapterError::ConcurrentClusterDrop => { write!(f, "the transaction's active cluster has been dropped") } + AdapterError::ConcurrentDependencyDrop { + dependency_kind, + dependency_id, + } => { + write!(f, "{dependency_kind} '{dependency_id}' was dropped while executing a statement") + } AdapterError::NoClusterReplicasAvailable { name, .. } => { write!( f, @@ -1025,4 +1041,26 @@ impl From for AdapterError { } } +impl From for AdapterError { + fn from(e: PeekError) -> AdapterError { + match e { + PeekError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop{ + dependency_kind: "cluster", + dependency_id: id.to_string(), + }, + PeekError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: id.to_string(), + }, + PeekError::ReplicaMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "replica", + dependency_id: id.to_string(), + }, + e @ PeekError::SinceViolation(_) => AdapterError::internal("peek error", e), + e @ PeekError::ReadHoldIdMismatch(_) => AdapterError::internal("peek error", e), + e @ PeekError::ReadHoldInsufficient(_) => AdapterError::internal("peek error", e), + } + } +} + impl Error for AdapterError {} diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index b9857cb22a2ab..a1613299049b0 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -11,7 +11,7 @@ use std::collections::BTreeMap; use std::sync::Arc; use differential_dataflow::consolidation::consolidate; -use mz_compute_client::controller::error::CollectionMissing; +use mz_compute_client::controller::error::CollectionLookupError; use mz_compute_client::protocol::command::PeekTarget; use mz_compute_types::ComputeInstanceId; use mz_expr::row::RowCollection; @@ -155,7 +155,7 @@ impl PeekClient { pub async fn acquire_read_holds_and_least_valid_write( &mut self, id_bundle: &CollectionIdBundle, - ) -> Result<(ReadHolds, Antichain), CollectionMissing> { + ) -> Result<(ReadHolds, Antichain), CollectionLookupError> { let mut read_holds = ReadHolds::new(); let mut upper = Antichain::new(); @@ -350,7 +350,7 @@ impl PeekClient { target_replica, rows_tx, ) - .await; + .await?; let peek_response_stream = Coordinator::create_peek_response_stream( rows_rx, diff --git a/src/adapter/src/util.rs b/src/adapter/src/util.rs index 1bc458c89a7cc..a7b6d191b8dd7 100644 --- a/src/adapter/src/util.rs +++ b/src/adapter/src/util.rs @@ -395,7 +395,9 @@ impl ShouldTerminateGracefully for PeekError { PeekError::SinceViolation(_) | PeekError::InstanceMissing(_) | PeekError::CollectionMissing(_) - | PeekError::ReplicaMissing(_) => false, + | PeekError::ReplicaMissing(_) + | PeekError::ReadHoldIdMismatch(_) + | PeekError::ReadHoldInsufficient(_) => false, } } } diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 2447daebcd249..3c497f2a8bd46 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -1077,12 +1077,17 @@ impl InstanceState { self.client.call(f) } + /// Calls the given function on the instance task, and awaits the result. + /// + /// # Panics + /// + /// Panics if the instance corresponding to `self` does not exist. async fn call_sync(&self, f: F) -> R where F: FnOnce(&mut Instance) -> R + Send + 'static, R: Send + 'static, { - self.client.call_sync(f).await + self.client.call_sync(f).await.expect("controller should validate") } /// Acquires a [`ReadHold`] for the identified compute collection. diff --git a/src/compute-client/src/controller/error.rs b/src/compute-client/src/controller/error.rs index 38861ca92d09c..0f7d132c08c60 100644 --- a/src/compute-client/src/controller/error.rs +++ b/src/compute-client/src/controller/error.rs @@ -169,6 +169,12 @@ pub enum PeekError { /// TODO(database-issues#7533): Add documentation. #[error("peek timestamp is not beyond the since of collection: {0}")] SinceViolation(GlobalId), + /// TODO(database-issues#7533): Add documentation. + #[error("read hold ID does not match peeked collection: {0}")] + ReadHoldIdMismatch(GlobalId), + /// TODO(database-issues#7533): Add documentation. + #[error("insufficient read hold provided: {0}")] + ReadHoldInsufficient(GlobalId), } impl From for PeekError { @@ -192,6 +198,17 @@ impl From for PeekError { } } +impl From for PeekError { + fn from(error: crate::controller::instance::PeekError) -> Self { + use crate::controller::instance::PeekError::*; + match error { + ReplicaMissing(id) => PeekError::ReplicaMissing(id), + ReadHoldIdMismatch(id) => PeekError::ReadHoldIdMismatch(id), + ReadHoldInsufficient(id) => PeekError::ReadHoldInsufficient(id), + } + } +} + /// Errors arising during collection updates. #[derive(Error, Debug)] pub enum CollectionUpdateError { diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index d34fac0f64fef..f7340f470fc47 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -51,9 +51,7 @@ use tokio::sync::{mpsc, oneshot}; use tracing::debug_span; use uuid::Uuid; -use crate::controller::error::{ - CollectionLookupError, CollectionMissing, ERROR_TARGET_REPLICA_FAILED, HydrationCheckBadTarget, -}; +use crate::controller::error::{CollectionLookupError, CollectionMissing, ERROR_TARGET_REPLICA_FAILED, HydrationCheckBadTarget, InstanceMissing}; use crate::controller::replica::{ReplicaClient, ReplicaConfig}; use crate::controller::{ ComputeControllerResponse, ComputeControllerTimestamp, IntrospectionUpdates, PeekNotification, @@ -139,6 +137,8 @@ pub struct Client { /// A sender for read hold changes for collections installed on the instance. #[derivative(Debug = "ignore")] read_hold_tx: read_holds::ChangeTx, + /// The ID of the instance. + id: ComputeInstanceId, } impl Client { @@ -163,7 +163,7 @@ impl Client { /// Call a method to be run on the instance task, by sending a message to the instance and /// waiting for a response message. - pub(super) async fn call_sync(&self, f: F) -> R + pub(super) async fn call_sync(&self, f: F) -> Result where F: FnOnce(&mut Instance) -> R + Send + 'static, R: Send + 'static, @@ -177,9 +177,9 @@ impl Client { let result = f(instance); let _ = tx.send(result); })) - .expect("instance not dropped"); + .map_err(|_send_error| InstanceMissing(self.id))?; - rx.await.expect("instance not dropped") + Ok(rx.await.map_err(|_| InstanceMissing(self.id))?) } } @@ -235,6 +235,7 @@ where Self { command_tx, read_hold_tx, + id, } } @@ -243,7 +244,7 @@ where pub async fn acquire_read_holds_and_collection_write_frontiers( &self, ids: Vec, - ) -> Result, Antichain)>, CollectionMissing> { + ) -> Result, Antichain)>, CollectionLookupError> { self.call_sync(move |i| { let mut result = Vec::new(); for id in ids.into_iter() { @@ -255,7 +256,7 @@ where } Ok(result) }) - .await + .await? } /// Issue a peek by calling into the instance task. @@ -271,7 +272,7 @@ where target_read_hold: ReadHold, target_replica: Option, peek_response_tx: oneshot::Sender, - ) { + ) -> Result<(), crate::controller::error::PeekError> { self.call_sync(move |i| { i.peek( peek_target, @@ -285,9 +286,8 @@ where target_replica, peek_response_tx, ) - .expect("validated by instance"); }) - .await + .await.map(|r| r.map_err(|e| e.into()))? } } From 187dbaad67088656fe7fdee24d1b4989b55e20ba Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 15:55:29 +0200 Subject: [PATCH 32/55] Error handling 2 --- src/adapter/src/command.rs | 6 +++--- src/adapter/src/coord/command_handler.rs | 3 +-- src/adapter/src/error.rs | 11 ++++++++++- src/adapter/src/peek_client.rs | 10 ++++------ 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/src/adapter/src/command.rs b/src/adapter/src/command.rs index 612d136b1ce5a..1bd86a123dcba 100644 --- a/src/adapter/src/command.rs +++ b/src/adapter/src/command.rs @@ -25,7 +25,7 @@ use mz_persist_client::PersistClient; use mz_pgcopy::CopyFormatParams; use mz_repr::global_id::TransientIdGen; use mz_repr::role_id::RoleId; -use mz_repr::{CatalogItemId, ColumnIndex, RowIterator, Timestamp}; +use mz_repr::{CatalogItemId, ColumnIndex, RowIterator}; use mz_sql::ast::{FetchDirection, Raw, Statement}; use mz_sql::catalog::ObjectType; use mz_sql::optimizer_metrics::OptimizerMetrics; @@ -164,7 +164,7 @@ pub enum Command { tx: oneshot::Sender< Result< mz_compute_client::controller::instance::Client, - AdapterError, + mz_compute_client::controller::error::InstanceMissing, >, >, }, @@ -172,7 +172,7 @@ pub enum Command { GetOracle { timeline: Timeline, tx: oneshot::Sender< - Result + Send + Sync>, AdapterError>, + Result + Send + Sync>, AdapterError>, >, }, } diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index 11a56d337d9f9..b16578d9d9e35 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -275,8 +275,7 @@ impl Coordinator { let _ = tx.send( self.controller .compute - .instance_client(instance_id) - .map_err(|_| AdapterError::ConcurrentClusterDrop), + .instance_client(instance_id), ); } diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index b54f3c2957ca4..a91164b7b077b 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -33,7 +33,7 @@ use smallvec::SmallVec; use timely::progress::Antichain; use tokio::sync::oneshot; use tokio_postgres::error::SqlState; -use mz_compute_client::controller::error::PeekError; +use mz_compute_client::controller::error::{InstanceMissing, PeekError}; use crate::coord::NetworkPolicyError; use crate::optimize::OptimizerError; @@ -1063,4 +1063,13 @@ impl From for AdapterError { } } +impl From for AdapterError { + fn from(e: InstanceMissing) -> Self { + AdapterError::ConcurrentDependencyDrop { + dependency_kind: "cluster", + dependency_id: e.0.to_string(), + } + } +} + impl Error for AdapterError {} diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index a1613299049b0..8a8a976def268 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -11,7 +11,7 @@ use std::collections::BTreeMap; use std::sync::Arc; use differential_dataflow::consolidation::consolidate; -use mz_compute_client::controller::error::CollectionLookupError; +use mz_compute_client::controller::error::{CollectionLookupError, InstanceMissing}; use mz_compute_client::protocol::command::PeekTarget; use mz_compute_types::ComputeInstanceId; use mz_expr::row::RowCollection; @@ -84,7 +84,7 @@ impl PeekClient { pub async fn ensure_compute_instance_client( &mut self, compute_instance: ComputeInstanceId, - ) -> Result<&mut mz_compute_client::controller::instance::Client, AdapterError> { + ) -> Result<&mut mz_compute_client::controller::instance::Client, InstanceMissing> { if !self.compute_instances.contains_key(&compute_instance) { let client = self .call_coordinator(|tx| Command::GetComputeInstanceClient { @@ -183,8 +183,7 @@ impl PeekClient { for (&instance_id, collection_ids) in &id_bundle.compute_ids { let client = self .ensure_compute_instance_client(instance_id) - .await - .expect("missing compute instance client"); + .await?; for (id, read_hold, write_frontier) in client .acquire_read_holds_and_collection_write_frontiers( @@ -334,8 +333,7 @@ impl PeekClient { // Issue the peek to the instance let client = self .ensure_compute_instance_client(compute_instance) - .await - .expect("missing compute instance client"); + .await?; let finishing_for_instance = finishing.clone(); client .peek( From 136ce841b19cc53977891fc9392744fc1ef008c9 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 16:13:29 +0200 Subject: [PATCH 33/55] Error handling 3 I think this superfluous error variant existed because earlier acquire_read_holds's parameters used to specify the time, but then the interface was changed to just acquire the earliest possible read hold, and we forgot to drop this error variant. --- src/compute-client/src/controller/error.rs | 2 -- src/storage-types/src/read_holds.rs | 2 -- 2 files changed, 4 deletions(-) diff --git a/src/compute-client/src/controller/error.rs b/src/compute-client/src/controller/error.rs index 0f7d132c08c60..e6eb4d025a1fd 100644 --- a/src/compute-client/src/controller/error.rs +++ b/src/compute-client/src/controller/error.rs @@ -149,7 +149,6 @@ impl From for DataflowCreationError { fn from(error: ReadHoldError) -> Self { match error { ReadHoldError::CollectionMissing(id) => Self::CollectionMissing(id), - ReadHoldError::SinceViolation(id) => Self::SinceViolation(id), } } } @@ -193,7 +192,6 @@ impl From for PeekError { fn from(error: ReadHoldError) -> Self { match error { ReadHoldError::CollectionMissing(id) => Self::CollectionMissing(id), - ReadHoldError::SinceViolation(id) => Self::SinceViolation(id), } } } diff --git a/src/storage-types/src/read_holds.rs b/src/storage-types/src/read_holds.rs index 7de24172678ca..87fedaad56487 100644 --- a/src/storage-types/src/read_holds.rs +++ b/src/storage-types/src/read_holds.rs @@ -208,6 +208,4 @@ impl Drop for ReadHold { pub enum ReadHoldError { #[error("collection does not exist: {0}")] CollectionMissing(GlobalId), - #[error("desired read hold frontier is not beyond the since of collection: {0}")] - SinceViolation(GlobalId), } From 0f60b1e72fbd740da1cf5ba1892a01b550d73b3e Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 16:23:46 +0200 Subject: [PATCH 34/55] Error handling 4: Move CollectionMissing from the compute controller to the storage controller in order to be able to use this instead of ReadHoldError in the next commit. --- src/compute-client/src/controller/error.rs | 7 ++----- src/storage-types/src/errors.rs | 8 +++++++- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/compute-client/src/controller/error.rs b/src/compute-client/src/controller/error.rs index e6eb4d025a1fd..ab0b227268a19 100644 --- a/src/compute-client/src/controller/error.rs +++ b/src/compute-client/src/controller/error.rs @@ -23,6 +23,8 @@ use thiserror::Error; use crate::controller::{ComputeInstanceId, ReplicaId}; +pub use mz_storage_types::errors::CollectionMissing; + /// The error returned by replica-targeted peeks and subscribes when the target replica /// disconnects. pub const ERROR_TARGET_REPLICA_FAILED: &str = "target replica failed or was dropped"; @@ -38,11 +40,6 @@ pub struct InstanceMissing(pub ComputeInstanceId); #[error("instance exists already: {0}")] pub struct InstanceExists(pub ComputeInstanceId); -/// Error returned in response to a reference to an unknown compute collection. -#[derive(Error, Debug)] -#[error("collection does not exist: {0}")] -pub struct CollectionMissing(pub GlobalId); - /// Error returned in response to a reference to an unknown compute collection. #[derive(Error, Debug)] #[error("No replicas found in cluster for target list.")] diff --git a/src/storage-types/src/errors.rs b/src/storage-types/src/errors.rs index a1571f472140a..fd29c0622287b 100644 --- a/src/storage-types/src/errors.rs +++ b/src/storage-types/src/errors.rs @@ -15,12 +15,13 @@ use bytes::BufMut; use mz_expr::EvalError; use mz_kafka_util::client::TunnelingClientContext; use mz_proto::{IntoRustIfSome, ProtoType, RustType, TryFromProtoError}; -use mz_repr::Row; +use mz_repr::{GlobalId, Row}; use mz_ssh_util::tunnel::SshTunnelStatus; use proptest_derive::Arbitrary; use prost::Message; use rdkafka::error::KafkaError; use serde::{Deserialize, Serialize}; +use thiserror::Error; use tracing::warn; include!(concat!(env!("OUT_DIR"), "/mz_storage_types.errors.rs")); @@ -1098,6 +1099,11 @@ pub enum CsrConnectError { Other(#[from] anyhow::Error), } +/// Error returned in response to a reference to an unknown collection. +#[derive(Error, Debug)] +#[error("collection does not exist: {0}")] +pub struct CollectionMissing(pub GlobalId); + #[cfg(test)] mod tests { use crate::errors::DecodeErrorKind; From 20f9dd13c8420de6e929ed4ffdec5eba3c3fbe45 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 16:40:06 +0200 Subject: [PATCH 35/55] Error handling 5: Delete ReadHoldError and make use of the fact that acquire_read_holds now returns a common error. --- src/adapter/src/peek_client.rs | 3 +-- src/compute-client/src/as_of_selection.rs | 6 +++--- src/compute-client/src/controller/error.rs | 17 ----------------- src/storage-client/src/storage_collections.rs | 9 +++++---- src/storage-types/src/read_holds.rs | 6 ------ 5 files changed, 9 insertions(+), 32 deletions(-) diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 8a8a976def268..c810ab1967404 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -163,8 +163,7 @@ impl PeekClient { let desired_storage: Vec<_> = id_bundle.storage_ids.iter().copied().collect(); let storage_read_holds = self .storage_collections - .acquire_read_holds(desired_storage) - .expect("missing storage collections"); + .acquire_read_holds(desired_storage)?; read_holds.storage_holds = storage_read_holds .into_iter() .map(|hold| (hold.id(), hold)) diff --git a/src/compute-client/src/as_of_selection.rs b/src/compute-client/src/as_of_selection.rs index 74f41fbc31eba..f791ce5538d3b 100644 --- a/src/compute-client/src/as_of_selection.rs +++ b/src/compute-client/src/as_of_selection.rs @@ -870,11 +870,11 @@ mod tests { use mz_storage_types::connections::inline::InlinedConnection; use mz_storage_types::controller::{CollectionMetadata, StorageError}; use mz_storage_types::parameters::StorageParameters; - use mz_storage_types::read_holds::ReadHoldError; use mz_storage_types::sources::{GenericSourceConnection, SourceDesc}; use mz_storage_types::sources::{SourceData, SourceExportDataConfig}; use mz_storage_types::time_dependence::{TimeDependence, TimeDependenceError}; use timely::progress::Timestamp as TimelyTimestamp; + use mz_storage_types::errors::CollectionMissing; use super::*; @@ -1089,13 +1089,13 @@ mod tests { fn acquire_read_holds( &self, desired_holds: Vec, - ) -> Result>, ReadHoldError> { + ) -> Result>, CollectionMissing> { let mut holds = Vec::with_capacity(desired_holds.len()); for id in desired_holds { let (read, _write) = self .0 .get(&id) - .ok_or(ReadHoldError::CollectionMissing(id))?; + .ok_or(CollectionMissing(id))?; let (tx, _rx) = tokio::sync::mpsc::unbounded_channel(); holds.push(ReadHold::with_channel(id, read.clone(), tx)); } diff --git a/src/compute-client/src/controller/error.rs b/src/compute-client/src/controller/error.rs index ab0b227268a19..b4d5907c9a93e 100644 --- a/src/compute-client/src/controller/error.rs +++ b/src/compute-client/src/controller/error.rs @@ -18,7 +18,6 @@ //! of each method and make it easy for callers to ensure that all possible errors are handled. use mz_repr::GlobalId; -use mz_storage_types::read_holds::ReadHoldError; use thiserror::Error; use crate::controller::{ComputeInstanceId, ReplicaId}; @@ -142,14 +141,6 @@ impl From for DataflowCreationError { } } -impl From for DataflowCreationError { - fn from(error: ReadHoldError) -> Self { - match error { - ReadHoldError::CollectionMissing(id) => Self::CollectionMissing(id), - } - } -} - /// Errors arising during peek processing. #[derive(Error, Debug)] pub enum PeekError { @@ -185,14 +176,6 @@ impl From for PeekError { } } -impl From for PeekError { - fn from(error: ReadHoldError) -> Self { - match error { - ReadHoldError::CollectionMissing(id) => Self::CollectionMissing(id), - } - } -} - impl From for PeekError { fn from(error: crate::controller::instance::PeekError) -> Self { use crate::controller::instance::PeekError::*; diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index ace20dd86e326..987ef691de89d 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -47,7 +47,7 @@ use mz_storage_types::connections::inline::InlinedConnection; use mz_storage_types::controller::{CollectionMetadata, StorageError, TxnsCodecRow}; use mz_storage_types::dyncfgs::STORAGE_DOWNGRADE_SINCE_DURING_FINALIZATION; use mz_storage_types::parameters::StorageParameters; -use mz_storage_types::read_holds::{ReadHold, ReadHoldError}; +use mz_storage_types::read_holds::ReadHold; use mz_storage_types::read_policy::ReadPolicy; use mz_storage_types::sources::{ GenericSourceConnection, SourceData, SourceDesc, SourceEnvelope, SourceExport, @@ -64,6 +64,7 @@ use timely::progress::{Antichain, ChangeBatch, Timestamp as TimelyTimestamp}; use tokio::sync::{mpsc, oneshot}; use tokio::time::MissedTickBehavior; use tracing::{debug, info, trace, warn}; +use mz_storage_types::errors::CollectionMissing; use crate::client::TimestamplessUpdateBuilder; use crate::controller::{ @@ -343,7 +344,7 @@ pub trait StorageCollections: Debug { fn acquire_read_holds( &self, desired_holds: Vec, - ) -> Result>, ReadHoldError>; + ) -> Result>, CollectionMissing>; /// Get the time dependence for a storage collection. Returns no value if unknown or if /// the object isn't managed by storage. @@ -2512,7 +2513,7 @@ where fn acquire_read_holds( &self, desired_holds: Vec, - ) -> Result>, ReadHoldError> { + ) -> Result>, CollectionMissing> { if desired_holds.is_empty() { return Ok(vec![]); } @@ -2533,7 +2534,7 @@ where for id in desired_holds.iter() { let collection = collections .get(id) - .ok_or(ReadHoldError::CollectionMissing(*id))?; + .ok_or(CollectionMissing(*id))?; let since = collection.read_capabilities.frontier().to_owned(); advanced_holds.push((*id, since)); } diff --git a/src/storage-types/src/read_holds.rs b/src/storage-types/src/read_holds.rs index 87fedaad56487..e51729b25ce89 100644 --- a/src/storage-types/src/read_holds.rs +++ b/src/storage-types/src/read_holds.rs @@ -203,9 +203,3 @@ impl Drop for ReadHold { self.release(); } } - -#[derive(Error, Debug)] -pub enum ReadHoldError { - #[error("collection does not exist: {0}")] - CollectionMissing(GlobalId), -} From 71583259b54703b8354090d1b0b750e25c6f811b Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 17:02:45 +0200 Subject: [PATCH 36/55] Error handling 6: Make StorageCollections::collections_frontiers return a more specific error and make use of this in acquire_read_holds_and_least_valid_write --- src/adapter/src/peek_client.rs | 3 +-- src/compute-client/src/as_of_selection.rs | 4 ++-- src/storage-client/src/controller.rs | 9 +++++---- src/storage-client/src/storage_collections.rs | 12 ++++++------ src/storage-controller/src/lib.rs | 6 +++--- src/storage-types/src/controller.rs | 8 +++++++- 6 files changed, 24 insertions(+), 18 deletions(-) diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index c810ab1967404..d5749ac58d1f7 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -172,8 +172,7 @@ impl PeekClient { let storage_ids: Vec<_> = id_bundle.storage_ids.iter().copied().collect(); for f in self .storage_collections - .collections_frontiers(storage_ids) - .expect("missing collections") + .collections_frontiers(storage_ids)? { upper.extend(f.write_frontier); } diff --git a/src/compute-client/src/as_of_selection.rs b/src/compute-client/src/as_of_selection.rs index f791ce5538d3b..bd9d395f0b582 100644 --- a/src/compute-client/src/as_of_selection.rs +++ b/src/compute-client/src/as_of_selection.rs @@ -921,11 +921,11 @@ mod tests { fn collections_frontiers( &self, ids: Vec, - ) -> Result>, StorageError> + ) -> Result>, CollectionMissing> { let mut frontiers = Vec::with_capacity(ids.len()); for id in ids { - let (read, write) = self.0.get(&id).ok_or(StorageError::IdentifierMissing(id))?; + let (read, write) = self.0.get(&id).ok_or(CollectionMissing(id))?; frontiers.push(CollectionFrontiers { id, write_frontier: write.clone(), diff --git a/src/storage-client/src/controller.rs b/src/storage-client/src/controller.rs index 4f464c1ed2816..ff331524c2180 100644 --- a/src/storage-client/src/controller.rs +++ b/src/storage-client/src/controller.rs @@ -57,6 +57,7 @@ use timely::progress::Timestamp as TimelyTimestamp; use timely::progress::frontier::MutableAntichain; use timely::progress::{Antichain, Timestamp}; use tokio::sync::{mpsc, oneshot}; +use mz_storage_types::errors::CollectionMissing; use crate::client::{AppendOnlyUpdate, StatusUpdate, TableData}; use crate::statistics::WebhookStatistics; @@ -329,7 +330,7 @@ pub trait StorageController: Debug { id: GlobalId, ) -> Result< (Antichain, Antichain), - StorageError, + CollectionMissing, >; /// Returns the since/upper frontiers of the identified collections. @@ -349,14 +350,14 @@ pub trait StorageController: Debug { Antichain, Antichain, )>, - StorageError, + CollectionMissing, >; /// Acquire an iterator over [CollectionMetadata] for all active /// collections. /// - /// A collection is "active" when it has a non empty frontier of read - /// capabilties. + /// A collection is "active" when it has a non-empty frontier of read + /// capabilities. fn active_collection_metadatas(&self) -> Vec<(GlobalId, CollectionMetadata)>; /// Returns the IDs of ingestion exports running on the given instance. This diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index 987ef691de89d..936f52611cb32 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -123,7 +123,7 @@ pub trait StorageCollections: Debug { fn collection_frontiers( &self, id: GlobalId, - ) -> Result, StorageError> { + ) -> Result, CollectionMissing> { let frontiers = self .collections_frontiers(vec![id])? .expect_element(|| "known to exist"); @@ -136,12 +136,12 @@ pub trait StorageCollections: Debug { fn collections_frontiers( &self, id: Vec, - ) -> Result>, StorageError>; + ) -> Result>, CollectionMissing>; /// Atomically gets and returns the frontiers of all active collections. /// - /// A collection is "active" when it has a non empty frontier of read - /// capabilties. + /// A collection is "active" when it has a non-empty frontier of read + /// capabilities. fn active_collection_frontiers(&self) -> Vec>; /// Checks whether a collection exists under the given `GlobalId`. Returns @@ -1449,7 +1449,7 @@ where fn collections_frontiers( &self, ids: Vec, - ) -> Result>, StorageError> { + ) -> Result>, CollectionMissing> { if ids.is_empty() { return Ok(vec![]); } @@ -1467,7 +1467,7 @@ where implied_capability: c.implied_capability.clone(), read_capabilities: c.read_capabilities.frontier().to_owned(), }) - .ok_or(StorageError::IdentifierMissing(id)) + .ok_or(CollectionMissing(id)) }) .collect::, _>>()?; diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index 5774a17d42ed6..ab205ac0c8c2c 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -92,7 +92,7 @@ use tokio::sync::{mpsc, oneshot}; use tokio::time::MissedTickBehavior; use tokio::time::error::Elapsed; use tracing::{debug, info, warn}; - +use mz_storage_types::errors::CollectionMissing; use crate::collection_mgmt::{ AppendOnlyIntrospectionConfig, CollectionManagerKind, DifferentialIntrospectionConfig, }; @@ -452,7 +452,7 @@ where id: GlobalId, ) -> Result< (Antichain, Antichain), - StorageError, + CollectionMissing, > { let frontiers = self.storage_collections.collection_frontiers(id)?; Ok((frontiers.implied_capability, frontiers.write_frontier)) @@ -461,7 +461,7 @@ where fn collections_frontiers( &self, mut ids: Vec, - ) -> Result, Antichain)>, StorageError> { + ) -> Result, Antichain)>, CollectionMissing> { let mut result = vec![]; // In theory, we could pull all our frontiers from storage collections... // but in practice those frontiers may not be identical. For historical reasons, we use the diff --git a/src/storage-types/src/controller.rs b/src/storage-types/src/controller.rs index b8bc54f6bf089..48be2ef42abaf 100644 --- a/src/storage-types/src/controller.rs +++ b/src/storage-types/src/controller.rs @@ -24,7 +24,7 @@ use serde::{Deserialize, Serialize}; use timely::progress::Antichain; use tracing::error; -use crate::errors::DataflowError; +use crate::errors::{CollectionMissing, DataflowError}; use crate::instances::StorageInstanceId; use crate::sources::SourceData; @@ -332,6 +332,12 @@ impl From for StorageError { } } +impl From for StorageError { + fn from(err: CollectionMissing) -> Self { + Self::IdentifierMissing(err.0) + } +} + #[derive(Debug)] pub struct TxnsCodecRow; From ef1e4fcffb61d82a66233462e2cd0ed7d079eba4 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 17:07:57 +0200 Subject: [PATCH 37/55] Error handling 7: Make StorageCollections::collection_metadata also return a more specific error and make use of this in acquire_read_holds_and_least_valid_write --- src/adapter/src/error.rs | 11 ++++++++++- src/adapter/src/peek_client.rs | 3 +-- src/compute-client/src/as_of_selection.rs | 2 +- src/storage-client/src/controller.rs | 2 +- src/storage-client/src/storage_collections.rs | 14 +++++++------- src/storage-controller/src/lib.rs | 2 +- 6 files changed, 21 insertions(+), 13 deletions(-) diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index a91164b7b077b..c1e667b127a3a 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -34,7 +34,7 @@ use timely::progress::Antichain; use tokio::sync::oneshot; use tokio_postgres::error::SqlState; use mz_compute_client::controller::error::{InstanceMissing, PeekError}; - +use mz_storage_types::errors::CollectionMissing; use crate::coord::NetworkPolicyError; use crate::optimize::OptimizerError; @@ -1072,4 +1072,13 @@ impl From for AdapterError { } } +impl From for AdapterError { + fn from(e: CollectionMissing) -> Self { + AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: e.0.to_string(), + } + } +} + impl Error for AdapterError {} diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index d5749ac58d1f7..4bd1665c22eea 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -293,8 +293,7 @@ impl PeekClient { let literal_constraints = literal_constraint.map(|r| vec![r]); let metadata = self .storage_collections - .collection_metadata(coll_id) - .expect("storage collection for fast-path peek") + .collection_metadata(coll_id)? .clone(); let peek_target = PeekTarget::Persist { id: coll_id, diff --git a/src/compute-client/src/as_of_selection.rs b/src/compute-client/src/as_of_selection.rs index bd9d395f0b582..b7eef313090aa 100644 --- a/src/compute-client/src/as_of_selection.rs +++ b/src/compute-client/src/as_of_selection.rs @@ -910,7 +910,7 @@ mod tests { fn collection_metadata( &self, _id: GlobalId, - ) -> Result> { + ) -> Result { unimplemented!() } diff --git a/src/storage-client/src/controller.rs b/src/storage-client/src/controller.rs index ff331524c2180..434d532c2c672 100644 --- a/src/storage-client/src/controller.rs +++ b/src/storage-client/src/controller.rs @@ -298,7 +298,7 @@ pub trait StorageController: Debug { fn collection_metadata( &self, id: GlobalId, - ) -> Result>; + ) -> Result; /// Returns `true` iff the given collection/ingestion has been hydrated. /// diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index 936f52611cb32..a4c1ddc576179 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -110,7 +110,7 @@ pub trait StorageCollections: Debug { fn collection_metadata( &self, id: GlobalId, - ) -> Result>; + ) -> Result; /// Acquire an iterator over [CollectionMetadata] for all active /// collections. @@ -1081,7 +1081,7 @@ where { let metadata = match self.collection_metadata(id) { Ok(metadata) => metadata.clone(), - Err(e) => return async { Err(e) }.boxed(), + Err(e) => return async { Err(e.into()) }.boxed(), }; let txns_read = metadata.txns_shard.as_ref().map(|txns_id| { assert_eq!(txns_id, txns_read.txns_id()); @@ -1146,7 +1146,7 @@ where let metadata = match self.collection_metadata(id) { Ok(metadata) => metadata.clone(), - Err(e) => return async { Err(e) }.boxed(), + Err(e) => return async { Err(e.into()) }.boxed(), }; let txns_read = metadata.txns_shard.as_ref().map(|txns_id| { assert_eq!(txns_id, txns_read.txns_id()); @@ -1427,13 +1427,13 @@ where fn collection_metadata( &self, id: GlobalId, - ) -> Result> { + ) -> Result { let collections = self.collections.lock().expect("lock poisoned"); collections .get(&id) .map(|c| c.collection_metadata.clone()) - .ok_or(StorageError::IdentifierMissing(id)) + .ok_or(CollectionMissing(id)) } fn active_collection_metadatas(&self) -> Vec<(GlobalId, CollectionMetadata)> { @@ -1631,7 +1631,7 @@ where { let metadata = match self.collection_metadata(id) { Ok(metadata) => metadata.clone(), - Err(e) => return async { Err(e) }.boxed(), + Err(e) => return async { Err(e.into()) }.boxed(), }; let txns_read = metadata.txns_shard.as_ref().map(|txns_id| { // Ensure the txn's shard the controller has is the same that this @@ -1705,7 +1705,7 @@ where > { let metadata = match self.collection_metadata(id) { Ok(m) => m, - Err(e) => return Box::pin(async move { Err(e) }), + Err(e) => return Box::pin(async move { Err(e.into()) }), }; let persist = Arc::clone(&self.persist); diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index ab205ac0c8c2c..4c3efc58b2c2d 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -327,7 +327,7 @@ where fn collection_metadata( &self, id: GlobalId, - ) -> Result> { + ) -> Result { self.storage_collections.collection_metadata(id) } From 69a8298b5e568caf58d53f95f1b7dc2066127c64 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 17:16:24 +0200 Subject: [PATCH 38/55] Error handling 8 --- src/adapter/src/error.rs | 17 ++++++++++++++++- src/adapter/src/frontend_peek.rs | 3 +-- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index c1e667b127a3a..5c000789f154f 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -33,7 +33,7 @@ use smallvec::SmallVec; use timely::progress::Antichain; use tokio::sync::oneshot; use tokio_postgres::error::SqlState; -use mz_compute_client::controller::error::{InstanceMissing, PeekError}; +use mz_compute_client::controller::error::{CollectionLookupError, InstanceMissing, PeekError}; use mz_storage_types::errors::CollectionMissing; use crate::coord::NetworkPolicyError; use crate::optimize::OptimizerError; @@ -1081,4 +1081,19 @@ impl From for AdapterError { } } +impl From for AdapterError { + fn from(e: CollectionLookupError) -> Self { + match e { + CollectionLookupError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "cluster", + dependency_id: id.to_string(), + }, + CollectionLookupError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: id.to_string(), + }, + } + } +} + impl Error for AdapterError {} diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 374072fc9dc5a..aceb79f61d9b8 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -540,8 +540,7 @@ impl PeekClient { let (read_holds, upper) = self .acquire_read_holds_and_least_valid_write(id_bundle) - .await - .expect("missing collection"); + .await?; let (det, read_holds) = ::determine_timestamp_for_inner( session, id_bundle, From cabe87d71ece9ac92505846d09cff2b7c2208a34 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 17:26:37 +0200 Subject: [PATCH 39/55] Error handling 9: Make conversions to ConcurrentDependencyDrop explicit --- src/adapter/src/error.rs | 106 +++++++++++++++---------------- src/adapter/src/frontend_peek.rs | 2 +- src/adapter/src/peek_client.rs | 6 +- 3 files changed, 55 insertions(+), 59 deletions(-) diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index 5c000789f154f..e08b69d0153fe 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -614,6 +614,57 @@ impl AdapterError { pub fn internal(context: &str, e: E) -> AdapterError { AdapterError::Internal(format!("{context}: {e}")) } + + // We don't want the following error conversions to `ConcurrentDependencyDrop` to happen + // automatically, because it might depend on the context whether `ConcurrentDependencyDrop` + // is appropriate, so we want to make the conversion target explicit at the call site. + // For example, maybe we get an `InstanceMissing` if the user specifies a non-existing cluster, + // in which case `ConcurrentDependencyDrop` would not be appropriate. + pub fn concurrent_dependency_drop_from_instance_missing(e: InstanceMissing) -> Self { + AdapterError::ConcurrentDependencyDrop { + dependency_kind: "cluster", + dependency_id: e.0.to_string(), + } + } + pub fn concurrent_dependency_drop_from_collection_missing(e: CollectionMissing) -> Self { + AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: e.0.to_string(), + } + } + + pub fn concurrent_dependency_drop_from_collection_lookup_error(e: CollectionLookupError) -> Self { + match e { + CollectionLookupError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "cluster", + dependency_id: id.to_string(), + }, + CollectionLookupError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: id.to_string(), + }, + } + } + + pub fn concurrent_dependency_drop_from_peek_error(e: PeekError) -> AdapterError { + match e { + PeekError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop{ + dependency_kind: "cluster", + dependency_id: id.to_string(), + }, + PeekError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: id.to_string(), + }, + PeekError::ReplicaMissing(id) => AdapterError::ConcurrentDependencyDrop { + dependency_kind: "replica", + dependency_id: id.to_string(), + }, + e @ PeekError::SinceViolation(_) => AdapterError::internal("peek error", e), + e @ PeekError::ReadHoldIdMismatch(_) => AdapterError::internal("peek error", e), + e @ PeekError::ReadHoldInsufficient(_) => AdapterError::internal("peek error", e), + } + } } impl fmt::Display for AdapterError { @@ -1041,59 +1092,4 @@ impl From for AdapterError { } } -impl From for AdapterError { - fn from(e: PeekError) -> AdapterError { - match e { - PeekError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop{ - dependency_kind: "cluster", - dependency_id: id.to_string(), - }, - PeekError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { - dependency_kind: "collection", - dependency_id: id.to_string(), - }, - PeekError::ReplicaMissing(id) => AdapterError::ConcurrentDependencyDrop { - dependency_kind: "replica", - dependency_id: id.to_string(), - }, - e @ PeekError::SinceViolation(_) => AdapterError::internal("peek error", e), - e @ PeekError::ReadHoldIdMismatch(_) => AdapterError::internal("peek error", e), - e @ PeekError::ReadHoldInsufficient(_) => AdapterError::internal("peek error", e), - } - } -} - -impl From for AdapterError { - fn from(e: InstanceMissing) -> Self { - AdapterError::ConcurrentDependencyDrop { - dependency_kind: "cluster", - dependency_id: e.0.to_string(), - } - } -} - -impl From for AdapterError { - fn from(e: CollectionMissing) -> Self { - AdapterError::ConcurrentDependencyDrop { - dependency_kind: "collection", - dependency_id: e.0.to_string(), - } - } -} - -impl From for AdapterError { - fn from(e: CollectionLookupError) -> Self { - match e { - CollectionLookupError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop { - dependency_kind: "cluster", - dependency_id: id.to_string(), - }, - CollectionLookupError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { - dependency_kind: "collection", - dependency_id: id.to_string(), - }, - } - } -} - impl Error for AdapterError {} diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index aceb79f61d9b8..2c1ae3a200d5a 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -540,7 +540,7 @@ impl PeekClient { let (read_holds, upper) = self .acquire_read_holds_and_least_valid_write(id_bundle) - .await?; + .await.map_err(AdapterError::concurrent_dependency_drop_from_collection_lookup_error)?; let (det, read_holds) = ::determine_timestamp_for_inner( session, id_bundle, diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 4bd1665c22eea..f01be8b1aa05e 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -293,7 +293,7 @@ impl PeekClient { let literal_constraints = literal_constraint.map(|r| vec![r]); let metadata = self .storage_collections - .collection_metadata(coll_id)? + .collection_metadata(coll_id).map_err(AdapterError::concurrent_dependency_drop_from_collection_missing)? .clone(); let peek_target = PeekTarget::Persist { id: coll_id, @@ -330,7 +330,7 @@ impl PeekClient { // Issue the peek to the instance let client = self .ensure_compute_instance_client(compute_instance) - .await?; + .await.map_err(AdapterError::concurrent_dependency_drop_from_instance_missing)?; let finishing_for_instance = finishing.clone(); client .peek( @@ -345,7 +345,7 @@ impl PeekClient { target_replica, rows_tx, ) - .await?; + .await.map_err(AdapterError::concurrent_dependency_drop_from_peek_error)?; let peek_response_stream = Coordinator::create_peek_response_stream( rows_rx, From a6dff21c130d535810daad5268134665c07477ef Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 17:36:23 +0200 Subject: [PATCH 40/55] cargo fmt and clippy --- src/adapter/src/coord/command_handler.rs | 6 +--- src/adapter/src/error.rs | 28 ++++++++++++------- src/adapter/src/frontend_peek.rs | 3 +- src/adapter/src/peek_client.rs | 16 ++++++----- src/compute-client/src/as_of_selection.rs | 10 ++----- src/compute-client/src/controller.rs | 5 +++- src/compute-client/src/controller/instance.rs | 10 +++++-- src/storage-client/src/controller.rs | 12 ++------ src/storage-client/src/storage_collections.rs | 16 +++-------- src/storage-controller/src/lib.rs | 13 +++------ 10 files changed, 55 insertions(+), 64 deletions(-) diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index b16578d9d9e35..d718b277cc981 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -272,11 +272,7 @@ impl Coordinator { } Command::GetComputeInstanceClient { instance_id, tx } => { - let _ = tx.send( - self.controller - .compute - .instance_client(instance_id), - ); + let _ = tx.send(self.controller.compute.instance_client(instance_id)); } Command::GetOracle { timeline, tx } => { diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index e08b69d0153fe..9e4f6af4c6475 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -16,6 +16,7 @@ use dec::TryFromDecimalError; use itertools::Itertools; use mz_catalog::builtin::MZ_CATALOG_SERVER_CLUSTER; use mz_compute_client::controller::error as compute_error; +use mz_compute_client::controller::error::{CollectionLookupError, InstanceMissing, PeekError}; use mz_expr::EvalError; use mz_ore::error::ErrorExt; use mz_ore::stack::RecursionLimitError; @@ -29,12 +30,12 @@ use mz_sql::rbac; use mz_sql::session::vars::VarError; use mz_storage_types::connections::ConnectionValidationError; use mz_storage_types::controller::StorageError; +use mz_storage_types::errors::CollectionMissing; use smallvec::SmallVec; use timely::progress::Antichain; use tokio::sync::oneshot; use tokio_postgres::error::SqlState; -use mz_compute_client::controller::error::{CollectionLookupError, InstanceMissing, PeekError}; -use mz_storage_types::errors::CollectionMissing; + use crate::coord::NetworkPolicyError; use crate::optimize::OptimizerError; @@ -104,7 +105,7 @@ pub enum AdapterError { /// A dependency was dropped while sequencing a statement. ConcurrentDependencyDrop { dependency_kind: &'static str, - dependency_id: String + dependency_id: String, }, /// Target cluster has no replicas to service query. NoClusterReplicasAvailable { @@ -633,22 +634,26 @@ impl AdapterError { } } - pub fn concurrent_dependency_drop_from_collection_lookup_error(e: CollectionLookupError) -> Self { + pub fn concurrent_dependency_drop_from_collection_lookup_error( + e: CollectionLookupError, + ) -> Self { match e { CollectionLookupError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop { dependency_kind: "cluster", dependency_id: id.to_string(), }, - CollectionLookupError::CollectionMissing(id) => AdapterError::ConcurrentDependencyDrop { - dependency_kind: "collection", - dependency_id: id.to_string(), - }, + CollectionLookupError::CollectionMissing(id) => { + AdapterError::ConcurrentDependencyDrop { + dependency_kind: "collection", + dependency_id: id.to_string(), + } + } } } pub fn concurrent_dependency_drop_from_peek_error(e: PeekError) -> AdapterError { match e { - PeekError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop{ + PeekError::InstanceMissing(id) => AdapterError::ConcurrentDependencyDrop { dependency_kind: "cluster", dependency_id: id.to_string(), }, @@ -730,7 +735,10 @@ impl fmt::Display for AdapterError { dependency_kind, dependency_id, } => { - write!(f, "{dependency_kind} '{dependency_id}' was dropped while executing a statement") + write!( + f, + "{dependency_kind} '{dependency_id}' was dropped while executing a statement" + ) } AdapterError::NoClusterReplicasAvailable { name, .. } => { write!( diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 2c1ae3a200d5a..643ffd2f29235 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -540,7 +540,8 @@ impl PeekClient { let (read_holds, upper) = self .acquire_read_holds_and_least_valid_write(id_bundle) - .await.map_err(AdapterError::concurrent_dependency_drop_from_collection_lookup_error)?; + .await + .map_err(AdapterError::concurrent_dependency_drop_from_collection_lookup_error)?; let (det, read_holds) = ::determine_timestamp_for_inner( session, id_bundle, diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index f01be8b1aa05e..f7d19dd417156 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -84,7 +84,8 @@ impl PeekClient { pub async fn ensure_compute_instance_client( &mut self, compute_instance: ComputeInstanceId, - ) -> Result<&mut mz_compute_client::controller::instance::Client, InstanceMissing> { + ) -> Result<&mut mz_compute_client::controller::instance::Client, InstanceMissing> + { if !self.compute_instances.contains_key(&compute_instance) { let client = self .call_coordinator(|tx| Command::GetComputeInstanceClient { @@ -179,9 +180,7 @@ impl PeekClient { } for (&instance_id, collection_ids) in &id_bundle.compute_ids { - let client = self - .ensure_compute_instance_client(instance_id) - .await?; + let client = self.ensure_compute_instance_client(instance_id).await?; for (id, read_hold, write_frontier) in client .acquire_read_holds_and_collection_write_frontiers( @@ -293,7 +292,8 @@ impl PeekClient { let literal_constraints = literal_constraint.map(|r| vec![r]); let metadata = self .storage_collections - .collection_metadata(coll_id).map_err(AdapterError::concurrent_dependency_drop_from_collection_missing)? + .collection_metadata(coll_id) + .map_err(AdapterError::concurrent_dependency_drop_from_collection_missing)? .clone(); let peek_target = PeekTarget::Persist { id: coll_id, @@ -330,7 +330,8 @@ impl PeekClient { // Issue the peek to the instance let client = self .ensure_compute_instance_client(compute_instance) - .await.map_err(AdapterError::concurrent_dependency_drop_from_instance_missing)?; + .await + .map_err(AdapterError::concurrent_dependency_drop_from_instance_missing)?; let finishing_for_instance = finishing.clone(); client .peek( @@ -345,7 +346,8 @@ impl PeekClient { target_replica, rows_tx, ) - .await.map_err(AdapterError::concurrent_dependency_drop_from_peek_error)?; + .await + .map_err(AdapterError::concurrent_dependency_drop_from_peek_error)?; let peek_response_stream = Coordinator::create_peek_response_stream( rows_rx, diff --git a/src/compute-client/src/as_of_selection.rs b/src/compute-client/src/as_of_selection.rs index b7eef313090aa..3f260000390c0 100644 --- a/src/compute-client/src/as_of_selection.rs +++ b/src/compute-client/src/as_of_selection.rs @@ -869,12 +869,12 @@ mod tests { use mz_storage_types::StorageDiff; use mz_storage_types::connections::inline::InlinedConnection; use mz_storage_types::controller::{CollectionMetadata, StorageError}; + use mz_storage_types::errors::CollectionMissing; use mz_storage_types::parameters::StorageParameters; use mz_storage_types::sources::{GenericSourceConnection, SourceDesc}; use mz_storage_types::sources::{SourceData, SourceExportDataConfig}; use mz_storage_types::time_dependence::{TimeDependence, TimeDependenceError}; use timely::progress::Timestamp as TimelyTimestamp; - use mz_storage_types::errors::CollectionMissing; use super::*; @@ -921,8 +921,7 @@ mod tests { fn collections_frontiers( &self, ids: Vec, - ) -> Result>, CollectionMissing> - { + ) -> Result>, CollectionMissing> { let mut frontiers = Vec::with_capacity(ids.len()); for id in ids { let (read, write) = self.0.get(&id).ok_or(CollectionMissing(id))?; @@ -1092,10 +1091,7 @@ mod tests { ) -> Result>, CollectionMissing> { let mut holds = Vec::with_capacity(desired_holds.len()); for id in desired_holds { - let (read, _write) = self - .0 - .get(&id) - .ok_or(CollectionMissing(id))?; + let (read, _write) = self.0.get(&id).ok_or(CollectionMissing(id))?; let (tx, _rx) = tokio::sync::mpsc::unbounded_channel(); holds.push(ReadHold::with_channel(id, read.clone(), tx)); } diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 3c497f2a8bd46..663181d07c007 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -1087,7 +1087,10 @@ impl InstanceState { F: FnOnce(&mut Instance) -> R + Send + 'static, R: Send + 'static, { - self.client.call_sync(f).await.expect("controller should validate") + self.client + .call_sync(f) + .await + .expect("controller should validate") } /// Acquires a [`ReadHold`] for the identified compute collection. diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index f7340f470fc47..b7242e1fdec98 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -51,7 +51,10 @@ use tokio::sync::{mpsc, oneshot}; use tracing::debug_span; use uuid::Uuid; -use crate::controller::error::{CollectionLookupError, CollectionMissing, ERROR_TARGET_REPLICA_FAILED, HydrationCheckBadTarget, InstanceMissing}; +use crate::controller::error::{ + CollectionLookupError, CollectionMissing, ERROR_TARGET_REPLICA_FAILED, HydrationCheckBadTarget, + InstanceMissing, +}; use crate::controller::replica::{ReplicaClient, ReplicaConfig}; use crate::controller::{ ComputeControllerResponse, ComputeControllerTimestamp, IntrospectionUpdates, PeekNotification, @@ -179,7 +182,7 @@ impl Client { })) .map_err(|_send_error| InstanceMissing(self.id))?; - Ok(rx.await.map_err(|_| InstanceMissing(self.id))?) + rx.await.map_err(|_| InstanceMissing(self.id)) } } @@ -287,7 +290,8 @@ where peek_response_tx, ) }) - .await.map(|r| r.map_err(|e| e.into()))? + .await + .map(|r| r.map_err(|e| e.into()))? } } diff --git a/src/storage-client/src/controller.rs b/src/storage-client/src/controller.rs index 434d532c2c672..9e2cbd4792b2d 100644 --- a/src/storage-client/src/controller.rs +++ b/src/storage-client/src/controller.rs @@ -42,6 +42,7 @@ use mz_repr::{Datum, Diff, GlobalId, RelationDesc, RelationVersion, Row}; use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::connections::inline::InlinedConnection; use mz_storage_types::controller::{CollectionMetadata, StorageError}; +use mz_storage_types::errors::CollectionMissing; use mz_storage_types::instances::StorageInstanceId; use mz_storage_types::oneshot_sources::{OneshotIngestionRequest, OneshotResultCallback}; use mz_storage_types::parameters::StorageParameters; @@ -57,7 +58,6 @@ use timely::progress::Timestamp as TimelyTimestamp; use timely::progress::frontier::MutableAntichain; use timely::progress::{Antichain, Timestamp}; use tokio::sync::{mpsc, oneshot}; -use mz_storage_types::errors::CollectionMissing; use crate::client::{AppendOnlyUpdate, StatusUpdate, TableData}; use crate::statistics::WebhookStatistics; @@ -295,10 +295,7 @@ pub trait StorageController: Debug { fn config(&self) -> &StorageConfiguration; /// Returns the [CollectionMetadata] of the collection identified by `id`. - fn collection_metadata( - &self, - id: GlobalId, - ) -> Result; + fn collection_metadata(&self, id: GlobalId) -> Result; /// Returns `true` iff the given collection/ingestion has been hydrated. /// @@ -328,10 +325,7 @@ pub trait StorageController: Debug { fn collection_frontiers( &self, id: GlobalId, - ) -> Result< - (Antichain, Antichain), - CollectionMissing, - >; + ) -> Result<(Antichain, Antichain), CollectionMissing>; /// Returns the since/upper frontiers of the identified collections. /// diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index a4c1ddc576179..607b79d7ac4a9 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -46,6 +46,7 @@ use mz_storage_types::connections::ConnectionContext; use mz_storage_types::connections::inline::InlinedConnection; use mz_storage_types::controller::{CollectionMetadata, StorageError, TxnsCodecRow}; use mz_storage_types::dyncfgs::STORAGE_DOWNGRADE_SINCE_DURING_FINALIZATION; +use mz_storage_types::errors::CollectionMissing; use mz_storage_types::parameters::StorageParameters; use mz_storage_types::read_holds::ReadHold; use mz_storage_types::read_policy::ReadPolicy; @@ -64,7 +65,6 @@ use timely::progress::{Antichain, ChangeBatch, Timestamp as TimelyTimestamp}; use tokio::sync::{mpsc, oneshot}; use tokio::time::MissedTickBehavior; use tracing::{debug, info, trace, warn}; -use mz_storage_types::errors::CollectionMissing; use crate::client::TimestamplessUpdateBuilder; use crate::controller::{ @@ -107,10 +107,7 @@ pub trait StorageCollections: Debug { fn update_parameters(&self, config_params: StorageParameters); /// Returns the [CollectionMetadata] of the collection identified by `id`. - fn collection_metadata( - &self, - id: GlobalId, - ) -> Result; + fn collection_metadata(&self, id: GlobalId) -> Result; /// Acquire an iterator over [CollectionMetadata] for all active /// collections. @@ -1424,10 +1421,7 @@ where .update(config_params); } - fn collection_metadata( - &self, - id: GlobalId, - ) -> Result { + fn collection_metadata(&self, id: GlobalId) -> Result { let collections = self.collections.lock().expect("lock poisoned"); collections @@ -2532,9 +2526,7 @@ where // to pass around ReadHold tokens, we might tighten this up and instead // acquire read holds at the implied capability. for id in desired_holds.iter() { - let collection = collections - .get(id) - .ok_or(CollectionMissing(*id))?; + let collection = collections.get(id).ok_or(CollectionMissing(*id))?; let since = collection.read_capabilities.frontier().to_owned(); advanced_holds.push((*id, since)); } diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index 4c3efc58b2c2d..57479868e58f7 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -69,6 +69,7 @@ use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::connections::ConnectionContext; use mz_storage_types::connections::inline::InlinedConnection; use mz_storage_types::controller::{AlterError, CollectionMetadata, StorageError, TxnsCodecRow}; +use mz_storage_types::errors::CollectionMissing; use mz_storage_types::instances::StorageInstanceId; use mz_storage_types::oneshot_sources::{OneshotIngestionRequest, OneshotResultCallback}; use mz_storage_types::parameters::StorageParameters; @@ -92,7 +93,7 @@ use tokio::sync::{mpsc, oneshot}; use tokio::time::MissedTickBehavior; use tokio::time::error::Elapsed; use tracing::{debug, info, warn}; -use mz_storage_types::errors::CollectionMissing; + use crate::collection_mgmt::{ AppendOnlyIntrospectionConfig, CollectionManagerKind, DifferentialIntrospectionConfig, }; @@ -324,10 +325,7 @@ where &self.config } - fn collection_metadata( - &self, - id: GlobalId, - ) -> Result { + fn collection_metadata(&self, id: GlobalId) -> Result { self.storage_collections.collection_metadata(id) } @@ -450,10 +448,7 @@ where fn collection_frontiers( &self, id: GlobalId, - ) -> Result< - (Antichain, Antichain), - CollectionMissing, - > { + ) -> Result<(Antichain, Antichain), CollectionMissing> { let frontiers = self.storage_collections.collection_frontiers(id)?; Ok((frontiers.implied_capability, frontiers.write_frontier)) } From ffb308a21c44586d07a072a8e4879324e06689d5 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 18:29:30 +0200 Subject: [PATCH 41/55] minor --- src/adapter/src/frontend_peek.rs | 2 ++ src/adapter/src/peek_client.rs | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 643ffd2f29235..e534ba2a6d46b 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -95,6 +95,8 @@ impl PeekClient { let (stmt, params) = { let portal = session .get_portal_unverified(portal_name) + // The portal is a session-level thing, so it couldn't have concurrently disappeared + // since the above verification. .expect("called verify_portal above"); let params = portal.parameters.clone(); let stmt = portal.stmt.clone(); diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index f7d19dd417156..91eb09d8e7012 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -138,7 +138,8 @@ impl PeekClient { { let (tx, rx) = oneshot::channel(); self.coordinator_client.send(f(tx)); - rx.await.expect("sender dropped") + rx.await + .expect("if the coordinator is still alive, it shouldn't have dropped our call") } /// Acquire read holds on the given compute/storage collections, and From c8727483814382a2daf945e53d6ae0ba09e0c579 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 19:22:59 +0200 Subject: [PATCH 42/55] Ignore AdapterError::ConcurrentDependencyDrop in Parallel Workload DDL --- misc/python/materialize/parallel_workload/action.py | 1 + 1 file changed, 1 insertion(+) diff --git a/misc/python/materialize/parallel_workload/action.py b/misc/python/materialize/parallel_workload/action.py index bf2b49dafc399..edd699c496665 100644 --- a/misc/python/materialize/parallel_workload/action.py +++ b/misc/python/materialize/parallel_workload/action.py @@ -172,6 +172,7 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: "real-time source dropped before ingesting the upstream system's visible frontier", # Expected, see https://buildkite.com/materialize/nightly/builds/9399#0191be17-1f4c-4321-9b51-edc4b08b71c5 "object state changed while transaction was in progress", # Old error msg, can remove this ignore later "another session modified the catalog while this DDL transaction was open", + "was dropped while executing a statement", ] ) if exe.db.scenario == Scenario.Cancel: From 8daaf4f6398cd788fae25c0200978b5b511db08a Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 19:49:17 +0200 Subject: [PATCH 43/55] Version guards for enable_frontend_peek_sequencing alters --- .../checks/all_checks/statement_logging.py | 2 +- test/cluster/mzcompose.py | 12 ++++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/misc/python/materialize/checks/all_checks/statement_logging.py b/misc/python/materialize/checks/all_checks/statement_logging.py index 478de63fbe0b0..78a1d1f818dad 100644 --- a/misc/python/materialize/checks/all_checks/statement_logging.py +++ b/misc/python/materialize/checks/all_checks/statement_logging.py @@ -21,7 +21,7 @@ def initialize(self) -> Testdrive: $ postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET statement_logging_max_sample_rate TO 1.0 - $ postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} + $[version>=16200] postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_frontend_peek_sequencing = false; """ ) diff --git a/test/cluster/mzcompose.py b/test/cluster/mzcompose.py index 085bf49e161a4..6c8ef321ac270 100644 --- a/test/cluster/mzcompose.py +++ b/test/cluster/mzcompose.py @@ -3688,17 +3688,25 @@ def workflow_statement_logging(c: Composition, parser: WorkflowArgumentParser) - ): c.up("materialized") - # TODO(peek-seq): enable_frontend_peek_sequencing when it supports statement logging. c.sql( """ ALTER SYSTEM SET statement_logging_max_sample_rate = 1.0; ALTER SYSTEM SET statement_logging_default_sample_rate = 1.0; - ALTER SYSTEM SET enable_frontend_peek_sequencing = false; """, port=6877, user="mz_system", ) + # TODO(peek-seq): enable_frontend_peek_sequencing when it supports statement logging. + c.testdrive( + input=dedent( + """ + $[version>=16200] postgres-execute connection=postgres://mz_system@${testdrive.materialize-internal-sql-addr} + ALTER SYSTEM SET enable_frontend_peek_sequencing = false; + """ + ) + ) + c.run_testdrive_files("statement-logging/statement-logging.td") From 38f928739bb38319ae5934d3f3a3ec5dbbbf0661 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 18 Oct 2025 19:58:30 +0200 Subject: [PATCH 44/55] Disable frontend peek sequencing in balancerd tests because of mz_sessions / waiting_on_startup_appends --- test/balancerd/mzcompose.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/test/balancerd/mzcompose.py b/test/balancerd/mzcompose.py index dc4f4e4ea1e62..ea81df0894ed5 100644 --- a/test/balancerd/mzcompose.py +++ b/test/balancerd/mzcompose.py @@ -200,6 +200,16 @@ def grant_all_admin_user(c: Composition): ) +def disable_frontend_peek_sequencing(c: Composition): + # This should be called if a test uses mz_sessions. + # TODO(peek-seq): This won't be needed once `waiting_on_startup_appends` is fixed in the new peek sequencing. + sql_cursor(c) + mz_system_cursor = c.sql_cursor(service="materialized", port=6877, user="mz_system") + mz_system_cursor.execute( + "ALTER SYSTEM SET enable_frontend_peek_sequencing = false;" + ) + + # Assert that contains is present in balancer metrics. def assert_metrics(c: Composition, contains: str): result = c.exec( @@ -358,6 +368,8 @@ def workflow_ip_forwarding(c: Composition) -> None: if ip != "127.0.0.1" ][0] + disable_frontend_peek_sequencing(c) + r = requests.post( f"https://localhost:{balancer_port}/api/sql", headers={}, From a3270a2859b876a8dc7d1f7511a1e38a47209223 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 19 Oct 2025 18:05:17 +0200 Subject: [PATCH 45/55] enable_frontend_peek_sequencing adjustments --- misc/python/materialize/mzcompose/__init__.py | 10 +++++++++- misc/python/materialize/parallel_workload/action.py | 4 ++++ src/sql/src/session/vars/definitions.rs | 2 +- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/misc/python/materialize/mzcompose/__init__.py b/misc/python/materialize/mzcompose/__init__.py index 1f314f17c89bc..b1320da1cd25f 100644 --- a/misc/python/materialize/mzcompose/__init__.py +++ b/misc/python/materialize/mzcompose/__init__.py @@ -67,7 +67,9 @@ def get_minimal_system_parameters( version: MzVersion, ) -> dict[str, str]: """Settings we need in order to have tests run at all, but otherwise stay - with the defaults: not changing performance or increasing coverage.""" + with the defaults: not changing performance or increasing coverage. + Note: This is not used unless we explicitly select "System Parameters: Minimal" in trigger-ci. + """ return { # ----- @@ -130,6 +132,7 @@ class VariableSystemParameter: # TODO: The linter should check this too +# Note: Only the default is tested unless we explicitly select "System Parameters: Random" in trigger-ci. def get_variable_system_parameters( version: MzVersion, force_source_table_syntax: bool, @@ -186,6 +189,11 @@ def get_variable_system_parameters( "true", ["true", "false"], ), + VariableSystemParameter( + "enable_frontend_peek_sequencing", + "false", + ["true", "false"], + ), VariableSystemParameter( "kafka_default_metadata_fetch_interval", "1s", diff --git a/misc/python/materialize/parallel_workload/action.py b/misc/python/materialize/parallel_workload/action.py index edd699c496665..08e1580087822 100644 --- a/misc/python/materialize/parallel_workload/action.py +++ b/misc/python/materialize/parallel_workload/action.py @@ -1323,6 +1323,10 @@ def __init__( "314572800", # 300 MiB, the production value ] self.flags_with_values["cluster"] = ["quickstart", "dont_exist"] + self.flags_with_values["enable_frontend_peek_sequencing"] = [ + "true", + "false", + ] # If you are adding a new config flag in Materialize, consider using it # here instead of just marking it as uninteresting to silence the diff --git a/src/sql/src/session/vars/definitions.rs b/src/sql/src/session/vars/definitions.rs index c2de13b29bf7b..b17e46d3871f8 100644 --- a/src/sql/src/session/vars/definitions.rs +++ b/src/sql/src/session/vars/definitions.rs @@ -2210,7 +2210,7 @@ feature_flags!( { name: enable_frontend_peek_sequencing, // currently, changes only take effect for new sessions desc: "Enables the new peek sequencing code, which does most of its work in the Adapter Frontend instead of the Coordinator main task.", - default: true, ////////// todo: default to false before merging the PR. (but figure out what to do in CI) + default: false, enable_for_item_parsing: false, }, ); From e41408596955e619764f0ff6ad37c297d37ee581 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Wed, 22 Oct 2025 13:56:58 +0200 Subject: [PATCH 46/55] Tweak error msg --- src/adapter/src/error.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index 9e4f6af4c6475..22df31d375621 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -737,7 +737,7 @@ impl fmt::Display for AdapterError { } => { write!( f, - "{dependency_kind} '{dependency_id}' was dropped while executing a statement" + "{dependency_kind} '{dependency_id}' was dropped" ) } AdapterError::NoClusterReplicasAvailable { name, .. } => { From 5a605fb0242367c3c5f5b97f5528e06e25bb56e4 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 25 Oct 2025 16:47:02 +0200 Subject: [PATCH 47/55] Lighten some catalog references We had a bunch of places with a `&mut Catalog` or an `Arc` where a simple `&Catalog` is enough. `&Catalog` is better performance-wise than `&mut Catalog`, because creating the latter (with calling `catalog_mut`) clones the whole catalog if there are any other Arcs to it. Similarly, `&Catalog` is also better than an `Arc`, because the latter's existence can cause other `catalog_mut` calls to make a clone. --- src/adapter/src/coord/command_handler.rs | 6 ++--- src/adapter/src/coord/sequencer.rs | 2 +- src/adapter/src/coord/sequencer/inner.rs | 23 ++++++++----------- .../src/coord/sequencer/inner/cluster.rs | 2 +- .../sequencer/inner/create_continual_task.rs | 2 +- .../src/coord/sequencer/inner/create_index.rs | 2 +- .../inner/create_materialized_view.rs | 2 +- .../src/coord/sequencer/inner/create_view.rs | 2 +- .../src/coord/sequencer/inner/secret.rs | 2 +- src/adapter/src/error.rs | 5 +--- 10 files changed, 20 insertions(+), 28 deletions(-) diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index d718b277cc981..5b48bc2a9477b 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -1170,8 +1170,7 @@ impl Coordinator { Err(e) => return ctx.retire(Err(e)), }; - let owned_catalog = self.owned_catalog(); - let catalog = owned_catalog.for_session(ctx.session()); + let catalog = self.catalog().for_session(ctx.session()); purify_create_materialized_view_options( catalog, @@ -1216,8 +1215,7 @@ impl Coordinator { Err(e) => return ctx.retire(Err(e)), }; - let owned_catalog = self.owned_catalog(); - let catalog = owned_catalog.for_session(ctx.session()); + let catalog = self.catalog().for_session(ctx.session()); purify_create_materialized_view_options( catalog, diff --git a/src/adapter/src/coord/sequencer.rs b/src/adapter/src/coord/sequencer.rs index 383643ecd6114..e41c10393096e 100644 --- a/src/adapter/src/coord/sequencer.rs +++ b/src/adapter/src/coord/sequencer.rs @@ -187,7 +187,7 @@ impl Coordinator { Plan::CreateSource(plan) => { let id_ts = self.get_catalog_write_ts().await; let (item_id, global_id) = - return_if_err!(self.catalog_mut().allocate_user_id(id_ts).await, ctx); + return_if_err!(self.catalog().allocate_user_id(id_ts).await, ctx); let result = self .sequence_create_source( &mut ctx, diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index ba99fb7b8bc78..80772bc3e2420 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -464,7 +464,7 @@ impl Coordinator { let id_ts = self.get_catalog_write_ts().await; let ids = self - .catalog_mut() + .catalog() .allocate_user_ids(u64::cast_from(subsource_stmts.len()), id_ts) .await?; for (subsource_stmt, (item_id, global_id)) in @@ -540,7 +540,7 @@ impl Coordinator { // collection first. assert_none!(progress_stmt.of_source); let id_ts = self.get_catalog_write_ts().await; - let (item_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (item_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; let progress_plan = self.plan_subsource(ctx.session(), ¶ms, progress_stmt, item_id, global_id)?; let progress_full_name = self @@ -586,7 +586,7 @@ impl Coordinator { }; let id_ts = self.get_catalog_write_ts().await; - let (item_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (item_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; let source_full_name = self.catalog().resolve_full_name(&source_plan.name, None); let of_source = ResolvedItemName::Item { @@ -621,7 +621,7 @@ impl Coordinator { // 3. Finally, plan all the subsources let id_ts = self.get_catalog_write_ts().await; let ids = self - .catalog_mut() + .catalog() .allocate_user_ids(u64::cast_from(subsource_stmts.len()), id_ts) .await?; for (stmt, (item_id, global_id)) in subsource_stmts.into_iter().zip_eq(ids.into_iter()) { @@ -834,8 +834,7 @@ impl Coordinator { resolved_ids: ResolvedIds, ) { let id_ts = self.get_catalog_write_ts().await; - let (connection_id, connection_gid) = match self.catalog_mut().allocate_user_id(id_ts).await - { + let (connection_id, connection_gid) = match self.catalog().allocate_user_id(id_ts).await { Ok(item_id) => item_id, Err(err) => return ctx.retire(Err(err.into())), }; @@ -1097,10 +1096,8 @@ impl Coordinator { plan::AlterNetworkPolicyPlan { id, name, rules }: plan::AlterNetworkPolicyPlan, ) -> Result { // TODO(network_policy): Consider role based network policies here. - let current_network_policy_name = self - .owned_catalog() - .system_config() - .default_network_policy_name(); + let current_network_policy_name = + self.catalog().system_config().default_network_policy_name(); // Check if the way we're alerting the policy is still valid for the current connection. if current_network_policy_name == name { self.validate_alter_network_policy(session, &rules)?; @@ -1136,7 +1133,7 @@ impl Coordinator { None }; let id_ts = self.get_catalog_write_ts().await; - let (table_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (table_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; let collections = [(RelationVersion::root(), global_id)].into_iter().collect(); let data_source = match table.data_source { @@ -1394,7 +1391,7 @@ impl Coordinator { // First try to allocate an ID and an OID. If either fails, we're done. let id_ts = self.get_catalog_write_ts().await; let (item_id, global_id) = - return_if_err!(self.catalog_mut().allocate_user_id(id_ts).await, ctx); + return_if_err!(self.catalog().allocate_user_id(id_ts).await, ctx); let catalog_sink = Sink { create_sql: sink.create_sql, @@ -1511,7 +1508,7 @@ impl Coordinator { resolved_ids: ResolvedIds, ) -> Result { let id_ts = self.get_catalog_write_ts().await; - let (item_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (item_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; let typ = Type { create_sql: Some(plan.typ.create_sql), global_id, diff --git a/src/adapter/src/coord/sequencer/inner/cluster.rs b/src/adapter/src/coord/sequencer/inner/cluster.rs index 7c7296de73af7..0e1e9349f8c6c 100644 --- a/src/adapter/src/coord/sequencer/inner/cluster.rs +++ b/src/adapter/src/coord/sequencer/inner/cluster.rs @@ -602,7 +602,7 @@ impl Coordinator { tracing::debug!("sequence_create_cluster"); let id_ts = self.get_catalog_write_ts().await; - let id = self.catalog_mut().allocate_user_cluster_id(id_ts).await?; + let id = self.catalog().allocate_user_cluster_id(id_ts).await?; // The catalog items for the introspection sources are shared between all replicas // of a compute instance, so we create them unconditionally during instance creation. // Whether a replica actually maintains introspection arrangements is determined by the diff --git a/src/adapter/src/coord/sequencer/inner/create_continual_task.rs b/src/adapter/src/coord/sequencer/inner/create_continual_task.rs index 8be303b528013..a5a8a034179a4 100644 --- a/src/adapter/src/coord/sequencer/inner/create_continual_task.rs +++ b/src/adapter/src/coord/sequencer/inner/create_continual_task.rs @@ -60,7 +60,7 @@ impl Coordinator { // Put a placeholder in the catalog so the optimizer can find something // for the sink_id. let id_ts = self.get_catalog_write_ts().await; - let (item_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (item_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; let collections = [(RelationVersion::root(), global_id)].into_iter().collect(); let entry = CatalogEntry { diff --git a/src/adapter/src/coord/sequencer/inner/create_index.rs b/src/adapter/src/coord/sequencer/inner/create_index.rs index 7abbc8dcaefc4..829f38fce040c 100644 --- a/src/adapter/src/coord/sequencer/inner/create_index.rs +++ b/src/adapter/src/coord/sequencer/inner/create_index.rs @@ -313,7 +313,7 @@ impl Coordinator { .expect("compute instance does not exist"); let (item_id, global_id) = if let ExplainContext::None = explain_ctx { let id_ts = self.get_catalog_write_ts().await; - self.catalog_mut().allocate_user_id(id_ts).await? + self.catalog().allocate_user_id(id_ts).await? } else { self.allocate_transient_id() }; diff --git a/src/adapter/src/coord/sequencer/inner/create_materialized_view.rs b/src/adapter/src/coord/sequencer/inner/create_materialized_view.rs index ebe4a7b6efaa2..75412fb8c50f9 100644 --- a/src/adapter/src/coord/sequencer/inner/create_materialized_view.rs +++ b/src/adapter/src/coord/sequencer/inner/create_materialized_view.rs @@ -433,7 +433,7 @@ impl Coordinator { .expect("compute instance does not exist"); let (item_id, global_id) = if let ExplainContext::None = explain_ctx { let id_ts = self.get_catalog_write_ts().await; - self.catalog_mut().allocate_user_id(id_ts).await? + self.catalog().allocate_user_id(id_ts).await? } else { self.allocate_transient_id() }; diff --git a/src/adapter/src/coord/sequencer/inner/create_view.rs b/src/adapter/src/coord/sequencer/inner/create_view.rs index ecbe73c5b42ef..b89c9beb20871 100644 --- a/src/adapter/src/coord/sequencer/inner/create_view.rs +++ b/src/adapter/src/coord/sequencer/inner/create_view.rs @@ -286,7 +286,7 @@ impl Coordinator { }: CreateViewOptimize, ) -> Result>, AdapterError> { let id_ts = self.get_catalog_write_ts().await; - let (item_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (item_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; // Collect optimizer parameters. let optimizer_config = optimize::OptimizerConfig::from(self.catalog().system_config()) diff --git a/src/adapter/src/coord/sequencer/inner/secret.rs b/src/adapter/src/coord/sequencer/inner/secret.rs index 87d144fdfc34e..6e4f1fbdb82f8 100644 --- a/src/adapter/src/coord/sequencer/inner/secret.rs +++ b/src/adapter/src/coord/sequencer/inner/secret.rs @@ -118,7 +118,7 @@ impl Coordinator { CreateSecretEnsure { validity, mut plan }: CreateSecretEnsure, ) -> Result>, AdapterError> { let id_ts = self.get_catalog_write_ts().await; - let (item_id, global_id) = self.catalog_mut().allocate_user_id(id_ts).await?; + let (item_id, global_id) = self.catalog().allocate_user_id(id_ts).await?; let secrets_controller = Arc::clone(&self.secrets_controller); let payload = self.extract_secret(session, &mut plan.secret.secret_as)?; diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index 22df31d375621..c09c306af5dd6 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -735,10 +735,7 @@ impl fmt::Display for AdapterError { dependency_kind, dependency_id, } => { - write!( - f, - "{dependency_kind} '{dependency_id}' was dropped" - ) + write!(f, "{dependency_kind} '{dependency_id}' was dropped") } AdapterError::NoClusterReplicasAvailable { name, .. } => { write!( From ebfe43b41f02a9521a42172726024ac0b53586a5 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 25 Oct 2025 18:23:07 +0200 Subject: [PATCH 48/55] Move catalog_snapshot outside try_frontend_peek_inner so that the next commit needs to work only with SessionClient's catalog_snapshot, and not with PeekClient's catalog_snapshot (which is deleted in this commit). --- src/adapter/src/client.rs | 9 ++++++++- src/adapter/src/frontend_peek.rs | 11 ++--------- src/adapter/src/peek_client.rs | 18 +----------------- 3 files changed, 11 insertions(+), 27 deletions(-) diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 5e0777f3fa2a7..c326787e06374 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -1106,9 +1106,16 @@ impl SessionClient { portal_name: &str, ) -> Result, AdapterError> { if self.enable_frontend_peek_sequencing { + // TODO(peek-seq): This snapshot is wasted when we end up bailing out from the frontend peek + // sequencing. We could solve this is with that optimization where we + // continuously keep a catalog snapshot in the session, and only get a new one when the + // catalog revision has changed, which we could see with an atomic read. + // But anyhow, this problem will just go away when we reach the point that we never fall + // back to the old sequencing. + let catalog = self.catalog_snapshot("try_frontend_peek_inner").await; let session = self.session.as_mut().expect("SessionClient invariant"); self.peek_client - .try_frontend_peek_inner(portal_name, session) + .try_frontend_peek_inner(portal_name, session, catalog) .await } else { Ok(None) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index e534ba2a6d46b..8af81bc680c0b 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -27,7 +27,7 @@ use opentelemetry::trace::TraceContextExt; use tracing::{Span, debug}; use tracing_opentelemetry::OpenTelemetrySpanExt; -use crate::catalog::CatalogState; +use crate::catalog::{Catalog, CatalogState}; use crate::coord::peek::PeekPlan; use crate::coord::timestamp_selection::TimestampDetermination; use crate::coord::{Coordinator, ExplainContext, TargetCluster}; @@ -45,6 +45,7 @@ impl PeekClient { &mut self, portal_name: &str, session: &mut Session, + catalog: Arc, ) -> Result, AdapterError> { if session.transaction().is_in_multi_statement_transaction() { // TODO(peek-seq): handle multi-statement transactions @@ -75,14 +76,6 @@ impl PeekClient { } } - // TODO(peek-seq): This snapshot is wasted when we end up bailing out from the frontend peek - // sequencing. We could solve this is with that optimization where we - // continuously keep a catalog snapshot in the session, and only get a new one when the - // catalog revision has changed, which we could see with an atomic read. - // But anyhow, this problem will just go away when we reach the point that we never fall - // back to the old sequencing. - let catalog = self.catalog_snapshot("try_frontend_peek_inner").await; - if let Err(_) = Coordinator::verify_portal(&*catalog, session, portal_name) { // TODO(peek-seq): Don't fall back to the coordinator's peek sequencing here, but retire already. debug!( diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 91eb09d8e7012..b5a3319e90a42 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -28,8 +28,7 @@ use timely::progress::Antichain; use tokio::sync::oneshot; use uuid::Uuid; -use crate::catalog::Catalog; -use crate::command::{CatalogSnapshot, Command}; +use crate::command::Command; use crate::coord::Coordinator; use crate::coord::peek::FastPathPlan; use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds, statement_logging}; @@ -117,21 +116,6 @@ impl PeekClient { Ok(self.oracles.get_mut(&timeline).expect("ensured above")) } - /// Fetch a snapshot of the catalog for use in frontend peek sequencing. - /// Records the time taken in the adapter metrics, labeled by `context`. - pub async fn catalog_snapshot(&self, context: &str) -> Arc { - let start = std::time::Instant::now(); - let CatalogSnapshot { catalog } = self - .call_coordinator(|tx| Command::CatalogSnapshot { tx }) - .await; - self.coordinator_client - .metrics() - .catalog_snapshot_seconds - .with_label_values(&[context]) - .observe(start.elapsed().as_secs_f64()); - catalog - } - async fn call_coordinator(&self, f: F) -> T where F: FnOnce(oneshot::Sender) -> Command, From d3de56550848a966a6d7d7e212f54fec13628c8e Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 25 Oct 2025 18:37:04 +0200 Subject: [PATCH 49/55] Cache catalog snapshots in SessionClient --- src/adapter/src/catalog.rs | 8 ++++++- src/adapter/src/catalog/open.rs | 2 ++ src/adapter/src/catalog/transact.rs | 2 ++ src/adapter/src/client.rs | 34 +++++++++++++++++----------- src/environmentd/src/http/catalog.rs | 4 ++-- 5 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/adapter/src/catalog.rs b/src/adapter/src/catalog.rs index d7251011706f8..c2a6ad8cc3cfb 100644 --- a/src/adapter/src/catalog.rs +++ b/src/adapter/src/catalog.rs @@ -15,7 +15,7 @@ use std::borrow::Cow; use std::collections::{BTreeMap, BTreeSet, VecDeque}; use std::convert; use std::sync::Arc; - +use std::sync::atomic::{AtomicU64, Ordering}; use futures::future::BoxFuture; use futures::{Future, FutureExt}; use itertools::Itertools; @@ -140,6 +140,7 @@ pub struct Catalog { expr_cache_handle: Option, storage: Arc>>, transient_revision: u64, + latest_transient_revision: Arc, } // Implement our own Clone because derive can't unless S is Clone, which it's @@ -152,6 +153,7 @@ impl Clone for Catalog { expr_cache_handle: self.expr_cache_handle.clone(), storage: Arc::clone(&self.storage), transient_revision: self.transient_revision, + latest_transient_revision: Arc::clone(&self.latest_transient_revision), } } } @@ -490,6 +492,10 @@ impl Catalog { self.transient_revision } + pub fn latest_transient_revision(&self) -> u64 { + self.latest_transient_revision.load(Ordering::SeqCst) ////////// Is the ordering ok? + } + /// Creates a debug catalog from the current /// `METADATA_BACKEND_URL` with parameters set appropriately for debug contexts, /// like in tests. diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 89eb1553ecdc5..2e1486d542383 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -13,6 +13,7 @@ mod builtin_item_migration; use std::collections::{BTreeMap, BTreeSet}; use std::sync::Arc; +use std::sync::atomic::AtomicU64; use std::time::{Duration, Instant}; use futures::future::{BoxFuture, FutureExt}; @@ -551,6 +552,7 @@ impl Catalog { plans: CatalogPlans::default(), expr_cache_handle, transient_revision: 1, + latest_transient_revision: Arc::new(AtomicU64::new(1)), storage: Arc::new(tokio::sync::Mutex::new(storage)), }; diff --git a/src/adapter/src/catalog/transact.rs b/src/adapter/src/catalog/transact.rs index e6681f5fbd3b3..b4746ee0de6df 100644 --- a/src/adapter/src/catalog/transact.rs +++ b/src/adapter/src/catalog/transact.rs @@ -12,6 +12,7 @@ use std::borrow::Cow; use std::collections::{BTreeMap, BTreeSet}; use std::sync::Arc; +use std::sync::atomic::Ordering; use std::time::Duration; use itertools::Itertools; @@ -454,6 +455,7 @@ impl Catalog { drop(storage); if let Some(new_state) = new_state { self.transient_revision += 1; + self.latest_transient_revision.fetch_add(1, Ordering::SeqCst); ////////// todo: Is the ordering ok? self.state = new_state; } diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index c326787e06374..4a9c42c721d51 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -282,6 +282,7 @@ impl Client { segment_client: self.segment_client.clone(), peek_client, enable_frontend_peek_sequencing: false, // initialized below, once we have a ConnCatalog + catalog: Arc::clone(&catalog), }; let session = client.session(); @@ -552,6 +553,7 @@ pub struct SessionClient { // check the actual feature flag value at every peek (without a Coordinator call) once we'll // always have a catalog snapshot at hand. pub enable_frontend_peek_sequencing: bool, + catalog: Arc, } impl SessionClient { @@ -781,24 +783,30 @@ impl SessionClient { /// Fetches the catalog. #[instrument(level = "debug")] - pub async fn catalog_snapshot(&self, context: &str) -> Arc { - let start = std::time::Instant::now(); - let CatalogSnapshot { catalog } = self - .send_without_session(|tx| Command::CatalogSnapshot { tx }) - .await; - self.inner() - .metrics() - .catalog_snapshot_seconds - .with_label_values(&[context]) - .observe(start.elapsed().as_secs_f64()); - catalog + pub async fn catalog_snapshot(&mut self, context: &str) -> Arc { + if self.catalog.transient_revision() == self.catalog.latest_transient_revision() { + Arc::clone(&self.catalog) + } else { + let start = std::time::Instant::now(); + let CatalogSnapshot { catalog } = self + .send_without_session(|tx| Command::CatalogSnapshot { tx }) + .await; + self.inner() + .metrics() + .catalog_snapshot_seconds + .with_label_values(&[context]) + .observe(start.elapsed().as_secs_f64()); + self.catalog = Arc::clone(&catalog); + assert_eq!(self.catalog.transient_revision(), self.catalog.latest_transient_revision()); + catalog + } } /// Dumps the catalog to a JSON string. /// /// No authorization is performed, so access to this function must be limited to internal /// servers or superusers. - pub async fn dump_catalog(&self) -> Result { + pub async fn dump_catalog(&mut self) -> Result { let catalog = self.catalog_snapshot("dump_catalog").await; catalog.dump().map_err(AdapterError::from) } @@ -808,7 +816,7 @@ impl SessionClient { /// /// No authorization is performed, so access to this function must be limited to internal /// servers or superusers. - pub async fn check_catalog(&self) -> Result<(), serde_json::Value> { + pub async fn check_catalog(&mut self) -> Result<(), serde_json::Value> { let catalog = self.catalog_snapshot("check_catalog").await; catalog.check_consistency() } diff --git a/src/environmentd/src/http/catalog.rs b/src/environmentd/src/http/catalog.rs index a7c3484d9d4e9..87c8471e5d8b8 100644 --- a/src/environmentd/src/http/catalog.rs +++ b/src/environmentd/src/http/catalog.rs @@ -16,14 +16,14 @@ use http::StatusCode; use crate::http::AuthedClient; -pub async fn handle_catalog_dump(client: AuthedClient) -> impl IntoResponse { +pub async fn handle_catalog_dump(mut client: AuthedClient) -> impl IntoResponse { match client.client.dump_catalog().await.map(|c| c.into_string()) { Ok(res) => Ok((TypedHeader(ContentType::json()), res)), Err(e) => Err((StatusCode::INTERNAL_SERVER_ERROR, e.to_string())), } } -pub async fn handle_catalog_check(client: AuthedClient) -> impl IntoResponse { +pub async fn handle_catalog_check(mut client: AuthedClient) -> impl IntoResponse { let response = match client.client.check_catalog().await { Ok(_) => serde_json::Value::String("".to_string()), Err(inconsistencies) => serde_json::json!({ "err": inconsistencies }), From 669904290bcbaaf6d8f485b59e418450808cc931 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sat, 25 Oct 2025 20:48:16 +0200 Subject: [PATCH 50/55] Plan caching (without any invalidation) --- src/adapter/src/frontend_peek.rs | 77 ++++++++++++++++++-------------- src/adapter/src/optimize/peek.rs | 2 +- src/adapter/src/peek_client.rs | 7 ++- src/expr/src/relation.rs | 2 +- src/sql/src/plan.rs | 6 +-- 5 files changed, 54 insertions(+), 40 deletions(-) diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 8af81bc680c0b..75e05c55ab46c 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -8,7 +8,6 @@ // by the Apache License, Version 2.0. use std::sync::Arc; - use mz_adapter_types::dyncfgs::CONSTRAINT_BASED_TIMESTAMP_SELECTION; use mz_adapter_types::timestamp_selection::ConstraintBasedTimestampSelection; use mz_compute_types::ComputeInstanceId; @@ -371,41 +370,53 @@ impl PeekClient { let span = Span::current(); - let (global_lir_plan, _optimization_finished_at) = match mz_ore::task::spawn_blocking( - || "optimize peek", - move || { - span.in_scope(|| { - let raw_expr = select_plan.source.clone(); - - // HIR ⇒ MIR lowering and MIR optimization (local) - let local_mir_plan = optimizer.catch_unwind_optimize(raw_expr)?; - // Attach resolved context required to continue the pipeline. - let local_mir_plan = - local_mir_plan.resolve(timestamp_context.clone(), &session_meta, stats); - // MIR optimization (global), MIR ⇒ LIR lowering, and LIR optimization (global) - let global_lir_plan = optimizer.catch_unwind_optimize(local_mir_plan)?; - - let optimization_finished_at = now(); - - // TODO(peek-seq): plan_insights stuff + let select_plan_cloned = select_plan.clone(); - Ok::<_, AdapterError>((global_lir_plan, optimization_finished_at)) - }) - }, - ) - .await - { - Ok(Ok(r)) => r, - Ok(Err(adapter_error)) => { - return Err(adapter_error); - } - Err(_join_error) => { - // Should only happen if the runtime is shutting down, because we - // - never call `abort`; - // - catch panics with `catch_unwind_optimize`. - return Err(AdapterError::Unstructured(anyhow::anyhow!( + let (global_lir_plan, _optimization_finished_at) = if self.plan_cache.contains_key(&select_plan) { + ( + self.plan_cache.get(&select_plan).unwrap().clone(), + now(), + ) + } else { + match mz_ore::task::spawn_blocking( + || "optimize peek", + move || { + span.in_scope(|| { + let raw_expr = select_plan.source.clone(); + + // HIR ⇒ MIR lowering and MIR optimization (local) + let local_mir_plan = optimizer.catch_unwind_optimize(raw_expr)?; + // Attach resolved context required to continue the pipeline. + let local_mir_plan = + local_mir_plan.resolve(timestamp_context.clone(), &session_meta, stats); + // MIR optimization (global), MIR ⇒ LIR lowering, and LIR optimization (global) + let global_lir_plan = optimizer.catch_unwind_optimize(local_mir_plan)?; + + let optimization_finished_at = now(); + + // TODO(peek-seq): plan_insights stuff + + Ok::<_, AdapterError>((global_lir_plan, optimization_finished_at)) + }) + }, + ) + .await + { + Ok(Ok(r)) => { + self.plan_cache.insert(select_plan_cloned.clone(), r.0.clone()); + r + }, + Ok(Err(adapter_error)) => { + return Err(adapter_error); + } + Err(_join_error) => { + // Should only happen if the runtime is shutting down, because we + // - never call `abort`; + // - catch panics with `catch_unwind_optimize`. + return Err(AdapterError::Unstructured(anyhow::anyhow!( "peek optimization aborted, because the system is shutting down" ))); + } } }; diff --git a/src/adapter/src/optimize/peek.rs b/src/adapter/src/optimize/peek.rs index 63456042b5296..40d8190021231 100644 --- a/src/adapter/src/optimize/peek.rs +++ b/src/adapter/src/optimize/peek.rs @@ -157,7 +157,7 @@ pub struct Resolved<'s> { /// 4. optimizing the resulting `DataflowDescription` with `MIR` plans. /// 5. MIR ⇒ LIR lowering, and /// 6. optimizing the resulting `DataflowDescription` with `LIR` plans. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct GlobalLirPlan { peek_plan: PeekPlan, df_meta: DataflowMetainfo, diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index b5a3319e90a42..59556911f4938 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -7,7 +7,7 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use differential_dataflow::consolidation::consolidate; @@ -27,11 +27,12 @@ use prometheus::Histogram; use timely::progress::Antichain; use tokio::sync::oneshot; use uuid::Uuid; - +use mz_sql::plan::SelectPlan; use crate::command::Command; use crate::coord::Coordinator; use crate::coord::peek::FastPathPlan; use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds, statement_logging}; +use crate::optimize::peek::GlobalLirPlan; /// Storage collections trait alias we need to consult for since/frontiers. pub type StorageCollectionsHandle = Arc< @@ -58,6 +59,7 @@ pub struct PeekClient { /// Per-timeline oracles from the coordinator. Lazily populated. oracles: BTreeMap + Send + Sync>>, persist_client: PersistClient, + pub plan_cache: HashMap, } impl PeekClient { @@ -77,6 +79,7 @@ impl PeekClient { optimizer_metrics, oracles: Default::default(), // lazily populated persist_client, + plan_cache: Default::default(), } } diff --git a/src/expr/src/relation.rs b/src/expr/src/relation.rs index 5a4d6214802ad..c4abe2add1ab0 100644 --- a/src/expr/src/relation.rs +++ b/src/expr/src/relation.rs @@ -3392,7 +3392,7 @@ impl JoinInputCharacteristicsV1 { /// The generic parameters are for accommodating prepared statement parameters in /// `limit` and `offset`: the planner can hold these fields as HirScalarExpr long enough to call /// `bind_parameters` on them. -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Hash)] pub struct RowSetFinishing, O = usize> { /// Order rows by the given columns. pub order_by: Vec, diff --git a/src/sql/src/plan.rs b/src/sql/src/plan.rs index e87385d68e642..de0a0507ee9cc 100644 --- a/src/sql/src/plan.rs +++ b/src/sql/src/plan.rs @@ -843,7 +843,7 @@ pub struct SetTransactionPlan { } /// A plan for select statements. -#[derive(Clone, Debug)] +#[derive(Clone, Debug, Hash, PartialEq, Eq)] pub struct SelectPlan { /// The `SELECT` statement itself. Used for explain/notices, but not otherwise /// load-bearing. Boxed to save stack space. @@ -1837,7 +1837,7 @@ pub struct Type { } /// Specifies when a `Peek` or `Subscribe` should occur. -#[derive(Deserialize, Clone, Debug, PartialEq)] +#[derive(Deserialize, Clone, Debug, PartialEq, Hash, Eq)] pub enum QueryWhen { /// The peek should occur at the latest possible timestamp that allows the /// peek to complete immediately. @@ -1925,7 +1925,7 @@ pub enum MutationKind { Delete, } -#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Ord)] +#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] pub enum CopyFormat { Text, Csv, From 5492a2a79cce573586ae48e185b4b8da229e0fe5 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 26 Oct 2025 16:20:06 +0100 Subject: [PATCH 51/55] Omit Command::Commit for single-peek implicit transactions --- src/pgwire/src/protocol.rs | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/src/pgwire/src/protocol.rs b/src/pgwire/src/protocol.rs index c396cd614cecf..94d46a9318488 100644 --- a/src/pgwire/src/protocol.rs +++ b/src/pgwire/src/protocol.rs @@ -20,10 +20,7 @@ use byteorder::{ByteOrder, NetworkEndian}; use futures::future::{BoxFuture, FutureExt, pending}; use itertools::Itertools; use mz_adapter::client::RecordFirstRowStream; -use mz_adapter::session::{ - EndTransactionAction, InProgressRows, LifecycleTimestamps, PortalRefMut, PortalState, - SessionConfig, TransactionStatus, -}; +use mz_adapter::session::{EndTransactionAction, InProgressRows, LifecycleTimestamps, PortalRefMut, PortalState, SessionConfig, Transaction, TransactionOps, TransactionStatus}; use mz_adapter::statement_logging::{StatementEndedExecutionReason, StatementExecutionStrategy}; use mz_adapter::{ AdapterError, AdapterNotice, ExecuteContextExtra, ExecuteResponse, PeekResponseUnary, metrics, @@ -1083,9 +1080,30 @@ where // Implicit transactions are closed at the end of a Query message. { - if self.adapter_client.session().transaction().is_implicit() { + + //println!("###### self.adapter_client.session().transaction():\n{:?}\n", self.adapter_client.session().transaction()); + + let single_query_peek = { + match self.adapter_client.session().transaction() { + TransactionStatus::Started(Transaction { + ops: TransactionOps::Peeks { + .. + }, + .. + }) => { + true + }, + _ => false, + } + }; + + if self.adapter_client.session().transaction().is_implicit() && !single_query_peek { self.commit_transaction().await?; } + if single_query_peek { + self.txn_needs_commit = false; + self.adapter_client.session().clear_transaction(); + } } if num_stmts == 0 { From 592c2bef6a2ba83c6d1bae5699f7ebab8c34f3fb Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 26 Oct 2025 18:08:23 +0100 Subject: [PATCH 52/55] Omit PeekNotification for frontend peeks --- src/adapter/src/peek_client.rs | 1 + src/compute-client/src/controller.rs | 3 ++- src/compute-client/src/controller/instance.rs | 23 ++++++++++++------- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index 59556911f4938..6c095190e6956 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -333,6 +333,7 @@ impl PeekClient { target_read_hold, target_replica, rows_tx, + true, ) .await .map_err(AdapterError::concurrent_dependency_drop_from_peek_error)?; diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 663181d07c007..954cca43c7ae8 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -675,7 +675,7 @@ where } tokio::select! { - resp = self.response_rx.recv() => { + resp = self.response_rx.recv() => { //////////// we get e.g. PeekNotification here let resp = resp.expect("`self.response_tx` not dropped"); self.stashed_response = Some(resp); } @@ -910,6 +910,7 @@ where read_hold, target_replica, peek_response_tx, + false, ) .expect("validated") }); diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index b7242e1fdec98..d736a3a3fc352 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -275,6 +275,7 @@ where target_read_hold: ReadHold, target_replica: Option, peek_response_tx: oneshot::Sender, + is_frontend_peek: bool, ) -> Result<(), crate::controller::error::PeekError> { self.call_sync(move |i| { i.peek( @@ -288,6 +289,7 @@ where target_read_hold, target_replica, peek_response_tx, + is_frontend_peek, ) }) .await @@ -1737,6 +1739,7 @@ where mut read_hold: ReadHold, target_replica: Option, peek_response_tx: oneshot::Sender, + is_frontend_peek: bool, ) -> Result<(), PeekError> { use PeekError::*; @@ -1769,6 +1772,7 @@ where peek_response_tx, limit: finishing.limit.map(usize::cast_from), offset: finishing.offset, + is_frontend_peek, }, ); @@ -2088,14 +2092,16 @@ where let duration = peek.requested_at.elapsed(); self.metrics.observe_peek_response(&response, duration); - let notification = PeekNotification::new(&response, peek.offset, peek.limit); - // NOTE: We use the `otel_ctx` from the response, not the pending peek, because we - // currently want the parent to be whatever the compute worker did with this peek. - self.deliver_response(ComputeControllerResponse::PeekNotification( - uuid, - notification, - otel_ctx, - )); + if !peek.is_frontend_peek { + let notification = PeekNotification::new(&response, peek.offset, peek.limit); + // NOTE: We use the `otel_ctx` from the response, not the pending peek, because we + // currently want the parent to be whatever the compute worker did with this peek. + self.deliver_response(ComputeControllerResponse::PeekNotification( + uuid, + notification, + otel_ctx, + )); + } self.finish_peek(uuid, response) } @@ -2993,6 +2999,7 @@ struct PendingPeek { limit: Option, /// The offset into the peek's result. offset: usize, + is_frontend_peek: bool, } #[derive(Debug, Clone)] From 044d03784b94bee7eb00088d0bc967c8ead94e5a Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 26 Oct 2025 18:59:08 +0100 Subject: [PATCH 53/55] Comment out irrelevant benchmarks --- .../materialize/scalability/schema/schema.py | 6 +- .../workload/workloads/dml_dql_workloads.py | 62 +++++++++---------- 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/misc/python/materialize/scalability/schema/schema.py b/misc/python/materialize/scalability/schema/schema.py index 96769b0e7cf50..50462278f4835 100644 --- a/misc/python/materialize/scalability/schema/schema.py +++ b/misc/python/materialize/scalability/schema/schema.py @@ -51,7 +51,7 @@ def init_sqls(self) -> list[str]: [ f"CREATE TABLE t{t} (f1 INTEGER DEFAULT 1);", f"INSERT INTO t{t} DEFAULT VALUES;", - f"CREATE OR REPLACE MATERIALIZED VIEW mv{t} AS SELECT count(*) AS count FROM t{t};", + # f"CREATE OR REPLACE MATERIALIZED VIEW mv{t} AS SELECT count(*) AS count FROM t{t};", ] ) @@ -62,9 +62,9 @@ def init_sqls(self) -> list[str]: # index to be ready. if self.create_index: init_sqls.append(f"CREATE INDEX i{t} ON t{t} (f1);") - init_sqls.append(f"CREATE INDEX mv_i{t} ON mv{t} (count);") + # init_sqls.append(f"CREATE INDEX mv_i{t} ON mv{t} (count);") init_sqls.append(f"SELECT f1 from t{t};") - init_sqls.append(f"SELECT count from mv{t};") + # init_sqls.append(f"SELECT count from mv{t};") return init_sqls diff --git a/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py b/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py index 95dfaef80c8ed..02a4b769704ec 100644 --- a/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py +++ b/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py @@ -7,27 +7,27 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. from materialize.scalability.operation.operations.operations import ( - InsertDefaultValues, - SelectCount, - SelectCountInMv, - SelectLimit, - SelectOne, + #InsertDefaultValues, + #SelectCount, + #SelectCountInMv, + #SelectLimit, + #SelectOne, SelectStar, - SelectUnionAll, - Update, + #SelectUnionAll, + #Update, ) from materialize.scalability.operation.scalability_operation import Operation from materialize.scalability.workload.workload_markers import DmlDqlWorkload -class InsertWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [InsertDefaultValues()] +# class InsertWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [InsertDefaultValues()] -class SelectOneWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [SelectOne()] +# class SelectOneWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [SelectOne()] class SelectStarWorkload(DmlDqlWorkload): @@ -35,31 +35,31 @@ def operations(self) -> list["Operation"]: return [SelectStar()] -class SelectLimitWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [SelectLimit()] +# class SelectLimitWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [SelectLimit()] -class SelectCountWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [SelectCount()] +# class SelectCountWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [SelectCount()] -class SelectUnionAllWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [SelectUnionAll()] +# class SelectUnionAllWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [SelectUnionAll()] -class InsertAndSelectCountInMvWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [InsertDefaultValues(), SelectCountInMv()] +# class InsertAndSelectCountInMvWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [InsertDefaultValues(), SelectCountInMv()] -class InsertAndSelectLimitWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [InsertDefaultValues(), SelectLimit()] +# class InsertAndSelectLimitWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [InsertDefaultValues(), SelectLimit()] -class UpdateWorkload(DmlDqlWorkload): - def operations(self) -> list["Operation"]: - return [Update()] +# class UpdateWorkload(DmlDqlWorkload): +# def operations(self) -> list["Operation"]: +# return [Update()] From e0d9fb39222dceaa35fb8ec4a021d506b534fe3d Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 26 Oct 2025 19:13:13 +0100 Subject: [PATCH 54/55] Uncomment SelectLimitWorkload --- .../scalability/workload/workloads/dml_dql_workloads.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py b/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py index 02a4b769704ec..f9c9bb5a8aab5 100644 --- a/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py +++ b/misc/python/materialize/scalability/workload/workloads/dml_dql_workloads.py @@ -10,7 +10,7 @@ #InsertDefaultValues, #SelectCount, #SelectCountInMv, - #SelectLimit, + SelectLimit, #SelectOne, SelectStar, #SelectUnionAll, @@ -35,9 +35,9 @@ def operations(self) -> list["Operation"]: return [SelectStar()] -# class SelectLimitWorkload(DmlDqlWorkload): -# def operations(self) -> list["Operation"]: -# return [SelectLimit()] +class SelectLimitWorkload(DmlDqlWorkload): + def operations(self) -> list["Operation"]: + return [SelectLimit()] # class SelectCountWorkload(DmlDqlWorkload): From f8cc3fb6427fb08c77bddc894c60db28b0bac9f1 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 26 Oct 2025 19:18:34 +0100 Subject: [PATCH 55/55] Turn on the flag --- misc/python/materialize/mzcompose/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/misc/python/materialize/mzcompose/__init__.py b/misc/python/materialize/mzcompose/__init__.py index b1320da1cd25f..d21bd12dd7bfc 100644 --- a/misc/python/materialize/mzcompose/__init__.py +++ b/misc/python/materialize/mzcompose/__init__.py @@ -191,7 +191,7 @@ def get_variable_system_parameters( ), VariableSystemParameter( "enable_frontend_peek_sequencing", - "false", + "true", ["true", "false"], ), VariableSystemParameter(