diff --git a/doc/developer/design/20260210_incremental_occ_read_then_write.md b/doc/developer/design/20260210_incremental_occ_read_then_write.md index db6286fb2f210..4534b7fbd3871 100644 --- a/doc/developer/design/20260210_incremental_occ_read_then_write.md +++ b/doc/developer/design/20260210_incremental_occ_read_then_write.md @@ -128,7 +128,7 @@ Session Task Coordinator | | |-- acquire OCC semaphore | | | - |-- CreateReadThenWriteSubscribe ----> | + |-- CreateInternalSubscribe ---------> | | <------------ subscribe channel -----| | | | +-- OCC Loop ------------------+ | @@ -141,7 +141,7 @@ Session Task Coordinator | | if Success: break | | | +------------------------------+ | | | - |-- DropReadThenWriteSubscribe ------> | + |-- DropInternalSubscribe -----------> | | | ``` @@ -193,7 +193,7 @@ subscribe. The subscribes created for read-then-write are internal: they do not appear in `mz_subscriptions` or other introspection tables, and they don't increment the active subscribes metric. They are created and dropped via dedicated `Command` -variants (`CreateReadThenWriteSubscribe`, `DropReadThenWriteSubscribe`). +variants (`CreateInternalSubscribe`, `DropInternalSubscribe`). ## Correctness diff --git a/misc/python/materialize/mzcompose/__init__.py b/misc/python/materialize/mzcompose/__init__.py index 112e0ce36c58d..619f5e2f4dec4 100644 --- a/misc/python/materialize/mzcompose/__init__.py +++ b/misc/python/materialize/mzcompose/__init__.py @@ -209,6 +209,11 @@ def get_variable_system_parameters( "true", ["true", "false"], ), + VariableSystemParameter( + "enable_adapter_frontend_occ_read_then_write", + "true", + ["true", "false"], + ), VariableSystemParameter( "enable_cast_elimination", "true", diff --git a/misc/python/materialize/parallel_workload/action.py b/misc/python/materialize/parallel_workload/action.py index 556e959f90590..4b677495f97b8 100644 --- a/misc/python/materialize/parallel_workload/action.py +++ b/misc/python/materialize/parallel_workload/action.py @@ -352,8 +352,8 @@ def errors_to_ignore(self, exe: Executor) -> list[str]: result.extend( [ "does not exist", - "subscribe has been terminated because underlying relation", - "subscribe has been terminated because underlying cluster", + "query could not complete because relation", + "query could not complete because cluster", ] ) return result @@ -1585,7 +1585,6 @@ def __init__( self.flags_with_values["enable_cast_elimination"] = BOOLEAN_FLAG_VALUES self.flags_with_values["enable_upsert_v2"] = BOOLEAN_FLAG_VALUES self.flags_with_values["enable_coalesce_case_transform"] = BOOLEAN_FLAG_VALUES - # If you are adding a new config flag in Materialize, consider using it # here instead of just marking it as uninteresting to silence the # linter. parallel-workload randomly flips the flags in @@ -1593,6 +1592,11 @@ def __init__( # behavior, you should add it. Feature flags which turn on/off # externally visible features should not be flipped. self.uninteresting_flags: list[str] = [ + # Read once at environmentd startup; runtime ALTER SYSTEM SET is + # rejected (see sequence_alter_system_set). Flipping it here would + # be a no-op at best and confusing if any future code path forgot + # to consult the cached value. + "enable_adapter_frontend_occ_read_then_write", "enable_compute_half_join2", "enable_mz_join_core", "enable_compute_correction_v2", diff --git a/misc/python/materialize/parallel_workload/parallel_workload.py b/misc/python/materialize/parallel_workload/parallel_workload.py index 081a69208b962..dee67e59cec0c 100644 --- a/misc/python/materialize/parallel_workload/parallel_workload.py +++ b/misc/python/materialize/parallel_workload/parallel_workload.py @@ -470,7 +470,10 @@ def print_stats( def parse_common_args(parser: argparse.ArgumentParser) -> None: - parser.add_argument("--seed", type=str, default=str(int(time.time()))) + # TEMP: pinned to the seed from nightly 16238 to deterministically + # exercise the parallel-workload-dml retraction failure on this + # branch. Restore to `str(int(time.time()))` before merging. + parser.add_argument("--seed", type=str, default="1777388214") parser.add_argument("--runtime", default=600, type=int, help="Runtime in seconds") parser.add_argument( "--complexity", diff --git a/misc/python/materialize/parallel_workload/worker.py b/misc/python/materialize/parallel_workload/worker.py index 15cc41b7f736c..96b8acbb84760 100644 --- a/misc/python/materialize/parallel_workload/worker.py +++ b/misc/python/materialize/parallel_workload/worker.py @@ -91,6 +91,10 @@ def run( try: self.exe.rollback() except QueryError as e: + # ROLLBACK can itself be cancelled by + # `pg_cancel_backend`, leaving psycopg in + # `InFailedSqlTransaction`. Force a reconnect rather + # than retry the rollback. if ( "Please disconnect and re-connect" in e.msg or "server closed the connection unexpectedly" in e.msg @@ -98,6 +102,8 @@ def run( or "Connection refused" in e.msg or "the connection is lost" in e.msg or "connection in transaction status INERROR" in e.msg + or "canceling statement due to user request" in e.msg + or "current transaction is aborted" in e.msg ): self.exe.reconnect_next = True self.exe.rollback_next = False diff --git a/src/adapter-types/src/dyncfgs.rs b/src/adapter-types/src/dyncfgs.rs index 4ed2e5a0f4cf4..ab221472bd8a2 100644 --- a/src/adapter-types/src/dyncfgs.rs +++ b/src/adapter-types/src/dyncfgs.rs @@ -214,6 +214,13 @@ pub const CONSOLE_OIDC_SCOPES: Config<&'static str> = Config::new( "Space-separated OIDC scopes requested by the web console.", ); +pub const FRONTEND_READ_THEN_WRITE: Config = Config::new( + "enable_adapter_frontend_occ_read_then_write", + // WIP: true for testing in ci, Should be false before merging. + true, + "Use frontend sequencing (with optimistic concurrency control) for \ + DELETE, UPDATE, and INSERT operations.", +); /// Adds the full set of all adapter `Config`s. pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { configs @@ -245,4 +252,5 @@ pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { .add(&USER_ID_POOL_BATCH_SIZE) .add(&CONSOLE_OIDC_CLIENT_ID) .add(&CONSOLE_OIDC_SCOPES) + .add(&FRONTEND_READ_THEN_WRITE) } diff --git a/src/adapter/src/active_compute_sink.rs b/src/adapter/src/active_compute_sink.rs index f7c0186b88b51..b682dfeffc5f3 100644 --- a/src/adapter/src/active_compute_sink.rs +++ b/src/adapter/src/active_compute_sink.rs @@ -29,7 +29,7 @@ use timely::progress::Antichain; use tokio::sync::{mpsc, oneshot}; use uuid::Uuid; -use crate::coord::peek::PeekResponseUnary; +use crate::coord::peek::{DroppedDependency, PeekResponseUnary}; use crate::{AdapterError, ExecuteContext, ExecuteResponse}; #[derive(Debug)] @@ -88,7 +88,7 @@ pub enum ActiveComputeSinkRetireReason { Canceled, /// The compute sink was forcibly terminated because an object it depended on /// was dropped. - DependencyDropped(String), + DependencyDropped(DroppedDependency), } /// A description of an active subscribe from coord's perspective @@ -116,6 +116,9 @@ pub struct ActiveSubscribe { pub start_time: EpochMillis, /// How to present the subscribe's output. pub output: SubscribeOutput, + /// If true, this is an internal subscribe that should not appear in + /// introspection tables like mz_subscriptions. + pub internal: bool, } impl ActiveSubscribe { @@ -386,9 +389,9 @@ impl ActiveSubscribe { let message = match reason { ActiveComputeSinkRetireReason::Finished => return, ActiveComputeSinkRetireReason::Canceled => PeekResponseUnary::Canceled, - ActiveComputeSinkRetireReason::DependencyDropped(d) => PeekResponseUnary::Error( - format!("subscribe has been terminated because underlying {d} was dropped"), - ), + ActiveComputeSinkRetireReason::DependencyDropped(d) => { + PeekResponseUnary::DependencyDropped(d) + } }; self.send(message); } @@ -440,9 +443,9 @@ impl ActiveCopyTo { let message = match reason { ActiveComputeSinkRetireReason::Finished => return, ActiveComputeSinkRetireReason::Canceled => Err(AdapterError::Canceled), - ActiveComputeSinkRetireReason::DependencyDropped(d) => Err(AdapterError::Unstructured( - anyhow!("copy has been terminated because underlying {d} was dropped"), - )), + ActiveComputeSinkRetireReason::DependencyDropped(dep) => Err( + AdapterError::Unstructured(anyhow!(dep.copy_terminated_error())), + ), }; let _ = self.tx.send(message); } diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index 232542fe3a589..5a00f3d6acf3c 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -18,13 +18,14 @@ use std::time::{Duration, Instant}; use anyhow::bail; use chrono::{DateTime, Utc}; use derivative::Derivative; -use futures::{Stream, StreamExt}; +use futures::{FutureExt, Stream, StreamExt}; use itertools::Itertools; use mz_adapter_types::connection::{ConnectionId, ConnectionIdType}; use mz_auth::password::Password; use mz_auth::{Authenticated, AuthenticatorKind}; use mz_build_info::BuildInfo; use mz_compute_types::ComputeInstanceId; +use mz_expr::UnmaterializableFunc; use mz_ore::channel::OneshotReceiverExt; use mz_ore::collections::CollectionExt; use mz_ore::id_gen::{IdAllocator, IdAllocatorInnerBitSet, MAX_ORG_ID, org_id_conn_bits}; @@ -58,6 +59,8 @@ use crate::command::{ use crate::coord::{Coordinator, ExecuteContextGuard}; use crate::error::AdapterError; use crate::metrics::Metrics; +use crate::optimize::dataflows::{EvalTime, ExprPrepOneShot}; +use crate::optimize::{self, Optimize, OptimizerError}; use crate::session::{ EndTransactionAction, PreparedStatement, Session, SessionConfig, StateRevision, TransactionId, }; @@ -283,6 +286,9 @@ impl Client { persist_client, statement_logging_frontend, superuser_attribute, + occ_write_semaphore, + frontend_read_then_write_enabled, + read_only, } = response; let peek_client = PeekClient::new( @@ -292,6 +298,9 @@ impl Client { optimizer_metrics, persist_client, statement_logging_frontend, + occ_write_semaphore, + frontend_read_then_write_enabled, + read_only, ); let mut client = SessionClient { @@ -584,6 +593,23 @@ pub struct SessionClient { pub enable_frontend_peek_sequencing: bool, } +/// Keeps a connection cancel watch installed in the coordinator for the +/// duration of a frontend read-then-write attempt. +struct FrontendConnectionCancelWatchGuard { + conn_id: ConnectionId, + client: Option, +} + +impl Drop for FrontendConnectionCancelWatchGuard { + fn drop(&mut self) { + if let Some(client) = self.client.take() { + client.send(Command::UnregisterConnectionCancelWatch { + conn_id: self.conn_id.clone(), + }); + } + } +} + impl SessionClient { /// Parses a SQL expression, reporting failures as a telemetry event if /// possible. @@ -733,6 +759,7 @@ impl SessionClient { outer_ctx_extra: Option, ) -> Result<(ExecuteResponse, Instant), AdapterError> { let execute_started = Instant::now(); + let cancel_future = cancel_future.map(|_| ()).shared(); // Attempt peek sequencing in the session task. // If unsupported, fall back to the Coordinator path. @@ -747,11 +774,24 @@ impl SessionClient { // No additional work needed here. return Ok((resp, execute_started)); } else { - debug!("frontend peek did not happen, falling back to `Command::Execute`"); + debug!("frontend peek did not happen, trying frontend read-then-write"); + } + + // Attempt read-then-write sequencing in the session task. + let rtw_result = self + .try_frontend_read_then_write_with_cancel( + &portal_name, + &mut outer_ctx_extra, + cancel_future.clone(), + ) + .await?; + if let Some(resp) = rtw_result { + debug!("frontend read-then-write succeeded"); + return Ok((resp, execute_started)); + } else { + debug!("frontend read-then-write did not happen, falling back to `Command::Execute`"); // If we bailed out, outer_ctx_extra is still present (if it was originally). // `Command::Execute` will handle it. - // (This is not true if we bailed out _after_ the frontend peek sequencing has already - // begun its own statement logging. That case would be a bug.) } let response = self @@ -762,7 +802,7 @@ impl SessionClient { tx, outer_ctx_extra, }, - cancel_future, + cancel_future.clone(), ) .await?; Ok((response, execute_started)) @@ -1017,7 +1057,7 @@ impl SessionClient { async fn send_with_cancel( &mut self, f: F, - cancel_future: impl Future + Send, + cancel_future: impl Future + Send, ) -> Result where F: FnOnce(oneshot::Sender>, Session) -> Command, @@ -1088,7 +1128,12 @@ impl SessionClient { | Command::UnregisterFrontendPeek { .. } | Command::ExplainTimestamp { .. } | Command::FrontendStatementLogging(..) - | Command::InjectAuditEvents { .. } => {} + | Command::InjectAuditEvents { .. } + | Command::RegisterConnectionCancelWatch { .. } + | Command::UnregisterConnectionCancelWatch { .. } + | Command::CreateInternalSubscribe { .. } + | Command::AttemptWrite { .. } + | Command::DropInternalSubscribe { .. } => {} }; cmd }); @@ -1119,7 +1164,7 @@ impl SessionClient { *client_session = Some(res.session); return res.result; }, - _err = &mut cancel_future, if !cancelled => { + _ = &mut cancel_future, if !cancelled => { cancelled = true; inner_client.send(Command::PrivilegedCancelRequest { conn_id: conn_id.clone(), @@ -1182,6 +1227,382 @@ impl SessionClient { Ok(None) } } + + /// Runs frontend read-then-write while reacting to both local/session + /// cancellation and coordinator-issued connection cancellation. + async fn try_frontend_read_then_write_with_cancel( + &mut self, + portal_name: &str, + outer_ctx_extra: &mut Option, + cancel_future: impl Future + Send, + ) -> Result, AdapterError> { + let conn_id = self.session().conn_id().clone(); + + let inner_client = self.inner().clone(); + let mut cancel_future = pin::pin!(cancel_future); + + // Cancellation can arrive via two independent paths: + // + // 1) `cancel_future`: local/session-side cancellation (e.g. client + // connection closes). For this path we must still forward a privileged + // cancel to the coordinator so in-flight work owned there is canceled. + // + // 2) `connection_cancel`: coordinator-side cancellation (e.g. + // `pg_cancel_backend`) reflected through the connection cancel watch. + // This can trigger while frontend RTW is still planning or optimizing, + // before coordinator-owned subscribe/write steps are installed. + // + // We select on both so frontend RTW exits promptly regardless of where + // cancellation originated. + + let mut connection_cancel_rx = self + .peek_client + .call_coordinator(|tx| Command::RegisterConnectionCancelWatch { + conn_id: conn_id.clone(), + tx, + }) + .await; + let _connection_cancel_guard = FrontendConnectionCancelWatchGuard { + conn_id: conn_id.clone(), + client: Some(inner_client.clone()), + }; + if *connection_cancel_rx.borrow() { + return Err(AdapterError::Canceled); + } + let connection_cancel = async move { + let was_cancelled = connection_cancel_rx.wait_for(|v| *v).await.is_ok(); + if !was_cancelled { + // The watch sender was dropped without signaling + // cancellation. This can happen due to a race between a + // fire-and-forget UnregisterConnectionCancelWatch from a + // previous operation and our RegisterConnectionCancelWatch. + // This is not a real cancellation, so wait forever (letting + // the other select! branches handle completion/cancel). + futures::future::pending::<()>().await; + } + }; + tokio::pin!(connection_cancel); + + let mut frontend_read_then_write = + pin::pin!(self.try_frontend_read_then_write(portal_name, outer_ctx_extra)); + + tokio::select! { + response = &mut frontend_read_then_write => response, + _ = &mut cancel_future => { + // This originates from a pgwire session disconnect. Let the + // Coordinator know so it can do cleanups. + inner_client.send(Command::PrivilegedCancelRequest { + conn_id: conn_id.clone(), + }); + + Err(AdapterError::Canceled) + } + _ = &mut connection_cancel => { + Err(AdapterError::Canceled) + } + } + } + + /// Attempt to sequence a read-then-write (DELETE/UPDATE/INSERT INTO .. + /// SELECT .. FROM) from the session task. + /// + /// Returns `Ok(Some(response))` if we handled the operation, or `Ok(None)` + /// to fall back to the Coordinator's sequencing. If it returns an error, it + /// should be returned to the user. + pub(crate) async fn try_frontend_read_then_write( + &mut self, + portal_name: &str, + outer_ctx_extra: &mut Option, + ) -> Result, AdapterError> { + use mz_expr::RowSetFinishing; + use mz_sql::plan::{MutationKind, Plan, ReadThenWritePlan}; + use mz_sql_parser::ast::Statement; + + // Check if frontend read-then-write is enabled (determined once at process startup + // to avoid a mixed-mode window where both the old lock-based path and the new OCC + // path are active concurrently). + if !self.peek_client.frontend_read_then_write_enabled { + return Ok(None); + } + + let catalog = self.catalog_snapshot("try_frontend_read_then_write").await; + + let (stmt, params, logging, lifecycle_timestamps) = { + let session = self.session.as_ref().expect("SessionClient invariant"); + let portal = match session.get_portal_unverified(portal_name) { + Some(portal) => portal, + None => return Ok(None), // Portal doesn't exist, fall back + }; + ( + portal.stmt.clone(), + portal.parameters.clone(), + Arc::clone(&portal.logging), + portal.lifecycle_timestamps.clone(), + ) + }; + + let stmt = match stmt { + Some(stmt) + if matches!( + &*stmt, + Statement::Delete(_) | Statement::Update(_) | Statement::Insert(_) + ) => + { + stmt + } + Some(_stmt) => { + return Ok(None); + } + None => { + return Ok(None); + } + }; + + // Reject mutations in read-only mode (e.g. during 0dt upgrades). Done + // early, before any planning or fast-path dispatch, so every sub-path + // (constant INSERT, OCC INSERT/UPDATE/DELETE) is covered uniformly. + if self.peek_client.read_only { + return Err(AdapterError::ReadOnly); + } + + let stmt_string = stmt.to_string(); + + let (plan, target_cluster, resolved_ids) = { + let session = self.session.as_mut().expect("SessionClient invariant"); + let conn_catalog = catalog.for_session(session); + 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 target_cluster = match session.transaction().cluster() { + Some(cluster_id) => crate::coord::TargetCluster::Transaction(cluster_id), + None => crate::coord::catalog_serving::auto_run_on_catalog_server( + &conn_catalog, + session, + &plan, + ), + }; + + (plan, target_cluster, resolved_ids) + }; + + // Cluster restrictions and RBAC, mirroring the coordinator's checks + // in sequencer.rs. Resolution may fail if the target cluster doesn't + // exist — that gets reported later (with the correct error) by + // `validate_read_then_write`; for the purposes of these checks we + // treat it as "no cluster known", consistent with the coordinator. + let (target_cluster_id, target_cluster_name) = { + let session = self.session.as_ref().expect("SessionClient invariant"); + match catalog.resolve_target_cluster(target_cluster.clone(), session) { + Ok(cluster) => (Some(cluster.id), Some(cluster.name.clone())), + Err(_) => (None, None), + } + }; + { + let session = self.session.as_ref().expect("SessionClient invariant"); + let conn_catalog = catalog.for_session(session); + if let Some(cluster_name) = &target_cluster_name { + crate::coord::catalog_serving::check_cluster_restrictions( + cluster_name, + &conn_catalog, + &plan, + )?; + } + if let Err(e) = mz_sql::rbac::check_plan( + &conn_catalog, + None:: Option>, + session, + &plan, + target_cluster_id, + &resolved_ids, + ) { + return Err(e.into()); + } + } + + // Wait for any in-flight startup builtin-table appends that this plan + // depends on. Mirrors the frontend_peek and coordinator sequencer + // paths; no-op for plans that don't depend on builtin tables. + { + let session = self.session.as_mut().expect("SessionClient invariant"); + if let Some((_, wait_future)) = + crate::coord::appends::waiting_on_startup_appends(&catalog, session, &plan) + { + wait_future.await; + } + } + + // Handle ReadThenWrite plans or Insert plans. + let rtw_plan = match plan { + Plan::ReadThenWrite(rtw_plan) => rtw_plan, + Plan::Insert(insert_plan) => { + // For INSERT, we need to check if it's a constant insert + // without RETURNING. Constant inserts use a fast path in the + // coordinator, so we fall back. + // + // We need to lower HIR to MIR to check for constants because + // VALUES statements are planned as Wrap calls at the HIR level. + // + // Only take the constant-INSERT fast path when the HIR + // names no persisted collections (no Get nodes on tables + // or MVs). The MIR optimizer can fold an MV reference + // into a literal when the MV's plan happens to be + // constant, but "plan is constant" is NOT the same as + // "content is visible at the current oracle_ts". A + // `REFRESH AT year 30000` MV has a constant plan but no + // durable content until the refresh fires; folding it + // and blind-writing the literal would skip timestamp + // selection and linearization, producing data that was + // never observable. + // + // Preserving HIR-level Get nodes routes the INSERT + // through the RTW path, where timestamp selection + // handles REFRESH and other time-dependent reads + // correctly. + let has_read_deps = { + use mz_expr::CollectionPlan; + !insert_plan.values.depends_on().is_empty() + }; + + if !has_read_deps { + let optimized_mir = if insert_plan.values.as_const().is_some() { + // Already constant at HIR level - just lower without optimization + let expr = insert_plan + .values + .clone() + .lower(catalog.system_config(), None)?; + mz_expr::OptimizedMirRelationExpr(expr) + } else { + // Need to optimize to check if it becomes constant. + // Use one-shot expression prep so unmaterializable + // functions like current_user() are resolved before we + // decide whether this can use the blind-write path. + let optimizer_config = + optimize::OptimizerConfig::from(catalog.system_config()); + let session = self.session.as_ref().expect("SessionClient invariant"); + let prep = ExprPrepOneShot { + logical_time: EvalTime::NotAvailable, + session, + catalog_state: catalog.state(), + }; + let mut optimizer = + optimize::view::Optimizer::new_with_prep(optimizer_config, None, prep); + match optimizer.optimize(insert_plan.values.clone()) { + Ok(expr) => expr, + Err(OptimizerError::UncallableFunction { + func: UnmaterializableFunc::MzNow, + .. + }) => { + // Preserve the established user-facing `mz_now()` + // error by falling back to the RTW validator. + let expr = insert_plan + .values + .clone() + .lower(catalog.system_config(), None)?; + mz_expr::OptimizedMirRelationExpr(expr) + } + Err(e) => return Err(e.into()), + } + }; + + // Constant INSERT without RETURNING are blind-writes. Add to + // the transaction and let those code paths handle it. + let inner_mir = optimized_mir.into_inner(); + if inner_mir.as_const().is_some() && insert_plan.returning.is_empty() { + let session = self.session.as_mut().expect("SessionClient invariant"); + + let logging_guard = self.peek_client.begin_statement_logging( + session, + ¶ms, + &logging, + &catalog, + lifecycle_timestamps, + outer_ctx_extra, + ); + if let (Some(logging_id), Some(cluster_id)) = + (logging_guard.id(), target_cluster_id) + { + self.peek_client.log_set_cluster(logging_id, cluster_id); + } + + let result = Coordinator::insert_constant( + &catalog, + session, + insert_plan.id, + inner_mir, + ); + + logging_guard.retire_with_result(&result); + + return Ok(Some(result?)); + } + } + + let desc_arity = match catalog.try_get_entry(&insert_plan.id) { + Some(table) => { + let desc = table + .relation_desc_latest() + .ok_or_else(|| AdapterError::Internal("table has no desc".into()))?; + desc.arity() + } + None => { + return Err(AdapterError::Catalog(mz_catalog::memory::error::Error { + kind: mz_catalog::memory::error::ErrorKind::Sql( + mz_sql::catalog::CatalogError::UnknownItem( + insert_plan.id.to_string(), + ), + ), + })); + } + }; + + let finishing = RowSetFinishing { + order_by: vec![], + limit: None, + offset: 0, + project: (0..desc_arity).collect(), + }; + + ReadThenWritePlan { + id: insert_plan.id, + selection: insert_plan.values, + finishing, + assignments: BTreeMap::new(), + kind: MutationKind::Insert, + returning: insert_plan.returning, + } + } + _ => { + return Err(AdapterError::Internal( + "unexpected plan type for mutation".into(), + )); + } + }; + + // The OCC path commits writes immediately and they cannot be rolled + // back, so reject explicit transaction blocks. (Constant INSERTs are + // handled above and don't go through OCC.) + { + let session = self.session.as_ref().expect("SessionClient invariant"); + if !session.transaction().is_implicit() { + return Err(AdapterError::OperationProhibitsTransaction(stmt_string)); + } + } + + let session = self.session.as_mut().expect("SessionClient invariant"); + self.peek_client + .frontend_read_then_write( + session, + rtw_plan, + target_cluster, + ¶ms, + &logging, + lifecycle_timestamps, + outer_ctx_extra, + ) + .await + .map(Some) + } } impl Drop for SessionClient { diff --git a/src/adapter/src/command.rs b/src/adapter/src/command.rs index 7c6d378f654c3..bcc416fef72cc 100644 --- a/src/adapter/src/command.rs +++ b/src/adapter/src/command.rs @@ -31,7 +31,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, GlobalId, RowIterator, SqlRelationType}; +use mz_repr::{CatalogItemId, ColumnIndex, Diff, GlobalId, Row, RowIterator, SqlRelationType}; use mz_sql::ast::{FetchDirection, Raw, Statement}; use mz_sql::catalog::ObjectType; use mz_sql::optimizer_metrics::OptimizerMetrics; @@ -42,16 +42,17 @@ 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 tokio::sync::{Semaphore, mpsc, oneshot, watch}; use uuid::Uuid; use crate::catalog::Catalog; -use crate::coord::appends::BuiltinTableAppendNotify; +use crate::coord::appends::{BuiltinTableAppendNotify, WriteResult}; use crate::coord::consistency::CoordinatorInconsistencies; use crate::coord::peek::{PeekDataflowPlan, PeekResponseUnary}; use crate::coord::timestamp_selection::TimestampDetermination; use crate::coord::{ExecuteContextExtra, ExecuteContextGuard}; use crate::error::AdapterError; +use crate::optimize::LirDataflowDescription; use crate::session::{EndTransactionAction, RowBatchStream, Session}; use crate::statement_logging::{ FrontendStatementLoggingEvent, StatementEndedExecutionReason, StatementExecutionStrategy, @@ -349,6 +350,66 @@ pub enum Command { /// Statement logging event from frontend peek sequencing. /// No response channel needed - this is fire-and-forget. FrontendStatementLogging(FrontendStatementLoggingEvent), + + /// Registers a connection-scoped cancellation watch and returns a receiver + /// that becomes `true` when cancellation is requested for the connection. + /// + /// This is shared by coordinator staged sequencing and frontend + /// read-then-write execution. + RegisterConnectionCancelWatch { + conn_id: ConnectionId, + tx: oneshot::Sender>, + }, + + /// Unregisters a previously registered connection-scoped cancellation watch. + UnregisterConnectionCancelWatch { + conn_id: ConnectionId, + }, + + /// Creates an internal subscribe (not visible in introspection) and returns + /// the response channel. Initially used for frontend-sequenced + /// read-then-write (DELETE/UPDATE/INSERT...SELECT) operations via OCC. + CreateInternalSubscribe { + df_desc: Box, + cluster_id: ComputeInstanceId, + replica_id: Option, + depends_on: BTreeSet, + as_of: mz_repr::Timestamp, + arity: usize, + sink_id: GlobalId, + conn_id: ConnectionId, + session_uuid: Uuid, + start_time: mz_ore::now::EpochMillis, + read_holds: ReadHolds, + tx: oneshot::Sender, AdapterError>>, + }, + + /// Submits a write attempt. Carries the accumulated diffs to write. + /// + /// `write_ts` selects between two modes: + /// - `Some(ts)`: timestamped write at a specific timestamp, fails when the + /// table timestamp is already past that. + /// - `None`: blind write where the coordinator picks the timestamp via the + /// oracle during group commit. This does not fail and will be retried + /// until the write succeeds. + AttemptWrite { + /// Connection originating the write. Used so the coordinator can + /// cancel this pending write if the connection is cancelled before + /// the write commits. + conn_id: ConnectionId, + target_id: CatalogItemId, + diffs: Vec<(Row, Diff)>, + write_ts: Option, + tx: oneshot::Sender, + }, + + /// Drops an internal subscribe. + /// + /// Used for cleanup after the subscribe's purpose is fulfilled or on error. + /// Fire-and-forget — the caller doesn't wait for completion. + DropInternalSubscribe { + sink_id: GlobalId, + }, } impl Command { @@ -386,7 +447,12 @@ impl Command { | Command::UnregisterFrontendPeek { .. } | Command::ExplainTimestamp { .. } | Command::FrontendStatementLogging(..) - | Command::InjectAuditEvents { .. } => None, + | Command::InjectAuditEvents { .. } + | Command::RegisterConnectionCancelWatch { .. } + | Command::UnregisterConnectionCancelWatch { .. } + | Command::CreateInternalSubscribe { .. } + | Command::AttemptWrite { .. } + | Command::DropInternalSubscribe { .. } => None, } } @@ -424,7 +490,12 @@ impl Command { | Command::UnregisterFrontendPeek { .. } | Command::ExplainTimestamp { .. } | Command::FrontendStatementLogging(..) - | Command::InjectAuditEvents { .. } => None, + | Command::InjectAuditEvents { .. } + | Command::RegisterConnectionCancelWatch { .. } + | Command::UnregisterConnectionCancelWatch { .. } + | Command::CreateInternalSubscribe { .. } + | Command::AttemptWrite { .. } + | Command::DropInternalSubscribe { .. } => None, } } } @@ -462,6 +533,15 @@ pub struct StartupResponse { pub optimizer_metrics: OptimizerMetrics, pub persist_client: PersistClient, pub statement_logging_frontend: StatementLoggingFrontend, + /// Semaphore for limiting concurrent OCC (optimistic concurrency control) + /// write operations. + pub occ_write_semaphore: Arc, + /// Whether frontend OCC read-then-write is enabled (determined once at + /// process startup). + pub frontend_read_then_write_enabled: bool, + /// Whether the coordinator is in read-only mode (e.g. during 0dt upgrades). + /// The frontend path must reject mutations when this is true. + pub read_only: bool, } #[derive(Derivative)] diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 079f7ce738fc6..2f6c7ce5ccc17 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -172,7 +172,7 @@ use thiserror::Error; use timely::progress::{Antichain, Timestamp as _}; use tokio::runtime::Handle as TokioHandle; use tokio::select; -use tokio::sync::{OwnedMutexGuard, mpsc, oneshot, watch}; +use tokio::sync::{OwnedMutexGuard, Semaphore, mpsc, oneshot, watch}; use tokio::time::{Interval, MissedTickBehavior}; use tracing::{Instrument, Level, Span, debug, info, info_span, span, warn}; use tracing_opentelemetry::OpenTelemetrySpanExt; @@ -217,6 +217,7 @@ pub(crate) mod id_bundle; pub(crate) mod in_memory_oracle; pub(crate) mod peek; pub(crate) mod read_policy; +pub(crate) mod read_then_write; pub(crate) mod sequencer; pub(crate) mod statement_logging; pub(crate) mod timeline; @@ -457,6 +458,13 @@ impl Message { Command::FrontendStatementLogging(..) => "frontend-statement-logging", Command::StartCopyFromStdin { .. } => "start-copy-from-stdin", Command::InjectAuditEvents { .. } => "inject-audit-events", + Command::RegisterConnectionCancelWatch { .. } => "register-connection-cancel-watch", + Command::UnregisterConnectionCancelWatch { .. } => { + "unregister-connection-cancel-watch" + } + Command::CreateInternalSubscribe { .. } => "create-internal-subscribe", + Command::AttemptWrite { .. } => "attempt-write", + Command::DropInternalSubscribe { .. } => "drop-internal-subscribe", }, Message::ControllerReady { controller: ControllerReadiness::Compute, @@ -1875,9 +1883,15 @@ pub struct Coordinator { /// to stage Batches in Persist that we will then link into the shard. active_copies: BTreeMap, - /// A map from connection ids to a watch channel that is set to `true` if the connection - /// received a cancel request. - staged_cancellation: BTreeMap, watch::Receiver)>, + /// Connection-scoped cancellation watches. + /// + /// Each entry is a watch channel whose value is `false` until cancellation + /// is requested for that connection, at which point it is set to `true`. + /// + /// Consumers install/remove these watches while they have cancellable work + /// in flight. This is used both by coordinator staged sequencing and by + /// frontend read-then-write sequencing. + connection_cancel_watches: BTreeMap, watch::Receiver)>, /// Active introspection subscribes. introspection_subscribes: BTreeMap, @@ -1889,6 +1903,24 @@ pub struct Coordinator { /// Pending writes waiting for a group commit. pending_writes: Vec, + /// Semaphore to limit concurrent OCC (optimistic concurrency control) + /// read-then-write operations. + /// + /// Each operation maintains a subscribe that continually receives and + /// consolidates updates; with N concurrent loops, every successful write + /// forces the other N-1 to redo work, so total work scales as `O(n^2)`. + /// The semaphore caps concurrency to keep that bounded. + /// + /// NOTE: The number of permits is read from `max_concurrent_occ_writes` at + /// coordinator startup; runtime changes require an `environmentd` restart. + occ_write_semaphore: Arc, + + /// Whether frontend OCC read-then-write is enabled. Read once at startup + /// from the `FRONTEND_READ_THEN_WRITE` dyncfg and fixed for the lifetime of + /// this process; see the module-level docs on `frontend_read_then_write` + /// for why mixed-mode operation is not allowed. + frontend_read_then_write_enabled: bool, + /// For the realtime timeline, an explicit SELECT or INSERT on a table will bump the /// table's timestamps, but there are cases where timestamps are not bumped but /// we expect the closed timestamps to advance (`AS OF X`, SUBSCRIBing views over @@ -3686,8 +3718,9 @@ impl Coordinator { // and make it follow from all the Spans in the pending // writes. let user_write_spans = self.pending_writes.iter().flat_map(|x| match x { - PendingWriteTxn::User{span, ..} => Some(span), - PendingWriteTxn::System{..} => None, + PendingWriteTxn::User { span, .. } => Some(span), + PendingWriteTxn::InternalTimestamped(w) => Some(&w.span), + PendingWriteTxn::System { .. } => None, }); let span = match user_write_spans.exactly_one() { Ok(span) => span.clone(), @@ -4689,6 +4722,15 @@ pub fn serve( } let catalog = Arc::new(catalog); + // Read once at startup; changing this system variable requires + // an environmentd restart to take effect (see field doc on + // `occ_write_semaphore`). + let max_concurrent_occ_writes = + usize::cast_from(catalog.system_config().max_concurrent_occ_writes()); + let frontend_read_then_write_enabled = { + use mz_adapter_types::dyncfgs::FRONTEND_READ_THEN_WRITE; + FRONTEND_READ_THEN_WRITE.get(catalog.system_config().dyncfgs()) + }; let caching_secrets_reader = CachingSecretsReader::new(secrets_controller.reader()); let mut coord = Coordinator { @@ -4708,11 +4750,13 @@ pub fn serve( active_compute_sinks: BTreeMap::new(), active_webhooks: BTreeMap::new(), active_copies: BTreeMap::new(), - staged_cancellation: BTreeMap::new(), + connection_cancel_watches: BTreeMap::new(), introspection_subscribes: BTreeMap::new(), write_locks: BTreeMap::new(), deferred_write_ops: BTreeMap::new(), pending_writes: Vec::new(), + occ_write_semaphore: Arc::new(Semaphore::new(max_concurrent_occ_writes)), + frontend_read_then_write_enabled, advance_timelines_interval, secrets_controller, caching_secrets_reader, diff --git a/src/adapter/src/coord/appends.rs b/src/adapter/src/coord/appends.rs index 265ffa5841d6a..4794e8fdc9241 100644 --- a/src/adapter/src/coord/appends.rs +++ b/src/adapter/src/coord/appends.rs @@ -31,6 +31,7 @@ use mz_sql::session::metadata::SessionMetadata; use mz_storage_client::client::TableData; use mz_timestamp_oracle::WriteTimestamp; use smallvec::SmallVec; +use timely::PartialOrder; use tokio::sync::{Notify, OwnedMutexGuard, OwnedSemaphorePermit, Semaphore, oneshot}; use tracing::{Instrument, Span, debug_span, info, warn}; @@ -131,24 +132,113 @@ pub(crate) enum BuiltinTableUpdateSource { Background(oneshot::Sender<()>), } +/// Result of an internally-submitted write (one where the caller receives the +/// result via a oneshot channel rather than through a session `ExecuteContext`). +/// +/// `TimestampPassed` is returned when the caller requested a specific target +/// timestamp but the oracle has already advanced past it by the time group +/// commit runs; the caller decides whether to retry. +#[derive(Debug, Clone)] +pub enum WriteResult { + /// The write was committed at the indicated timestamp. + Success { + /// The timestamp at which the write was committed. + timestamp: Timestamp, + }, + /// The caller requested a specific target timestamp, and the oracle has + /// already advanced past it. + TimestampPassed { + /// The timestamp the write was targeting. + target_timestamp: Timestamp, + /// The current write timestamp from the oracle. + current_write_ts: Timestamp, + }, + /// The write was canceled before it could commit, e.g. because a cancel + /// request arrived for the originating connection. + Canceled, + /// The coordinator is in read-only mode and cannot accept mutations. + /// Indicates the caller bypassed the read-only check at the RTW entry + /// point (see `try_frontend_read_then_write`); this is a defense in + /// depth to avoid panicking the coordinator. + ReadOnly, +} + +/// Where to deliver the result of a [`PendingWriteTxn::User`] write. +#[derive(Debug)] +pub(crate) enum UserWriteResponder { + /// Session-bound write. The coordinator retires the session's + /// `ExecuteContext` once the write commits. + Session(PendingTxn), + /// Internal write originating from `conn_id`. The coordinator sends the + /// result through the oneshot; caller decides what to do with it. The + /// `conn_id` lets cancellation (e.g. `handle_privileged_cancel`) identify + /// and cancel in-flight internal writes for that connection. + Internal { + conn_id: ConnectionId, + result_tx: oneshot::Sender, + }, +} + +impl UserWriteResponder { + /// The connection that originated this write. + pub(crate) fn conn_id(&self) -> &ConnectionId { + match self { + UserWriteResponder::Session(pending_txn) => pending_txn.ctx.session().conn_id(), + UserWriteResponder::Internal { conn_id, .. } => conn_id, + } + } +} + /// A pending write transaction that will be committing during the next group commit. #[derive(Debug)] pub(crate) enum PendingWriteTxn { - /// Write to a user table. + /// Write to a user table. The write timestamp is picked by the oracle + /// during group commit. The write lock is either handed off from the + /// submitting session (via `write_locks: Some(..)`) or acquired during + /// group commit (`write_locks: None`). User { span: Span, /// List of all write operations within the transaction. writes: BTreeMap>, /// If they exist, should contain locks for each [`CatalogItemId`] in `writes`. write_locks: Option, - /// Inner transaction. - pending_txn: PendingTxn, + /// Where to deliver the result once the write commits. + responder: UserWriteResponder, }, /// Write to a system table. System { updates: Vec, source: BuiltinTableUpdateSource, }, + /// A write submitted with a caller-chosen target timestamp. + /// + /// **Important**: This variant supports writes to multiple tables in a + /// single entry. It is the caller's responsibility to ensure that: + /// 1. All writes are consistent (e.g., computed from the same read + /// timestamp) + /// 2. Dependencies between tables have been properly resolved + /// + /// **Concurrency semantics**: Only ONE `InternalTimestamped` write is + /// processed per group commit round. The round's commit advances the oracle + /// past the chosen timestamp, so any other timestamped write targeting the + /// same (or a smaller) timestamp will fail with `TimestampPassed`. + /// Timestamped writes don't acquire write locks: a concurrent regular write + /// landing at the same target timestamp likewise makes one of them fail + /// with `TimestampPassed`. + InternalTimestamped(InternalTimestampedWrite), +} + +/// A write submitted with a caller-chosen target timestamp (see +/// [`PendingWriteTxn::InternalTimestamped`]). +#[derive(Debug)] +pub(crate) struct InternalTimestampedWrite { + /// Connection originating the write. Used by cancellation to identify + /// in-flight internal writes to remove. + pub conn_id: ConnectionId, + pub span: Span, + pub writes: BTreeMap>, + pub target_timestamp: Timestamp, + pub result_tx: oneshot::Sender, } impl PendingWriteTxn { @@ -158,7 +248,12 @@ impl PendingWriteTxn { source: BuiltinTableUpdateSource::Internal(_), .. } => true, - _ => false, + PendingWriteTxn::System { + source: BuiltinTableUpdateSource::Background(_), + .. + } + | PendingWriteTxn::User { .. } + | PendingWriteTxn::InternalTimestamped(_) => false, } } } @@ -259,7 +354,7 @@ impl Coordinator { span, writes, write_locks, - pending_txn, + responder: UserWriteResponder::Session(pending_txn), }); } } @@ -339,12 +434,16 @@ impl Coordinator { match pending_write { // We always allow system writes to proceed. PendingWriteTxn::System { .. } => validated_writes.push(pending_write), + // Timestamped writes don't acquire locks: a concurrent write + // landing at the same target timestamp makes one of them fail + // with `TimestampPassed` instead. + PendingWriteTxn::InternalTimestamped(_) => validated_writes.push(pending_write), // We have a set of locks! Validate they're correct (expected). PendingWriteTxn::User { span, write_locks: Some(write_locks), writes, - pending_txn, + responder, } => match write_locks.validate(writes.keys().copied()) { Ok(validated_locks) => { // Merge all of our write locks together since we can allow concurrent @@ -355,7 +454,7 @@ impl Coordinator { span, writes, write_locks: None, - pending_txn, + responder, }; validated_writes.push(validated_write); } @@ -365,62 +464,109 @@ impl Coordinator { // not taken we could violate serializability. Err(missing) => { let writes: Vec<_> = writes.keys().collect(); + let conn_id = responder.conn_id(); panic!( - "got to group commit with partial set of locks!\nmissing: {:?}, writes: {:?}, txn: {:?}", - missing, writes, pending_txn, + "got to group commit with partial set of locks!\nmissing: {:?}, writes: {:?}, conn_id: {}", + missing, writes, conn_id, ); } }, - // If we don't have any locks, try to acquire them, otherwise defer the write. + // If we don't have any locks, try to acquire them. On failure, + // session-bound writes defer via the lock-wait machinery; + // internal writes push back and re-trigger the group commit. PendingWriteTxn::User { span, writes, write_locks: None, - pending_txn, + responder, } => { let missing = group_write_locks.missing_locks(writes.keys().copied()); if missing.is_empty() { // We have all the locks! Queue the pending write. - let validated_write = PendingWriteTxn::User { + validated_writes.push(PendingWriteTxn::User { span, writes, write_locks: None, - pending_txn, - }; - validated_writes.push(validated_write); - } else { - // Try to acquire the locks we're missing. - let mut just_in_time_locks = WriteLocks::builder(missing.clone()); - for collection in missing { - if let Some(lock) = self.try_grant_object_write_lock(collection) { - just_in_time_locks.insert_lock(collection, lock); + responder, + }); + continue; + } + + match responder { + UserWriteResponder::Session(pending_txn) => { + // Try to acquire the locks we're missing. + let mut just_in_time_locks = WriteLocks::builder(missing.clone()); + for collection in missing { + if let Some(lock) = self.try_grant_object_write_lock(collection) { + just_in_time_locks.insert_lock(collection, lock); + } + } + + match just_in_time_locks + .all_or_nothing(pending_txn.ctx.session().conn_id()) + { + // We acquired all of the locks! Proceed with the write. + Ok(locks) => { + group_write_locks.merge(locks); + validated_writes.push(PendingWriteTxn::User { + span, + writes, + write_locks: None, + responder: UserWriteResponder::Session(pending_txn), + }); + } + // Darn. We couldn't acquire the locks, defer the write. + Err(missing) => { + let acquire_future = + self.grant_object_write_lock(missing).map(Option::Some); + let write = DeferredWrite { + span, + writes, + pending_txn, + }; + deferred_writes.push((acquire_future, write)); + } } } + UserWriteResponder::Internal { conn_id, result_tx } => { + // Internal writes have no session to hand them off to + // the deferred-op machinery, so we acquire directly + // into `group_write_locks` and push back on failure. + let mut acquired_all = true; + let mut acquired_locks = Vec::new(); + for collection in &missing { + if let Some(lock) = self.try_grant_object_write_lock(*collection) { + acquired_locks.push((*collection, lock)); + } else { + acquired_all = false; + break; + } + } - match just_in_time_locks.all_or_nothing(pending_txn.ctx.session().conn_id()) - { - // We acquired all of the locks! Proceed with the write. - Ok(locks) => { - group_write_locks.merge(locks); - let validated_write = PendingWriteTxn::User { + if acquired_all { + for (id, lock) in acquired_locks { + group_write_locks.insert_lock(id, lock); + } + validated_writes.push(PendingWriteTxn::User { span, writes, write_locks: None, - pending_txn, - }; - validated_writes.push(validated_write); - } - // Darn. We couldn't acquire the locks, defer the write. - Err(missing) => { - let acquire_future = - self.grant_object_write_lock(missing).map(Option::Some); - let write = DeferredWrite { + responder: UserWriteResponder::Internal { conn_id, result_tx }, + }); + } else { + // Could not acquire all locks. Push back for the + // next group commit round; the caller is waiting + // on `result_tx` so the retry is transparent to + // them. + drop(acquired_locks); + self.pending_writes.push(PendingWriteTxn::User { span, writes, - pending_txn, - }; - deferred_writes.push((acquire_future, write)); + write_locks: None, + responder: UserWriteResponder::Internal { conn_id, result_tx }, + }); + self.trigger_group_commit(); } } } @@ -433,17 +579,65 @@ impl Coordinator { self.defer_op(acquire_future, DeferredOp::Write(write)); } - // The value returned here still might be ahead of `now()` if `now()` has gone backwards at - // any point during this method or if this was triggered from DDL. We will still commit the - // write without waiting for `now()` to advance. This is ok because the next batch of writes - // will trigger the wait loop in `try_group_commit()` if `now()` hasn't advanced past the - // global timeline, preventing an unbounded advancing of the global timeline ahead of - // `now()`. Additionally DDL is infrequent enough and takes long enough that we don't think - // it's practical for continuous DDL to advance the global timestamp in an unbounded manner. - let WriteTimestamp { - timestamp, - advance_to, - } = self.get_local_write_ts().await; + // Separate timestamped writes from regular writes in a single pass. + let mut regular_writes = Vec::new(); + let mut timestamped_writes: Vec = Vec::new(); + for write in validated_writes { + match write { + PendingWriteTxn::InternalTimestamped(w) => timestamped_writes.push(w), + other => regular_writes.push(other), + } + } + + // Determine the write timestamp, resolving any timestamped writes. + // + // We can only process ONE InternalTimestamped write per group commit + // round: after committing at timestamp T the oracle advances past T, + // so any other write targeting T (or smaller) would fail with + // `TimestampPassed` anyway — we fail them early here. + // + // When a timestamped write is selected, we apply ONLY that write in + // this round and defer all regular writes to the next round. This + // avoids any subtle interactions between the timestamped write (which + // targets a specific timestamp) and regular writes. + let resolved = if !timestamped_writes.is_empty() { + self.resolve_timestamped_writes(timestamped_writes).await + } else { + None + }; + + let (timestamp, advance_to, validated_writes) = match resolved { + Some((ts, advance_to, selected_write)) => { + // Defer all regular writes to the next round. + self.pending_writes.extend(regular_writes); + if !self.pending_writes.is_empty() { + self.trigger_group_commit(); + } + ( + ts, + advance_to, + vec![PendingWriteTxn::InternalTimestamped(selected_write)], + ) + } + None => { + // The value returned here still might be ahead of `now()` if + // `now()` has gone backwards at any point during this method or + // if this was triggered from DDL. We will still commit the + // write without waiting for `now()` to advance. This is ok + // because the next batch of writes will trigger the wait loop + // in `try_group_commit()` if `now()` hasn't advanced past the + // global timeline, preventing an unbounded advancing of the + // global timeline ahead of `now()`. Additionally DDL is + // infrequent enough and takes long enough that we don't think + // it's practical for continuous DDL to advance the global + // timestamp in an unbounded manner. + let WriteTimestamp { + timestamp, + advance_to, + } = self.get_local_write_ts().await; + (timestamp, advance_to, regular_writes) + } + }; // Advance the catalog shard's upper to keep it in sync with the oracle // timestamp. This ensures that reads of mz_catalog_raw at the oracle's @@ -460,6 +654,7 @@ impl Coordinator { let mut appends: BTreeMap> = BTreeMap::new(); let mut responses = Vec::with_capacity(validated_writes.len()); let mut notifies = Vec::new(); + let mut write_result_txs = Vec::new(); for validated_write_txn in validated_writes { match validated_write_txn { @@ -467,12 +662,7 @@ impl Coordinator { span: _, writes, write_locks, - pending_txn: - PendingTxn { - ctx, - response, - action, - }, + responder, } => { assert_none!(write_locks, "should have merged together all locks above"); for (id, table_data) in writes { @@ -485,11 +675,24 @@ impl Coordinator { appends.entry(id).or_default().extend(table_data); } } - if let Some(id) = ctx.extra().contents() { - self.set_statement_execution_timestamp(id, timestamp); + match responder { + UserWriteResponder::Session(PendingTxn { + ctx, + response, + action, + }) => { + if let Some(id) = ctx.extra().contents() { + self.set_statement_execution_timestamp(id, timestamp); + } + responses.push(CompletedClientTransmitter::new(ctx, response, action)); + } + UserWriteResponder::Internal { + conn_id: _, + result_tx, + } => { + write_result_txs.push(result_tx); + } } - - responses.push(CompletedClientTransmitter::new(ctx, response, action)); } PendingWriteTxn::System { updates, source } => { for update in updates { @@ -501,6 +704,20 @@ impl Coordinator { | BuiltinTableUpdateSource::Background(tx) => notifies.push(tx), } } + PendingWriteTxn::InternalTimestamped(InternalTimestampedWrite { + conn_id: _, + span: _, + writes, + target_timestamp: _, + result_tx, + }) => { + for (id, table_data) in writes { + if self.catalog().try_get_entry(&id).is_some() { + appends.entry(id).or_default().extend(table_data); + } + } + write_result_txs.push(result_tx); + } } } @@ -580,9 +797,10 @@ impl Coordinator { .instrument(debug_span!("group_commit_apply::append_fut")) .await { - Ok(append_result) => { - append_result.unwrap_or_terminate("cannot fail to apply appends") - } + Ok(append_result) => append_result.unwrap_or_terminate(&format!( + "cannot fail to apply appends at {} ({:?})", + timestamp, permit + )), Err(_) => warn!("Writer terminated with writes in indefinite state"), }; @@ -613,6 +831,11 @@ impl Coordinator { // We don't care if the listeners have gone away. let _ = notify.send(()); } + + // Notify callers of internally-submitted writes. + for result_tx in write_result_txs { + let _ = result_tx.send(WriteResult::Success { timestamp }); + } } .instrument(span), ); @@ -620,6 +843,81 @@ impl Coordinator { timestamp } + /// Pick at most one timestamped write to apply this group-commit round. + /// + /// Selects the write at the lowest eligible target timestamp; writes whose + /// timestamp has already passed (or that tie at the chosen timestamp) are + /// failed with `TimestampPassed`, and writes at higher timestamps are + /// deferred to the next round. Returns `None` if all timestamped writes + /// had already passed, in which case the caller falls back to the oracle. + async fn resolve_timestamped_writes( + &mut self, + timestamped_writes: Vec, + ) -> Option<(Timestamp, Timestamp, InternalTimestampedWrite)> { + debug_assert!(!timestamped_writes.is_empty()); + + let next_eligible_write_ts = self.peek_local_write_ts().await.step_forward(); + + // Find the lowest eligible target timestamp in a single scan. + let lowest_eligible_ts = timestamped_writes + .iter() + .filter(|w| !w.target_timestamp.less_than(&next_eligible_write_ts)) + .map(|w| w.target_timestamp) + .min(); + + let Some(target_ts) = lowest_eligible_ts else { + // All timestamped writes have passed. Fail them all; the caller + // will fall back to the normal oracle for regular writes. + for w in timestamped_writes { + let _ = w.result_tx.send(WriteResult::TimestampPassed { + target_timestamp: w.target_timestamp, + current_write_ts: next_eligible_write_ts, + }); + } + return None; + }; + + // Dispatch each timestamped write in a single pass: + // - Passed (target < next_eligible_write_ts): fail with TimestampPassed + // - At target_ts, first one seen: select for this round + // - At target_ts, subsequent: fail (only one per round) + // - Above target_ts: defer to next round + let mut selected: Option = None; + for w in timestamped_writes { + if w.target_timestamp.less_than(&next_eligible_write_ts) { + // Timestamp has passed. + let _ = w.result_tx.send(WriteResult::TimestampPassed { + target_timestamp: w.target_timestamp, + current_write_ts: next_eligible_write_ts, + }); + } else if w.target_timestamp == target_ts && selected.is_none() { + // Winner: include in this round. + selected = Some(w); + } else if w.target_timestamp == target_ts { + // Loser at same timestamp — only one can write per round. + let _ = w.result_tx.send(WriteResult::TimestampPassed { + target_timestamp: w.target_timestamp, + current_write_ts: target_ts.step_forward(), + }); + } else { + // Higher timestamp — defer to next round. + self.pending_writes + .push(PendingWriteTxn::InternalTimestamped(w)); + } + } + + // If there are more timestamped writes waiting, trigger another group commit. + if self + .pending_writes + .iter() + .any(|w| matches!(w, PendingWriteTxn::InternalTimestamped(_))) + { + self.trigger_group_commit(); + } + + selected.map(|write| (target_ts, target_ts.step_forward(), write)) + } + /// Submit a write to be executed during the next group commit and trigger a group commit. pub(crate) fn submit_write(&mut self, pending_write_txn: PendingWriteTxn) { if self.controller.read_only() { diff --git a/src/adapter/src/coord/catalog_implications.rs b/src/adapter/src/coord/catalog_implications.rs index 2e0ccb655ccbb..b3bd98703af30 100644 --- a/src/adapter/src/coord/catalog_implications.rs +++ b/src/adapter/src/coord/catalog_implications.rs @@ -65,6 +65,7 @@ use crate::coord::Coordinator; use crate::coord::catalog_implications::parsed_state_updates::{ ParsedStateUpdate, ParsedStateUpdateKind, }; +use crate::coord::peek::DroppedDependency; use crate::coord::timeline::TimelineState; use crate::statement_logging::{StatementEndedExecutionReason, StatementLoggingId}; use crate::{AdapterError, CollectionIdBundle, ExecuteContext, ResultExt}; @@ -784,20 +785,24 @@ impl Coordinator { { let name = dropped_item_names .get(id) - .map(|n| format!("relation {}", n.quoted())) + .map(|n| n.quoted().to_string()) .expect("missing relation name"); active_compute_sinks_to_drop.insert( *sink_id, - ActiveComputeSinkRetireReason::DependencyDropped(name), + ActiveComputeSinkRetireReason::DependencyDropped(DroppedDependency::Relation { + name, + }), ); } else if clusters_to_drop.contains(&cluster_id) { let name = dropped_cluster_names .get(&cluster_id) - .map(|n| format!("cluster {}", n.quoted())) + .map(|n| n.quoted().to_string()) .expect("missing cluster name"); active_compute_sinks_to_drop.insert( *sink_id, - ActiveComputeSinkRetireReason::DependencyDropped(name), + ActiveComputeSinkRetireReason::DependencyDropped(DroppedDependency::Cluster { + name, + }), ); } } @@ -811,15 +816,15 @@ impl Coordinator { { let name = dropped_item_names .get(id) - .map(|n| format!("relation {}", n.quoted())) + .map(|n| n.quoted().to_string()) .expect("missing relation name"); - peeks_to_drop.push((name, uuid.clone())); + peeks_to_drop.push((DroppedDependency::Relation { name }, uuid.clone())); } else if clusters_to_drop.contains(&pending_peek.cluster_id) { let name = dropped_cluster_names .get(&pending_peek.cluster_id) - .map(|n| format!("cluster {}", n.quoted())) + .map(|n| n.quoted().to_string()) .expect("missing cluster name"); - peeks_to_drop.push((name, uuid.clone())); + peeks_to_drop.push((DroppedDependency::Cluster { name }, uuid.clone())); } } @@ -926,11 +931,9 @@ impl Coordinator { } if !peeks_to_drop.is_empty() { - for (dropped_name, uuid) in peeks_to_drop { + for (dep, uuid) in peeks_to_drop { if let Some(pending_peek) = self.remove_pending_peek(&uuid) { - let cancel_reason = PeekResponse::Error(format!( - "query could not complete because {dropped_name} was dropped" - )); + let cancel_reason = PeekResponse::Error(dep.query_terminated_error()); self.controller .compute .cancel_peek(pending_peek.cluster_id, uuid, cancel_reason) diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index d58c6b4c095aa..9ff705efb97f5 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -61,7 +61,7 @@ use mz_sql_parser::ast::{ }; use mz_storage_types::sources::Timeline; use opentelemetry::trace::TraceContextExt; -use tokio::sync::{mpsc, oneshot}; +use tokio::sync::{mpsc, oneshot, watch}; use tracing::{Instrument, debug_span, info, warn}; use tracing_opentelemetry::OpenTelemetrySpanExt; use uuid::Uuid; @@ -70,7 +70,7 @@ use crate::command::{ CatalogSnapshot, Command, ExecuteResponse, Response, SASLChallengeResponse, SASLVerifyProofResponse, StartupResponse, SuperuserAttribute, }; -use crate::coord::appends::PendingWriteTxn; +use crate::coord::appends::{PendingWriteTxn, UserWriteResponder, WriteResult}; use crate::coord::peek::PendingPeek; use crate::coord::{ ConnMeta, Coordinator, DeferredPlanStatement, Message, PendingTxn, PlanStatement, PlanValidity, @@ -562,6 +562,65 @@ impl Coordinator { Command::FrontendStatementLogging(event) => { self.handle_frontend_statement_logging_event(event); } + Command::RegisterConnectionCancelWatch { conn_id, tx } => { + // Always replace any existing entry. Another code path + // (e.g. `sequence_staged`) may have left a stale watch + // here, possibly already signaled `true` from a prior + // cancel. Reusing it via `or_insert_with` would hand out + // a `Receiver` that already reads `true`, causing the new + // operation to immediately return `Canceled` even though + // it hasn't been cancelled. + let (watch_tx, watch_rx) = watch::channel(false); + self.connection_cancel_watches + .insert(conn_id, (watch_tx, watch_rx.clone())); + let _ = tx.send(watch_rx); + } + Command::UnregisterConnectionCancelWatch { conn_id } => { + self.connection_cancel_watches.remove(&conn_id); + } + Command::CreateInternalSubscribe { + df_desc, + cluster_id, + replica_id, + depends_on, + as_of, + arity, + sink_id, + conn_id, + session_uuid, + start_time, + read_holds, + tx, + } => { + self.handle_create_internal_subscribe( + *df_desc, + cluster_id, + replica_id, + depends_on, + as_of, + arity, + sink_id, + conn_id, + session_uuid, + start_time, + read_holds, + tx, + ) + .await; + } + Command::AttemptWrite { + conn_id, + target_id, + diffs, + write_ts, + tx, + } => { + self.handle_attempt_write(conn_id, target_id, diffs, write_ts, tx) + .await; + } + Command::DropInternalSubscribe { sink_id } => { + self.drop_internal_subscribe(sink_id).await; + } } } .instrument(debug_span!("handle_command")) @@ -847,6 +906,9 @@ impl Coordinator { persist_client: self.persist_client.clone(), statement_logging_frontend, superuser_attribute, + occ_write_semaphore: Arc::clone(&self.occ_write_semaphore), + frontend_read_then_write_enabled: self.frontend_read_then_write_enabled, + read_only: self.controller.read_only(), }); if tx.send(resp).is_err() { // Failed to send to adapter, but everything is setup so we can terminate @@ -1806,20 +1868,45 @@ impl Coordinator { pub(crate) async fn handle_privileged_cancel(&mut self, conn_id: ConnectionId) { let mut maybe_ctx = None; - // Cancel pending writes. There is at most one pending write per session. - let pending_write_idx = self.pending_writes.iter().position(|pending_write_txn| { - matches!(pending_write_txn, PendingWriteTxn::User { - pending_txn: PendingTxn { ctx, .. }, - .. - } if *ctx.session().conn_id() == conn_id) - }); - if let Some(idx) = pending_write_idx { - if let PendingWriteTxn::User { - pending_txn: PendingTxn { ctx, .. }, - .. - } = self.pending_writes.remove(idx) - { - maybe_ctx = Some(ctx); + // Cancel all pending writes for this connection: + // - At most one session-bound write (`UserWriteResponder::Session`); + // retired via its `ExecuteContext` below. + // - Any number of internal writes submitted by the frontend + // read-then-write path (`UserWriteResponder::Internal` or + // `InternalTimestamped`); the waiter receives `WriteResult::Canceled`. + let (cancelled, kept): (Vec<_>, Vec<_>) = std::mem::take(&mut self.pending_writes) + .into_iter() + .partition(|pending_write_txn| match pending_write_txn { + PendingWriteTxn::User { + responder: UserWriteResponder::Session(PendingTxn { ctx, .. }), + .. + } => *ctx.session().conn_id() == conn_id, + PendingWriteTxn::User { + responder: UserWriteResponder::Internal { conn_id: c, .. }, + .. + } => *c == conn_id, + PendingWriteTxn::InternalTimestamped(w) => w.conn_id == conn_id, + PendingWriteTxn::System { .. } => false, + }); + self.pending_writes = kept; + for pending in cancelled { + match pending { + PendingWriteTxn::User { + responder: UserWriteResponder::Session(PendingTxn { ctx, .. }), + .. + } => { + maybe_ctx = Some(ctx); + } + PendingWriteTxn::User { + responder: UserWriteResponder::Internal { result_tx, .. }, + .. + } => { + let _ = result_tx.send(WriteResult::Canceled); + } + PendingWriteTxn::InternalTimestamped(w) => { + let _ = w.result_tx.send(WriteResult::Canceled); + } + PendingWriteTxn::System { .. } => unreachable!("filtered out above"), } } @@ -1858,7 +1945,7 @@ impl Coordinator { self.cancel_cluster_reconfigurations_for_conn(&conn_id) .await; self.cancel_pending_copy(&conn_id); - if let Some((tx, _rx)) = self.staged_cancellation.get_mut(&conn_id) { + if let Some((tx, _rx)) = self.connection_cancel_watches.get_mut(&conn_id) { let _ = tx.send(true); } } diff --git a/src/adapter/src/coord/ddl.rs b/src/adapter/src/coord/ddl.rs index 31fd5bffd00da..ca6974548ff15 100644 --- a/src/adapter/src/coord/ddl.rs +++ b/src/adapter/src/coord/ddl.rs @@ -660,7 +660,11 @@ impl Coordinator { for sink_id in sink_ids { let sink = match self.remove_active_compute_sink(sink_id).await { None => { - tracing::error!(%sink_id, "drop_compute_sinks called on nonexistent sink"); + // This can happen due to a race condition: an internal + // subscribe may be cleaned up via its own message while + // session disconnect cleanup is in progress. This is + // benign. + tracing::debug!(%sink_id, "drop_compute_sinks: sink already removed"); continue; } Some(sink) => sink, diff --git a/src/adapter/src/coord/peek.rs b/src/adapter/src/coord/peek.rs index 8ed68ac38a4ac..298016e2486d8 100644 --- a/src/adapter/src/coord/peek.rs +++ b/src/adapter/src/coord/peek.rs @@ -86,6 +86,46 @@ pub enum PeekResponseUnary { Rows(Box), Error(String), Canceled, + /// A dependency was dropped during execution. + /// + /// N.B. This is a bit of a workaround for the fact that our Error variant + /// is unstructured and right now we specifically care about this error and + /// need to render differently based on context. + DependencyDropped(DroppedDependency), +} + +/// A dependency that was dropped while a peek or subscribe was in flight. +/// +/// The `name` fields are already quoted (e.g. `"db.schema.t"` or `"c"`), so +/// `Display` produces the canonical descriptor — `relation "db.schema.t"` or +/// `cluster "c"` — that callers can drop directly into their error wording. +#[derive(Clone, Debug)] +pub enum DroppedDependency { + Relation { name: String }, + Cluster { name: String }, +} + +impl fmt::Display for DroppedDependency { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::Relation { name } => write!(f, "relation {name}"), + Self::Cluster { name } => write!(f, "cluster {name}"), + } + } +} + +impl DroppedDependency { + /// User-facing error for a query (peek or DML-via-internal-subscribe) that + /// could not finish because this dependency was dropped mid-flight. + pub fn query_terminated_error(&self) -> String { + format!("query could not complete because {self} was dropped") + } + + /// User-facing error for a `COPY ... TO` that was terminated because this + /// dependency was dropped mid-flight. + pub fn copy_terminated_error(&self) -> String { + format!("copy has been terminated because underlying {self} was dropped") + } } #[derive(Clone, Debug)] diff --git a/src/adapter/src/coord/read_then_write.rs b/src/adapter/src/coord/read_then_write.rs new file mode 100644 index 0000000000000..f39a7b70ed51e --- /dev/null +++ b/src/adapter/src/coord/read_then_write.rs @@ -0,0 +1,284 @@ +// 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. + +//! Coordinator-side support machinery for (frontend) read-then write. +//! +//! N.B. It's a bit annoying that we still have the write submission go through +//! the coordinator. We can imagine in the long run we want a group-commit task +//! that runs independently, and we can directly submit write requests there. + +use std::collections::BTreeSet; + +use mz_catalog::memory::objects::CatalogItem; +use mz_repr::CatalogItemId; +use mz_repr::{Diff, GlobalId, Row, Timestamp}; +use mz_sql::catalog::CatalogItemType; +use mz_sql::plan::SubscribeOutput; +use tokio::sync::mpsc; + +use crate::PeekResponseUnary; +use crate::active_compute_sink::{ActiveComputeSink, ActiveSubscribe}; +use crate::catalog::Catalog; +use crate::coord::Coordinator; +use crate::coord::appends::WriteResult; +use crate::error::AdapterError; + +impl Coordinator { + /// Handle a Command to create an internal subscribe. + /// + /// Internal subscribes are not visible in introspection collections. They + /// are initially used for frontend-sequenced read-then-write + /// (DELETE/UPDATE/INSERT ...SELECT) via OCC. + /// + /// This is called from the frontend OCC implementation after it has + /// acquired the semaphore permit. We create the subscribe here (on the + /// coordinator) and return the channel to the caller. + /// + /// The `read_holds` parameter contains the read holds for this specific + /// operation. They are passed directly through the stages (not via the + /// connection-keyed txn_read_holds map) to avoid issues where multiple + /// operations on the same connection could interfere with each other's + /// holds. + #[allow(clippy::too_many_arguments)] + pub(crate) async fn handle_create_internal_subscribe( + &mut self, + df_desc: crate::optimize::LirDataflowDescription, + cluster_id: mz_compute_types::ComputeInstanceId, + replica_id: Option, + depends_on: BTreeSet, + as_of: Timestamp, + arity: usize, + sink_id: GlobalId, + conn_id: mz_adapter_types::connection::ConnectionId, + session_uuid: uuid::Uuid, + start_time: mz_ore::now::EpochMillis, + read_holds: crate::ReadHolds, + response_tx: tokio::sync::oneshot::Sender< + Result, AdapterError>, + >, + ) { + // Client disconnected while waiting for the semaphore. + if !self.active_conns.contains_key(&conn_id) { + let _ = response_tx.send(Err(AdapterError::Canceled)); + return; + } + + let (tx, rx) = mpsc::unbounded_channel(); + + let active_subscribe = ActiveSubscribe { + conn_id: conn_id.clone(), + session_uuid, + channel: tx, + emit_progress: true, // We need progress updates for OCC + as_of, + arity, + cluster_id, + depends_on, + start_time, + output: SubscribeOutput::Diffs, + internal: true, // skip builtin table updates and metrics + }; + active_subscribe.initialize(); + + let write_notify_fut = self + .add_active_compute_sink(sink_id, ActiveComputeSink::Subscribe(active_subscribe)) + .await; + let ship_dataflow_fut = self.ship_dataflow(df_desc, cluster_id, replica_id); + let ((), ()) = futures::future::join(write_notify_fut, ship_dataflow_fut).await; + + let _ = response_tx.send(Ok(rx)); + + // Drop read holds only after `ship_dataflow` returns, so the since + // can't advance past `as_of` before the dataflow is running. + drop(read_holds); + } + + /// Handle a write attempt from the frontend OCC read-then-write loop. + /// + /// If `write_ts` is `Some`, submits an `InternalTimestamped` write targeting + /// that timestamp. If `None`, submits a regular `User` write whose + /// timestamp the oracle picks at group-commit time (a blind write). + pub(crate) async fn handle_attempt_write( + &mut self, + conn_id: mz_adapter_types::connection::ConnectionId, + target_id: mz_repr::CatalogItemId, + diffs: Vec<(Row, Diff)>, + write_ts: Option, + result_tx: tokio::sync::oneshot::Sender, + ) { + use crate::coord::appends::{ + InternalTimestampedWrite, PendingWriteTxn, UserWriteResponder, + }; + use mz_ore::soft_panic_or_log; + use mz_storage_client::client::TableData; + use smallvec::smallvec; + use std::collections::BTreeMap; + use tracing::Span; + + // `try_frontend_read_then_write` checks read-only at the entry point, + // so reaching here in read-only mode means that check is stale (e.g. + // the PeekClient's `read_only` snapshot from session startup doesn't + // match current controller state). Surface it to the caller rather + // than panicking the coordinator. + if self.controller.read_only() { + soft_panic_or_log!( + "attempting OCC read-then-write in read-only mode: write_ts={:?}, target_id={:?}", + write_ts, + target_id, + ); + let _ = result_tx.send(WriteResult::ReadOnly); + return; + } + + let table_data = TableData::Rows(diffs); + let writes = BTreeMap::from([(target_id, smallvec![table_data])]); + + match write_ts { + Some(write_ts) => { + // Reject early if the oracle has already advanced past the + // requested timestamp; the caller will retry. + let next_eligible_write_ts = self.peek_local_write_ts().await.step_forward(); + if write_ts < next_eligible_write_ts { + let _ = result_tx.send(WriteResult::TimestampPassed { + target_timestamp: write_ts, + current_write_ts: next_eligible_write_ts, + }); + return; + } + + tracing::trace!(?writes, ?write_ts, "about to attempt read-then-write"); + + self.pending_writes + .push(PendingWriteTxn::InternalTimestamped( + InternalTimestampedWrite { + conn_id, + span: Span::current(), + writes, + target_timestamp: write_ts, + result_tx, + }, + )); + } + None => { + // Blind write: the coordinator picks the timestamp via the + // oracle during group commit. Goes through the regular `User` + // path (including write-lock acquisition), with the result + // delivered via the oneshot channel instead of a session. + tracing::trace!(?writes, "about to attempt blind read-then-write"); + + self.pending_writes.push(PendingWriteTxn::User { + span: Span::current(), + writes, + write_locks: None, + responder: UserWriteResponder::Internal { conn_id, result_tx }, + }); + } + } + self.trigger_group_commit(); + } + + /// Drop an internal subscribe. + pub(crate) async fn drop_internal_subscribe(&mut self, sink_id: GlobalId) { + // Use drop_compute_sink instead of remove_active_compute_sink to also + // cancel the dataflow on the compute side, not just remove bookkeeping. + let _ = self.drop_compute_sink(sink_id).await; + } +} + +/// Validates that all dependencies are valid for read-then-write operations. +/// +/// Ensures all objects the selection depends on are valid for `ReadThenWrite` operations: +/// +/// - They do not refer to any objects whose notion of time moves differently than that of +/// user tables. This limitation is meant to ensure no writes occur between this read and the +/// subsequent write. +/// - They do not use mz_now(), whose time produced during read will differ from the write +/// timestamp. +pub(crate) fn validate_read_then_write_dependencies( + catalog: &Catalog, + id: &CatalogItemId, +) -> Result<(), AdapterError> { + use CatalogItemType::*; + use mz_catalog::memory::objects; + let mut ids_to_check = Vec::new(); + let valid = match catalog.try_get_entry(id) { + Some(entry) => { + if let CatalogItem::View(objects::View { + locally_optimized_expr: optimized_expr, + .. + }) + | CatalogItem::MaterializedView(objects::MaterializedView { + locally_optimized_expr: optimized_expr, + .. + }) = entry.item() + { + if optimized_expr.contains_temporal() { + return Err(AdapterError::Unsupported( + "calls to mz_now in write statements", + )); + } + } + match entry.item().typ() { + typ @ (Func | View | MaterializedView | ContinualTask) => { + ids_to_check.extend(entry.uses()); + let valid_id = id.is_user() || matches!(typ, Func); + valid_id + } + Source | Secret | Connection => false, + // Cannot select from sinks or indexes. + Sink | Index => unreachable!(), + Table => { + if !id.is_user() { + // We can't read from non-user tables + false + } else { + // We can't read from tables that are source-exports + entry.source_export_details().is_none() + } + } + Type => true, + } + } + None => false, + }; + if !valid { + let (object_name, object_type) = match catalog.try_get_entry(id) { + Some(entry) => { + let object_name = catalog.resolve_full_name(entry.name(), None).to_string(); + let object_type = match entry.item().typ() { + // We only need the disallowed types here; the allowed types are handled above. + Source => "source", + Secret => "secret", + Connection => "connection", + Table => { + if !id.is_user() { + "system table" + } else { + "source-export table" + } + } + View => "system view", + MaterializedView => "system materialized view", + ContinualTask => "system task", + _ => "invalid dependency", + }; + (object_name, object_type.to_string()) + } + None => (id.to_string(), "unknown".to_string()), + }; + return Err(AdapterError::InvalidTableMutationSelection { + object_name, + object_type, + }); + } + for id in ids_to_check { + validate_read_then_write_dependencies(catalog, &id)?; + } + Ok(()) +} diff --git a/src/adapter/src/coord/sequencer.rs b/src/adapter/src/coord/sequencer.rs index dfde2c5cb419d..b81fb154c87c4 100644 --- a/src/adapter/src/coord/sequencer.rs +++ b/src/adapter/src/coord/sequencer.rs @@ -882,6 +882,13 @@ impl Coordinator { // Consolidate rows. This is useful e.g. for an UPDATE where the row // doesn't change, and we need to reflect that in the number of // affected rows. + // + // NOTE: This behavior differs from PostgreSQL. In PostgreSQL, + // `UPDATE t SET x = x` reports the number of rows matching the WHERE + // clause, even if no values actually change. In Materialize, because + // we use differential dataflow, the +1 and -1 diffs for unchanged rows + // cancel out during consolidation, resulting in 0 affected rows. + // This has been Materialize's behavior since early versions. differential_dataflow::consolidation::consolidate(&mut plan.updates); affected_rows = Diff::ZERO; diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index ec3f127228e42..bc8c206b28f57 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -20,7 +20,9 @@ use futures::{Future, StreamExt, future}; use itertools::Itertools; 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_adapter_types::dyncfgs::{ + ENABLE_MULTI_REPLICA_SOURCES, ENABLE_PASSWORD_AUTH, FRONTEND_READ_THEN_WRITE, +}; use mz_catalog::memory::error::ErrorKind; use mz_catalog::memory::objects::{ CatalogItem, Connection, DataSourceDesc, Sink, Source, Table, TableDataSource, Type, @@ -51,7 +53,7 @@ use mz_sql::ast::{ use mz_sql::ast::{CreateSubsourceStatement, MySqlConfigOptionName, UnresolvedItemName}; use mz_sql::catalog::{ CatalogCluster, CatalogClusterReplica, CatalogDatabase, CatalogError, - CatalogItem as SqlCatalogItem, CatalogItemType, CatalogRole, CatalogSchema, CatalogTypeDetails, + CatalogItem as SqlCatalogItem, CatalogRole, CatalogSchema, CatalogTypeDetails, ErrorMessageObjectDescription, ObjectType, RoleAttributesRaw, RoleVars, SessionCatalog, }; use mz_sql::names::{ @@ -95,9 +97,12 @@ use timely::progress::Antichain; use tokio::sync::{oneshot, watch}; use tracing::{Instrument, Span, info, warn}; -use crate::catalog::{self, Catalog, ConnCatalog, DropObjectInfo, UpdatePrivilegeVariant}; +use crate::catalog::{self, ConnCatalog, DropObjectInfo, UpdatePrivilegeVariant}; use crate::command::{ExecuteResponse, Response}; -use crate::coord::appends::{BuiltinTableAppendNotify, DeferredOp, DeferredPlan, PendingWriteTxn}; +use crate::coord::appends::{ + BuiltinTableAppendNotify, DeferredOp, DeferredPlan, PendingWriteTxn, UserWriteResponder, +}; +use crate::coord::read_then_write::validate_read_then_write_dependencies; use crate::coord::sequencer::emit_optimizer_notices; use crate::coord::{ AlterConnectionValidationReady, AlterMaterializedViewReadyContext, AlterSinkReadyContext, @@ -159,10 +164,12 @@ struct CreateSourceInner { } impl Coordinator { - /// Sequences the next staged of a [Staged] plan. This is designed for use with plans that - /// execute both on and off of the coordinator thread. Stages can either produce another stage - /// to execute or a final response. An explicit [Span] is passed to allow for convenient - /// tracing. + /// Sequences a [Staged] plan. + /// + /// This is designed for plans that execute both on and off the coordinator + /// thread. Stages can either produce another stage to execute or a final + /// response. Maintains the connection-scoped cancel watch in + /// `connection_cancel_watches` while a stage is cancelable. pub(crate) async fn sequence_staged( &mut self, mut ctx: S::Ctx, @@ -180,7 +187,7 @@ impl Coordinator { // Channel to await cancellation. Insert a new channel, but check if the previous one // was already canceled. if let Some((_prev_tx, prev_rx)) = self - .staged_cancellation + .connection_cancel_watches .insert(session.conn_id().clone(), watch::channel(false)) { let was_canceled = *prev_rx.borrow(); @@ -192,7 +199,7 @@ impl Coordinator { } else { // If no cancel allowed, remove it so handle_spawn doesn't observe any previous value // when cancel_enabled may have been true on an earlier stage. - self.staged_cancellation.remove(session.conn_id()); + self.connection_cancel_watches.remove(session.conn_id()); } } else { cancel_enabled = false @@ -225,6 +232,8 @@ impl Coordinator { } } + /// Waits for either the spawned stage work to complete or cancellation to + /// be signaled through the connection-scoped cancel watch. fn handle_spawn( &self, ctx: C, @@ -238,7 +247,7 @@ impl Coordinator { { let rx: BoxFuture<()> = if let Some((_tx, rx)) = ctx .session() - .and_then(|session| self.staged_cancellation.get(session.conn_id())) + .and_then(|session| self.connection_cancel_watches.get(session.conn_id())) { let mut rx = rx.clone(); Box::pin(async move { @@ -2078,11 +2087,11 @@ impl Coordinator { span: Span::current(), writes: collected_writes, write_locks: validated_locks, - pending_txn: PendingTxn { + responder: UserWriteResponder::Session(PendingTxn { ctx, response, action, - }, + }), }); return; } @@ -2705,99 +2714,10 @@ impl Coordinator { return; } - // Ensure all objects `selection` depends on are valid for `ReadThenWrite` operations: - // - // - They do not refer to any objects whose notion of time moves differently than that of - // user tables. This limitation is meant to ensure no writes occur between this read and the - // subsequent write. - // - They do not use mz_now(), whose time produced during read will differ from the write - // timestamp. - fn validate_read_dependencies( - catalog: &Catalog, - id: &CatalogItemId, - ) -> Result<(), AdapterError> { - use CatalogItemType::*; - use mz_catalog::memory::objects; - let mut ids_to_check = Vec::new(); - let valid = match catalog.try_get_entry(id) { - Some(entry) => { - if let CatalogItem::View(objects::View { - locally_optimized_expr: optimized_expr, - .. - }) - | CatalogItem::MaterializedView(objects::MaterializedView { - locally_optimized_expr: optimized_expr, - .. - }) = entry.item() - { - if optimized_expr.contains_temporal() { - return Err(AdapterError::Unsupported( - "calls to mz_now in write statements", - )); - } - } - match entry.item().typ() { - typ @ (Func | View | MaterializedView | ContinualTask) => { - ids_to_check.extend(entry.uses()); - let valid_id = id.is_user() || matches!(typ, Func); - valid_id - } - Source | Secret | Connection => false, - // Cannot select from sinks or indexes. - Sink | Index => unreachable!(), - Table => { - if !id.is_user() { - // We can't read from non-user tables - false - } else { - // We can't read from tables that are source-exports - entry.source_export_details().is_none() - } - } - Type => true, - } - } - None => false, - }; - if !valid { - let (object_name, object_type) = match catalog.try_get_entry(id) { - Some(entry) => { - let object_name = catalog.resolve_full_name(entry.name(), None).to_string(); - let object_type = match entry.item().typ() { - // We only need the disallowed types here; the allowed types are handled above. - Source => "source", - Secret => "secret", - Connection => "connection", - Table => { - if !id.is_user() { - "system table" - } else { - "source-export table" - } - } - View => "system view", - MaterializedView => "system materialized view", - ContinualTask => "system task", - _ => "invalid dependency", - }; - (object_name, object_type.to_string()) - } - None => (id.to_string(), "unknown".to_string()), - }; - return Err(AdapterError::InvalidTableMutationSelection { - object_name, - object_type, - }); - } - for id in ids_to_check { - validate_read_dependencies(catalog, &id)?; - } - Ok(()) - } - + // Ensure all objects `selection` depends on are valid for `ReadThenWrite` operations. for gid in selection.depends_on() { let item_id = self.catalog().resolve_item_id(&gid); - if let Err(err) = validate_read_dependencies(self.catalog(), &item_id) { + if let Err(err) = validate_read_then_write_dependencies(self.catalog(), &item_id) { ctx.retire(Err(err)); return; } @@ -2964,6 +2884,11 @@ impl Coordinator { PeekResponseUnary::Error(e) => { break Err(AdapterError::Unstructured(anyhow!(e))); } + PeekResponseUnary::DependencyDropped(dep) => { + break Err(AdapterError::Unstructured(anyhow!( + dep.query_terminated_error() + ))); + } }, Ok(None) => break Ok(diffs), Err(_) => { @@ -4128,6 +4053,7 @@ impl Coordinator { plan::AlterSystemSetPlan { name, value }: plan::AlterSystemSetPlan, ) -> Result { self.is_user_allowed_to_alter_system(session, Some(&name))?; + Self::reject_if_startup_only(&name)?; // We want to ensure that the network policy we're switching too actually exists. if NETWORK_POLICY.name.to_string().to_lowercase() == name.clone().to_lowercase() { self.validate_alter_system_network_policy(session, &value)?; @@ -4158,6 +4084,7 @@ impl Coordinator { plan::AlterSystemResetPlan { name }: plan::AlterSystemResetPlan, ) -> Result { self.is_user_allowed_to_alter_system(session, Some(&name))?; + Self::reject_if_startup_only(&name)?; let op = catalog::Op::ResetSystemConfiguration { name: name.clone() }; self.catalog_transact(Some(session), vec![op]).await?; session.add_notice(AdapterNotice::VarDefaultUpdated { @@ -4183,6 +4110,30 @@ impl Coordinator { Ok(ExecuteResponse::AlteredSystemConfiguration) } + /// Rejects `ALTER SYSTEM SET` / `RESET` for system parameters whose value + /// is sampled once at `environmentd` startup and cannot be changed + /// dynamically. + /// + /// Mutating these at runtime would update the catalog without affecting + /// the running process, leaving operators (and us, in tests like + /// `parallel-workload`) with the false impression that the change took + /// effect. For switches that gate fundamentally different code paths — + /// e.g. `enable_adapter_frontend_occ_read_then_write`, where the + /// lock-based and OCC paths cannot safely run concurrently within one + /// process — that confusion is dangerous, so we refuse the operation + /// outright. + fn reject_if_startup_only(name: &str) -> Result<(), AdapterError> { + let startup_only: &[&str] = &[FRONTEND_READ_THEN_WRITE.name()]; + if startup_only.iter().any(|n| n.eq_ignore_ascii_case(name)) { + return Err(AdapterError::Unstructured(anyhow!( + "{name} is read once at environmentd startup and cannot be \ + changed at runtime; set it via system_parameter_default and \ + restart environmentd to change it" + ))); + } + Ok(()) + } + // TODO(jkosh44) Move this into rbac.rs once RBAC is always on. fn is_user_allowed_to_alter_system( &self, diff --git a/src/adapter/src/coord/sequencer/inner/subscribe.rs b/src/adapter/src/coord/sequencer/inner/subscribe.rs index 505396b74e9a7..08cf1ce5bb85e 100644 --- a/src/adapter/src/coord/sequencer/inner/subscribe.rs +++ b/src/adapter/src/coord/sequencer/inner/subscribe.rs @@ -505,6 +505,7 @@ impl Coordinator { depends_on: dependency_ids, start_time: self.now(), output: plan.output, + internal: false, }; active_subscribe.initialize(); diff --git a/src/adapter/src/coord/sql.rs b/src/adapter/src/coord/sql.rs index 6ac546fb4e7f4..376d2527147df 100644 --- a/src/adapter/src/coord/sql.rs +++ b/src/adapter/src/coord/sql.rs @@ -213,7 +213,7 @@ impl Coordinator { /// Clears coordinator state for a connection. pub(crate) async fn clear_connection(&mut self, conn_id: &ConnectionId) { - self.staged_cancellation.remove(conn_id); + self.connection_cancel_watches.remove(conn_id); self.retire_compute_sinks_for_conn(conn_id, ActiveComputeSinkRetireReason::Finished) .await; self.retire_cluster_reconfigurations_for_conn(conn_id).await; @@ -259,7 +259,11 @@ impl Coordinator { .drop_sinks .insert(id); - let ret_fut = match &active_sink { + let ret_fut: BuiltinTableAppendNotify = match &active_sink { + // Internal subscribes skip the builtin table update. + ActiveComputeSink::Subscribe(active_subscribe) if active_subscribe.internal => { + Box::pin(std::future::ready(())) + } ActiveComputeSink::Subscribe(active_subscribe) => { let update = self.catalog() @@ -308,18 +312,21 @@ impl Coordinator { match &sink { ActiveComputeSink::Subscribe(active_subscribe) => { - let update = self.catalog().state().pack_subscribe_update( - id, - active_subscribe, - Diff::MINUS_ONE, - ); - let update = self.catalog().state().resolve_builtin_table_update(update); - self.builtin_table_update().blocking(vec![update]).await; + // Skip builtin table update for internal subscribes + if !active_subscribe.internal { + let update = self.catalog().state().pack_subscribe_update( + id, + active_subscribe, + Diff::MINUS_ONE, + ); + let update = self.catalog().state().resolve_builtin_table_update(update); + self.builtin_table_update().blocking(vec![update]).await; - self.metrics - .active_subscribes - .with_label_values(&[session_type]) - .dec(); + self.metrics + .active_subscribes + .with_label_values(&[session_type]) + .dec(); + } } ActiveComputeSink::CopyTo(_) => { self.metrics diff --git a/src/adapter/src/frontend_peek.rs b/src/adapter/src/frontend_peek.rs index 6d5f7c5b8ac4a..02e414666b88e 100644 --- a/src/adapter/src/frontend_peek.rs +++ b/src/adapter/src/frontend_peek.rs @@ -196,71 +196,60 @@ impl PeekClient { // Set up statement logging, and log the beginning of execution. // (But only if we're not executing in the context of another statement.) - let statement_logging_id = if outer_ctx_extra.is_none() { - // This is a new statement, so begin statement logging - let result = self.statement_logging_frontend.begin_statement_execution( - session, - ¶ms, - &logging, - catalog.system_config(), - lifecycle_timestamps, - ); - - if let Some((logging_id, began_execution, mseh_update, prepared_statement)) = result { - self.log_began_execution(began_execution, mseh_update, prepared_statement); - Some(logging_id) - } else { - None - } - } else { - // We're executing in the context of another statement (e.g., FETCH), - // so extract the statement logging ID from the outer context if present. - // We take ownership and retire the outer context here. The end of execution will be - // logged in one of the following ways: - // - At the end of this function, if the execution is finished by then. - // - Later by the Coordinator, either due to RegisterFrontendPeek or ExecuteSlowPathPeek. - outer_ctx_extra - .take() - .and_then(|guard| guard.defuse().retire()) - }; + let logging_guard = self.begin_statement_logging( + session, + ¶ms, + &logging, + &catalog, + lifecycle_timestamps, + outer_ctx_extra, + ); + let statement_logging_id = logging_guard.id(); + // Streaming peek/subscribe/slow-path/copy-to responses hand off the + // end-execution event to the coordinator (via + // `handle_peek_notification` / `cancel_pending_peeks`), so letting + // the RAII guard's `Drop` also emit `Aborted` on mid-flight drop + // would double-end and panic at `end_statement_execution`. Defuse + // and manage the lifecycle explicitly below: streaming arms skip + // emitting; non-streaming arms emit via `log_ended_execution`. + logging_guard.defuse(); let result = self .try_frontend_peek_inner(session, catalog, stmt, params, statement_logging_id) .await; - // Log the end of execution if we are logging this statement and execution has already - // ended. + // Log the end of execution if we are logging this statement and + // execution has already ended. if let Some(logging_id) = statement_logging_id { let reason = match &result { - // Streaming results are handled asynchronously by the coordinator + // Streaming results are handled asynchronously by the + // coordinator — it will log the end via + // `handle_peek_notification`, so skip emitting here. Ok(Some( ExecuteResponse::SendingRowsStreaming { .. } | ExecuteResponse::Subscribing { .. }, )) => { - // Don't log here - the peek or subscribe is still executing. - // It will be logged when handle_peek_notification is called. return result; } - // COPY TO needs to check its inner response + // COPY TO wrapping a streaming response: same handoff. Ok(Some(resp @ ExecuteResponse::CopyTo { resp: inner, .. })) => { match inner.as_ref() { ExecuteResponse::SendingRowsStreaming { .. } | ExecuteResponse::Subscribing { .. } => { - // Don't log here - the peek or subscribe is still executing. - // It will be logged when handle_peek_notification is called. return result; } - // For non-streaming COPY TO responses, use the outer CopyTo for conversion + // For non-streaming COPY TO responses, use the outer + // CopyTo for conversion. _ => resp.into(), } } - // Bailout case, which should not happen + // Bailout case, which should not happen. Ok(None) => { soft_panic_or_log!( "Bailed out from `try_frontend_peek_inner` after we already logged the beginning of statement execution." ); - // This statement will be handled by the old peek sequencing, which will do its - // own statement logging from the beginning. So, let's close out this one. + // The old peek sequencing would start its own statement + // logging from scratch; close out this one as errored. self.log_ended_execution( logging_id, StatementEndedExecutionReason::Errored { @@ -270,10 +259,10 @@ impl PeekClient { ); return result; } - // All other success responses - use the From implementation - // TODO(peek-seq): After we delete the old peek sequencing, we'll be able to adjust - // the From implementation to do exactly what we need in the frontend peek - // sequencing, so that the above special cases won't be needed. + // All other success responses — use the `From` implementation. + // TODO(peek-seq): After we delete the old peek sequencing, we + // can adjust the `From` impl to do exactly what we need here, + // so the special cases above won't be needed. Ok(Some(resp)) => resp.into(), Err(e) => StatementEndedExecutionReason::Errored { error: e.to_string(), diff --git a/src/adapter/src/frontend_read_then_write.rs b/src/adapter/src/frontend_read_then_write.rs new file mode 100644 index 0000000000000..bd6477a6d754e --- /dev/null +++ b/src/adapter/src/frontend_read_then_write.rs @@ -0,0 +1,1217 @@ +// 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. + +//! Frontend sequencing for read-then-write operations. +//! +//! This module implements INSERT [...] SELECT FROM [...], DELETE and UPDATE +//! operations using a subscribe with optimistic concurrency control (OCC), +//! sequenced from the session task rather than the Coordinator. This reduces +//! coordinator bottlenecking. +//! +//! The approach is: +//! 1. Validate and optimize MIR locally +//! 2. Determine timestamp via coordinator +//! 3. Optimize LIR locally +//! 4. Acquire OCC semaphore +//! 5. Create subscribe via Coordinator Command +//! 6. Run OCC loop (receive diffs, attempt write, retry on conflict) +//! 7. Return result +//! +//! ## Rollout note +//! +//! The `FRONTEND_READ_THEN_WRITE` dyncfg is read once at process startup and +//! fixed for the lifetime of the `environmentd` process. This avoids a +//! mixed-mode window where both the lock-based coordinator path and this OCC +//! path are active concurrently — the coordinator path acquires write locks to +//! prevent concurrent writes between its read and write phases, but this OCC +//! path does not use write locks, so concurrent operation of both paths could +//! allow an OCC write to slip between a coordinator-path reader's read and +//! write. + +use std::collections::BTreeMap; +use std::collections::BTreeSet; +use std::num::{NonZeroI64, NonZeroUsize}; +use std::sync::Arc; +use std::time::{Duration, Instant}; + +use differential_dataflow::consolidation; +use itertools::Itertools; +use mz_catalog::memory::error::ErrorKind; +use mz_cluster_client::ReplicaId; +use mz_compute_types::ComputeInstanceId; +use mz_expr::{CollectionPlan, Id, LocalId, MirRelationExpr, MirScalarExpr}; +use mz_ore::cast::CastFrom; +use mz_ore::soft_panic_or_log; +use mz_repr::optimize::OverrideFrom; +use mz_repr::{ + CatalogItemId, Diff, GlobalId, IntoRowIterator, RelationDesc, Row, RowArena, Timestamp, +}; +use mz_sql::catalog::CatalogError; +use mz_sql::plan::{self, MutationKind, Params, QueryWhen}; +use mz_sql::session::metadata::SessionMetadata; +use qcell::QCell; +use timely::progress::Antichain; +use tokio::sync::mpsc; +use uuid::Uuid; + +use crate::catalog::Catalog; +use crate::command::{Command, ExecuteResponse}; +use crate::coord::appends::WriteResult; +use crate::coord::read_then_write::validate_read_then_write_dependencies; +use crate::coord::timestamp_selection::TimestampProvider; +use crate::coord::{Coordinator, ExecuteContextGuard, TargetCluster}; +use crate::error::AdapterError; +use crate::optimize::Optimize; +use crate::optimize::dataflows::{ComputeInstanceSnapshot, EvalTime, ExprPrep, ExprPrepOneShot}; +use crate::session::{LifecycleTimestamps, Session, TransactionOps}; +use crate::statement_logging::{ + PreparedStatementLoggingInfo, StatementLifecycleEvent, StatementLoggingId, +}; +use crate::{PeekClient, PeekResponseUnary, TimelineContext, optimize}; + +/// A handle to an internal subscribe (not visible in introspection collections +/// like `mz_subscriptions`). A `Drop` impl ensures the subscribe's dataflow is +/// cleaned up when dropped. +pub(crate) struct SubscribeHandle { + rx: mpsc::UnboundedReceiver, + sink_id: GlobalId, + /// Wrapped in `Option` so we can move it out in `Drop`. + client: Option, +} + +impl SubscribeHandle { + /// Receive the next message from the subscribe, waiting if necessary. + pub async fn recv(&mut self) -> Option { + self.rx.recv().await + } + + /// Try to receive a message without waiting. + pub fn try_recv(&mut self) -> Result { + self.rx.try_recv() + } +} + +impl Drop for SubscribeHandle { + fn drop(&mut self) { + if let Some(client) = self.client.take() { + // Fire-and-forget: if the coordinator is gone, the subscribe will + // be cleaned up when the process exits anyway. + client.send(Command::DropInternalSubscribe { + sink_id: self.sink_id, + }); + } + } +} + +impl PeekClient { + /// Execute a read-then-write operation using frontend sequencing. + /// + /// Called by session code when the frontend_read_then_write dyncfg is + /// enabled. + pub(crate) async fn frontend_read_then_write( + &mut self, + session: &mut Session, + plan: plan::ReadThenWritePlan, + target_cluster: TargetCluster, + params: &Params, + logging: &Arc>, + lifecycle_timestamps: Option, + outer_ctx_extra: &mut Option, + ) -> Result { + let catalog = self.catalog_snapshot("frontend_read_then_write").await; + + // The guard's `Drop` impl emits `Aborted` if the inner future is + // dropped mid-flight, so the end-execution event is never skipped. + let logging_guard = self.begin_statement_logging( + session, + params, + logging, + &catalog, + lifecycle_timestamps, + outer_ctx_extra, + ); + + let result = self + .frontend_read_then_write_inner( + session, + plan, + target_cluster, + &catalog, + logging_guard.id(), + ) + .await; + + logging_guard.retire_with_result(&result); + + result + } + + /// Separated from the outer function so the statement-logging guard always + /// retires on the same return path. + async fn frontend_read_then_write_inner( + &mut self, + session: &mut Session, + mut plan: plan::ReadThenWritePlan, + target_cluster: TargetCluster, + catalog: &Arc, + statement_logging_id: Option, + ) -> Result { + let validation_result = + self.validate_read_then_write(catalog, session, &plan, target_cluster)?; + + let ValidationResult { + cluster_id, + replica_id, + timeline, + depends_on, + table_desc, + } = validation_result; + + if let Some(logging_id) = statement_logging_id { + self.log_set_cluster(logging_id, cluster_id); + } + + // Read-then-write is rejected in explicit transaction blocks (checked + // in SessionClient::try_frontend_read_then_write), so we're always in + // an implicit (autocommit) transaction here. The actual data is written + // via the coordinator's group commit path, bypassing session transaction + // ops. The empty Writes(vec![]) just marks this as a write transaction + // in the session state machine so auto-commit handles it correctly. + // This is safe because there's no ROLLBACK opportunity in an implicit + // transaction. + debug_assert!( + session.transaction().is_implicit(), + "read-then-write should be rejected in explicit transactions" + ); + session.add_transaction_ops(TransactionOps::Writes(vec![]))?; + + // Prepare expressions (resolve unmaterializable functions like + // current_user()) + let style = ExprPrepOneShot { + logical_time: EvalTime::NotAvailable, // We already errored out on mz_now above. + session, + catalog_state: catalog.state(), + }; + for expr in plan + .assignments + .values_mut() + .chain(plan.returning.iter_mut()) + { + style.prep_scalar_expr(expr)?; + } + + let (optimizer, global_mir_plan) = + self.optimize_mir_read_then_write(catalog, session, &plan, cluster_id)?; + + // Acquire the OCC semaphore permit *before* acquiring read holds in + // `frontend_determine_timestamp`. Under contention, waiters will + // otherwise sit on read holds on the RTW's read dependencies for the + // entire time they are queued, pinning compaction on those + // collections. Waiting on the permit first keeps queued operations + // hold-free; once we have a permit we proceed to acquire the read + // holds needed for the rest of the operation. + // + // The semaphore is owned by the coordinator and outlives every + // session task, so `acquire_owned` cannot return `Err` in practice. + let permit = Arc::clone(&self.occ_write_semaphore) + .acquire_owned() + .await + .expect("semaphore is never closed during coordinator lifetime"); + + // Determine timestamp and acquire read holds. + let oracle_read_ts = self.oracle_read_ts(&timeline).await?; + let bundle = global_mir_plan.id_bundle(cluster_id); + let (determination, read_holds) = self + .frontend_determine_timestamp( + session, + &bundle, + &QueryWhen::FreshestTableWrite, + cluster_id, + &timeline, + oracle_read_ts, + None, + ) + .await?; + + let as_of = determination.timestamp_context.timestamp_or_default(); + + let global_lir_plan = + self.optimize_lir_read_then_write(optimizer, global_mir_plan, as_of)?; + + // Log optimization finished + if let Some(logging_id) = statement_logging_id { + self.log_lifecycle_event(logging_id, StatementLifecycleEvent::OptimizationFinished); + } + + let sink_id = global_lir_plan.sink_id(); + let target_id = plan.id; + let kind = plan.kind.clone(); + let returning = plan.returning.clone(); + + let (df_desc, _df_meta) = global_lir_plan.unapply(); + + let arity = df_desc + .sink_exports + .values() + .next() + .expect("has sink") + .from_desc + .arity(); + + let conn_id = session.conn_id().clone(); + let session_uuid = session.uuid(); + let start_time = (self.statement_logging_frontend.now)(); + let max_result_size = catalog.system_config().max_result_size(); + let max_occ_retries = usize::cast_from(catalog.system_config().max_occ_retries()); + let statement_timeout = *session.vars().statement_timeout(); + + // Linearize the read BEFORE subscribing or writing: block until + // the oracle for this query's timeline has advanced to `as_of`. + // + // The up-front ordering is load-bearing. If `as_of` is in the far + // future (e.g. reading from a `REFRESH AT` MV with a far-future + // since), submitting a write at `chosen_ts >= as_of` would have + // the group commit bump the oracle to that far-future value, + // stalling every subsequent write on the `EpochMilliseconds` + // timeline until then. By waiting here, a pathological RTW hits + // `statement_timeout` and returns without ever touching the + // oracle. + self.ensure_read_linearized(&timeline, as_of).await?; + + let subscribe_handle = self + .create_internal_subscribe( + Box::new(df_desc), + cluster_id, + replica_id, + depends_on.clone(), + as_of, + arity, + sink_id, + conn_id.clone(), + session_uuid, + start_time, + read_holds, + ) + .await?; + + let (retry_count, result) = self + .run_occ_loop( + subscribe_handle, + target_id, + kind, + returning, + max_result_size, + max_occ_retries, + table_desc, + statement_timeout, + conn_id, + statement_logging_id, + as_of, + ) + .await; + + self.coordinator_client() + .metrics() + .occ_retry_count + .observe(f64::from(u32::try_from(retry_count).unwrap_or(u32::MAX))); + + // Release the OCC permit only after the OCC loop has fully completed + // (success, failure, or timeout). Holding it for the entire operation + // is what bounds concurrency; an early drop would let a waiter start + // its subscribe while we are still consolidating diffs and retrying. + drop(permit); + + let result = result?; + + Ok(result) + } + + /// Validate a read-then-write operation. + fn validate_read_then_write( + &self, + catalog: &Arc, + session: &Session, + plan: &plan::ReadThenWritePlan, + target_cluster: TargetCluster, + ) -> Result { + // Disallow mz_now in any position because read time and write time differ. + let contains_temporal = plan.selection.contains_temporal()? + || plan.assignments.values().any(|e| e.contains_temporal()) + || plan.returning.iter().any(|e| e.contains_temporal()); + if contains_temporal { + return Err(AdapterError::Unsupported( + "calls to mz_now in write statements", + )); + } + + // Validate read dependencies. The plan was built against an earlier + // catalog snapshot; an item it depends on may have been dropped by + // concurrent DDL before we got here. + for gid in plan.selection.depends_on() { + let item_id = catalog.try_resolve_item_id(&gid).ok_or_else(|| { + AdapterError::Catalog(mz_catalog::memory::error::Error { + kind: ErrorKind::Sql(CatalogError::UnknownItem(gid.to_string())), + }) + })?; + validate_read_then_write_dependencies(catalog, &item_id)?; + } + + let cluster = catalog.resolve_target_cluster(target_cluster, session)?; + let cluster_id = cluster.id; + + if cluster.replicas().next().is_none() { + return Err(AdapterError::NoClusterReplicasAvailable { + name: cluster.name.clone(), + is_managed: cluster.is_managed(), + }); + } + + let replica_id = session + .vars() + .cluster_replica() + .map(|name| { + cluster + .replica_id(name) + .ok_or(AdapterError::UnknownClusterReplica { + cluster_name: cluster.name.clone(), + replica_name: name.to_string(), + }) + }) + .transpose()?; + + let depends_on = plan.selection.depends_on(); + let timeline = catalog.validate_timeline_context(depends_on.iter().copied())?; + + // Get the table descriptor for constraint validation. The plan's + // target table may have been dropped by concurrent DDL between + // planning and here, so tolerate a missing entry. + let table_desc = match catalog.try_get_entry(&plan.id) { + Some(entry) => entry + .relation_desc_latest() + .expect("table has desc") + .into_owned(), + None => { + return Err(AdapterError::Catalog(mz_catalog::memory::error::Error { + kind: ErrorKind::Sql(CatalogError::UnknownItem(plan.id.to_string())), + })); + } + }; + + Ok(ValidationResult { + cluster_id, + replica_id, + timeline, + depends_on, + table_desc, + }) + } + + /// Optimize MIR for a read-then-write operation. + fn optimize_mir_read_then_write( + &self, + catalog: &Arc, + session: &dyn SessionMetadata, + plan: &plan::ReadThenWritePlan, + cluster_id: ComputeInstanceId, + ) -> Result< + ( + optimize::subscribe::Optimizer, + optimize::subscribe::GlobalMirPlan, + ), + AdapterError, + > { + // `finishing` is unused: the OCC path emits raw diffs and + // `apply_mutation_to_mir` handles update projection. + let plan::ReadThenWritePlan { + id: _, + selection, + finishing: _, + assignments, + kind, + returning: _, + } = plan; + + let expr = selection.clone().lower(catalog.system_config(), None)?; + let mut expr = apply_mutation_to_mir(expr, kind, assignments); + + // Resolve unmaterializable functions (now(), current_user, ...) before + // the subscribe optimizer sees them: it uses `ExprPrepMaintained`, + // which rejects them, but our subscribe is a one-shot read so we can + // resolve them to constants. `mz_now()` is rejected upstream by + // `validate_read_then_write`. + let style = ExprPrepOneShot { + logical_time: EvalTime::NotAvailable, + session, + catalog_state: catalog.state(), + }; + expr.try_visit_scalars_mut(&mut |s| style.prep_scalar_expr(s))?; + + let compute_instance = ComputeInstanceSnapshot::new_without_collections(cluster_id); + let (_, view_id) = self.transient_id_gen.allocate_id(); + let (_, sink_id) = self.transient_id_gen.allocate_id(); + let debug_name = format!("frontend-read-then-write-subscribe-{}", sink_id); + let optimizer_config = optimize::OptimizerConfig::from(catalog.system_config()) + .override_from(&catalog.get_cluster(cluster_id).config.features()); + + let mut optimizer = optimize::subscribe::Optimizer::new( + Arc::::clone(catalog), + compute_instance, + view_id, + sink_id, + true, // with_snapshot + None, // up_to + debug_name, + optimizer_config, + self.optimizer_metrics.clone(), + ); + + let expr_typ = expr.typ(); + let sql_typ = mz_repr::SqlRelationType::from_repr(&expr_typ); + let column_names: Vec = (0..sql_typ.column_types.len()) + .map(|i| format!("column{}", i)) + .collect(); + let relation_desc = RelationDesc::new(sql_typ, column_names.iter().map(|s| s.as_str())); + + // MIR => MIR optimization (global). We use optimize_mir because the + // mutation has already been applied in MIR, so we bypass the normal + // SubscribePlan path which expects HIR. + let global_mir_plan = optimizer.optimize_mir(expr, relation_desc)?; + + Ok((optimizer, global_mir_plan)) + } + + /// Optimize LIR for a read-then-write operation. + fn optimize_lir_read_then_write( + &self, + mut optimizer: optimize::subscribe::Optimizer, + global_mir_plan: optimize::subscribe::GlobalMirPlan, + as_of: Timestamp, + ) -> Result { + let global_mir_plan = global_mir_plan.resolve(Antichain::from_elem(as_of)); + let global_lir_plan = optimizer.optimize(global_mir_plan)?; + Ok(global_lir_plan) + } + + /// Get the oracle read timestamp hint for the timeline of this query. + async fn oracle_read_ts( + &mut self, + timeline: &TimelineContext, + ) -> Result, AdapterError> { + // See `ensure_read_linearized` for why `get_timeline` is the right + // function here: the write target lives on `EpochMilliseconds`, so + // we want that oracle's read_ts as the hint for timestamp + // selection even when the read side is MV-only + // (`TimestampDependent`). + let timeline = ::get_timeline(timeline); + + match timeline { + Some(timeline) => { + let oracle = self.ensure_oracle(timeline).await?; + Ok(Some(oracle.read_ts().await)) + } + None => Ok(None), + } + } + + /// Block until the oracle for this query's timeline has advanced to + /// `as_of`. Returns immediately if it already has. + /// + /// This implements the strict-serializable read guarantee for RTW: + /// once this returns, any session observing the oracle sees a read + /// timestamp at least as large as `as_of`, so reads at `as_of` (and + /// writes derived from them) cannot appear to "go backwards" relative + /// to subsequent queries. + async fn ensure_read_linearized( + &mut self, + timeline: &TimelineContext, + as_of: Timestamp, + ) -> Result<(), AdapterError> { + // Pick the oracle this RTW operates against. `timeline` is derived + // from the read side (`plan.selection.depends_on()`), so an + // MV-only read produces `TimestampDependent` — an MV itself + // doesn't pin the query to any source timeline. The write target, + // however, is always a Table living on `EpochMilliseconds`, and + // future readers of that table will consult the + // `EpochMilliseconds` oracle, so linearization must target + // `EpochMilliseconds` regardless of the read side. + // + // `get_timeline` encodes that defaulting (`TimestampDependent` → + // `Some(EpochMilliseconds)`). `TimelineContext::timeline()` + // answers a different question ("is there a source-forced + // timeline?") and would return `None` for MV-only reads, + // silently skipping linearization. + let tl = match ::get_timeline(timeline) { + Some(tl) => tl, + None => return Ok(()), + }; + + let oracle = self.ensure_oracle(tl).await?; + + loop { + let oracle_ts = oracle.read_ts().await; + if as_of <= oracle_ts { + return Ok(()); + } + + // Sleep for roughly the difference between as_of and the current + // oracle timestamp. Since timestamps are epoch milliseconds, the + // difference is the approximate wall-clock time we need to wait. + // Cap at 1s to avoid very long sleeps if clocks are skewed, + // matching the cap in `message_linearize_reads`. + let wait_ms = u64::from(as_of.saturating_sub(oracle_ts)); + let wait = Duration::from_millis(wait_ms).min(Duration::from_secs(1)); + tokio::time::sleep(wait).await; + } + } + + /// Creates an internal subscribe that does not appear in introspection + /// tables. Returns a [`SubscribeHandle`] that ensures cleanup on drop. + async fn create_internal_subscribe( + &self, + df_desc: Box, + cluster_id: ComputeInstanceId, + replica_id: Option, + depends_on: BTreeSet, + as_of: Timestamp, + arity: usize, + sink_id: GlobalId, + conn_id: mz_adapter_types::connection::ConnectionId, + session_uuid: Uuid, + start_time: mz_ore::now::EpochMillis, + read_holds: crate::ReadHolds, + ) -> Result { + let rx: mpsc::UnboundedReceiver = self + .call_coordinator(|tx| Command::CreateInternalSubscribe { + df_desc, + cluster_id, + replica_id, + depends_on, + as_of, + arity, + sink_id, + conn_id, + session_uuid, + start_time, + read_holds, + tx, + }) + .await?; + + Ok(SubscribeHandle { + rx, + sink_id, + client: Some(self.coordinator_client().clone()), + }) + } + + /// Run the OCC loop: drain the subscribe at `as_of`, apply the + /// mutation, and submit the resulting diffs as a write. + /// + /// Semantically this is a SELECT at `as_of` followed by an INSERT. + /// Because we hold no write lock, a concurrent writer may bump the + /// target table's upper past our chosen write timestamp, in which + /// case the coordinator returns `WriteResult::TimestampPassed`; we + /// then wait for the subscribe to advance and retry, up to + /// `max_occ_retries` times. + /// + /// Read linearization is the caller's responsibility: `as_of` must + /// already be linearized (oracle read_ts >= `as_of`) on entry. See + /// `ensure_read_linearized` at the call site. + /// + /// Returns `(retry_count, result)` so the caller can record OCC retry + /// metrics regardless of whether the operation succeeded or failed. + async fn run_occ_loop( + &self, + mut subscribe_handle: SubscribeHandle, + target_id: CatalogItemId, + kind: MutationKind, + returning: Vec, + max_result_size: u64, + max_occ_retries: usize, + table_desc: RelationDesc, + statement_timeout: Duration, + conn_id: mz_adapter_types::connection::ConnectionId, + statement_logging_id: Option, + as_of: Timestamp, + ) -> (usize, Result) { + // Timeout of 0 is equivalent to "off", meaning we will wait "forever." + let effective_timeout = if statement_timeout == Duration::ZERO { + Duration::MAX + } else { + statement_timeout + }; + let start_time = Instant::now(); + + let mut state = OccState::new(); + + // Correctness invariant for retries: + // + // `all_diffs` accumulates *all* rows ever received from the subscribe, + // across retries. The subscribe emits a snapshot (at the as_of + // timestamp) followed by incremental updates. We consolidate on every + // progress message (flattening timestamps to MIN first), so after + // consolidation `all_diffs` always represents "what the query returns + // as of the latest progress timestamp" — old snapshot rows that were + // retracted by newer updates cancel out, and new rows appear. This is + // exactly the set of diffs we want to write. + // + // Consolidating on every progress also means the NoRowsMatched check + // works correctly across retries: if the consolidated result becomes + // logically empty (all diffs cancel out), `all_diffs` will be empty + // and we early-return without attempting a write. + let result = loop { + // Check for timeout + let remaining = effective_timeout.saturating_sub(start_time.elapsed()); + if remaining.is_zero() { + // Guard handles cleanup on drop. + break Err(AdapterError::StatementTimeout); + } + + let msg = match tokio::time::timeout(remaining, subscribe_handle.recv()).await { + Ok(Some(msg)) => msg, + Ok(None) => { + // Channel closed cleanly: the SELECT is constant (no + // table dependency). Submit the accumulated diffs as a + // blind write — the oracle picks the timestamp at group + // commit, so we just flatten to `Timestamp::MIN` for + // `consolidate_updates`. + state.consolidate(Timestamp::MIN); + if state.all_diffs.is_empty() { + break build_no_rows_response(&kind, &returning); + } + let success_response = + match self.build_success_response(&kind, &returning, &state.all_diffs) { + Ok(response) => response, + Err(e) => break Err(e), + }; + let diffs = state + .all_diffs + .iter() + .map(|(row, _ts, diff)| (row.clone(), *diff)) + .collect_vec(); + let result = self + .call_coordinator(|tx| Command::AttemptWrite { + conn_id: conn_id.clone(), + target_id, + diffs, + write_ts: None, + tx, + }) + .await; + match result { + WriteResult::Success { timestamp } => { + if let Some(id) = statement_logging_id { + self.log_set_timestamp(id, timestamp); + } + break Ok(success_response); + } + WriteResult::TimestampPassed { .. } => { + // Unreachable: blind writes use + // `UserWriteResponder::Internal`, which group + // commit never resolves to `TimestampPassed`. + soft_panic_or_log!( + "blind read-then-write unexpectedly got TimestampPassed" + ); + break Err(AdapterError::Internal( + "blind write unexpectedly got TimestampPassed".into(), + )); + } + WriteResult::Canceled => break Err(AdapterError::Canceled), + WriteResult::ReadOnly => break Err(AdapterError::ReadOnly), + } + } + Err(_) => { + // Timed out + break Err(AdapterError::StatementTimeout); + } + }; + + match process_message(msg, &mut state, as_of, max_result_size, &table_desc) { + ProcessResult::Continue { ready_to_write } => { + if !ready_to_write { + continue; + } + + // Drain pending messages before attempting write + let drain_err = loop { + match subscribe_handle.try_recv() { + Ok(msg) => { + match process_message( + msg, + &mut state, + as_of, + max_result_size, + &table_desc, + ) { + ProcessResult::Continue { .. } => {} + ProcessResult::NoRowsMatched => { + break Some(build_no_rows_response(&kind, &returning)); + } + ProcessResult::Error(e) => { + break Some(Err(e)); + } + } + } + Err(mpsc::error::TryRecvError::Empty) => break None, + // The subscribe can finish (coordinator drops the + // sender after `process_response` returns true) + // between our last recv() and this drain. This is + // benign — all buffered messages have already been + // consumed via the Ok(msg) arm above. + Err(mpsc::error::TryRecvError::Disconnected) => break None, + } + }; + if let Some(result) = drain_err { + break result; + } + + let write_ts = state + .current_upper + .expect("must have seen progress to be ready to write"); + + // Consolidate any rows received during the drain + // (the bulk was already consolidated on the last progress). + state.consolidate(write_ts); + + let success_response = + match self.build_success_response(&kind, &returning, &state.all_diffs) { + Ok(response) => response, + Err(e) => break Err(e), + }; + + // Submit write. + // + // perf: clones every row on each attempt. Under contention + // we retry up to `max_occ_retries` times (default 1000), + // so a large DELETE/UPDATE under heavy contention can do a + // lot of row-cloning work. If this shows up in profiles, + // consider storing `Arc` in `all_diffs` to make the + // per-attempt copy cheap. + let result = self + .call_coordinator(|tx| Command::AttemptWrite { + conn_id: conn_id.clone(), + target_id, + diffs: state + .all_diffs + .iter() + .map(|(row, _ts, diff)| (row.clone(), *diff)) + .collect_vec(), + write_ts: Some(write_ts), + tx, + }) + .await; + + match result { + WriteResult::Success { timestamp } => { + if let Some(id) = statement_logging_id { + self.log_set_timestamp(id, timestamp); + } + // N.B. subscribe_handle is dropped here, which + // fires off the cleanup message. + break Ok(success_response); + } + WriteResult::TimestampPassed { + current_write_ts, .. + } => { + // Do not advance `state.current_upper` (and + // therefore `write_ts`) from `current_write_ts`. + // The diffs in `all_diffs` are only known to be + // correct as of subscribe progress we have actually + // observed. Retrying at a newer oracle timestamp + // before subscribe progress catches up would risk + // applying stale diffs at the wrong timestamp. So + // on `TimestampPassed` we wait for the subscribe to + // progress and retry using that observed frontier. + state.retry_count += 1; + if state.retry_count >= max_occ_retries { + // High contention is a user-visible + // condition, not an internal invariant + // violation. Surface it as + // `Unstructured` so it doesn't trip + // internal-error alerts. + break Err(AdapterError::Unstructured(anyhow::anyhow!( + "read-then-write exceeded maximum retry attempts under contention", + ))); + } + tracing::debug!( + retry_count = state.retry_count, + write_ts = %write_ts, + current_write_ts = %current_write_ts, + "OCC write conflict, retrying" + ); + continue; + } + WriteResult::Canceled => break Err(AdapterError::Canceled), + WriteResult::ReadOnly => break Err(AdapterError::ReadOnly), + } + } + ProcessResult::NoRowsMatched => { + break build_no_rows_response(&kind, &returning); + } + ProcessResult::Error(e) => { + break Err(e); + } + } + }; + + (state.retry_count, result) + } + + /// Build the success response after a successful write. + fn build_success_response( + &self, + kind: &MutationKind, + returning: &[MirScalarExpr], + all_diffs: &[(Row, Timestamp, Diff)], + ) -> Result { + if returning.is_empty() { + // For UPDATE each changed row produces a retraction (-1) and an + // insertion (+1), so we divide by 2 below. + let row_count = all_diffs + .iter() + .map(|(_, _, diff)| diff.into_inner().unsigned_abs()) + .sum::(); + let row_count = + usize::try_from(row_count).expect("positive row count must fit in usize"); + + return Ok(match kind { + MutationKind::Delete => ExecuteResponse::Deleted(row_count), + MutationKind::Update => ExecuteResponse::Updated(row_count / 2), + MutationKind::Insert => ExecuteResponse::Inserted(row_count), + }); + } + + let mut returning_rows = Vec::new(); + let arena = RowArena::new(); + + for (row, _ts, diff) in all_diffs { + let include = match kind { + MutationKind::Delete => diff.is_negative(), + MutationKind::Update | MutationKind::Insert => diff.is_positive(), + }; + + if !include { + continue; + } + + let mut returning_row = Row::with_capacity(returning.len()); + let mut packer = returning_row.packer(); + let datums: Vec<_> = row.iter().collect(); + + for expr in returning { + match expr.eval(&datums, &arena) { + Ok(datum) => packer.push(datum), + Err(err) => return Err(err.into()), + } + } + + let multiplicity = NonZeroUsize::try_from( + NonZeroI64::try_from(diff.into_inner().abs()).expect("diff is non-zero"), + ) + .map_err(AdapterError::from)?; + + returning_rows.push((returning_row, multiplicity)); + } + + let rows: Vec = returning_rows + .into_iter() + .flat_map(|(row, count)| std::iter::repeat(row).take(count.get())) + .collect(); + Ok(ExecuteResponse::SendingRowsImmediate { + rows: Box::new(rows.into_row_iter()), + }) + } +} + +/// Result of validating a read-then-write operation. +struct ValidationResult { + cluster_id: ComputeInstanceId, + replica_id: Option, + timeline: TimelineContext, + depends_on: BTreeSet, + /// The table descriptor, used for constraint validation. + table_desc: RelationDesc, +} + +/// Accumulated state for the OCC loop in `run_occ_loop`. +struct OccState { + all_diffs: Vec<(Row, Timestamp, Diff)>, + current_upper: Option, + retry_count: usize, + byte_size: u64, +} + +impl OccState { + fn new() -> Self { + Self { + all_diffs: Vec::new(), + current_upper: None, + retry_count: 0, + byte_size: 0, + } + } + + /// Forward all diff timestamps to `target_ts` and consolidate. + /// + /// After consolidation, `all_diffs` represents the net state of the + /// query as of `target_ts`. Rows that were retracted by newer updates + /// cancel out, and `byte_size` is recomputed to reflect the + /// consolidated data. + fn consolidate(&mut self, target_ts: Timestamp) { + for (_, ts, _) in self.all_diffs.iter_mut() { + *ts = target_ts; + } + consolidation::consolidate_updates(&mut self.all_diffs); + self.byte_size = self + .all_diffs + .iter() + .map(|(row, _, _)| u64::cast_from(row.byte_len())) + .sum(); + } +} + +/// Result of processing a single subscribe message in the OCC loop. +enum ProcessResult { + Continue { ready_to_write: bool }, + NoRowsMatched, + Error(AdapterError), +} + +/// Process one subscribe message, updating `state` in place. +/// +/// Data rows are accumulated into `state.all_diffs` (with per-row constraint +/// and max-result-size checks). Progress messages trigger consolidation and +/// can promote the accumulated diffs to "ready to write". +fn process_message( + response: PeekResponseUnary, + state: &mut OccState, + as_of: Timestamp, + max_result_size: u64, + table_desc: &RelationDesc, +) -> ProcessResult { + match response { + PeekResponseUnary::Rows(mut rows) => { + let mut saw_progress = false; + + while let Some(row) = rows.next() { + let mut datums = row.iter(); + + // Extract mz_timestamp (SubscribeOutput::Diffs format: + // mz_timestamp, mz_progressed, mz_diff, ...data columns...). + // + // Format drift would mean we'd silently commit an incorrect + // write, so surface every shape mismatch as an internal + // error rather than panicking the process. + let Some(ts_datum) = datums.next() else { + return ProcessResult::Error(AdapterError::Internal( + "missing mz_timestamp in subscribe output".into(), + )); + }; + let ts = match ts_datum { + mz_repr::Datum::Numeric(n) => match n.0.try_into() { + Ok(ts_u64) => Timestamp::new(ts_u64), + Err(_) => { + return ProcessResult::Error(AdapterError::Internal(format!( + "mz_timestamp in subscribe output is not a valid u64: {n}" + ))); + } + }, + other => { + return ProcessResult::Error(AdapterError::Internal(format!( + "unexpected mz_timestamp datum: {other:?}" + ))); + } + }; + + // Extract mz_progressed + let Some(progressed_datum) = datums.next() else { + return ProcessResult::Error(AdapterError::Internal( + "missing mz_progressed in subscribe output".into(), + )); + }; + let is_progress = matches!(progressed_datum, mz_repr::Datum::True); + + if is_progress { + state.current_upper = Some(ts); + saw_progress = true; + + // Consolidate incrementally on each progress + // message. This keeps memory bounded by the + // consolidated size and makes the byte_size check + // below accurate (except for rows received between + // two progress messages, which is a small window). + state.consolidate(ts); + + // The very first progress message we receive is + // always at `as_of`, emitted synchronously by + // `ActiveSubscribe::initialize` *before* any data + // batch is processed. At that point `all_diffs` is + // empty by construction, regardless of whether the + // snapshot is actually empty, so we must not + // conclude `NoRowsMatched` from it. Progress + // messages emitted later from `process_response` + // are gated on `batch.upper > as_of`, so any + // progress with `ts > as_of` is past the initial + // one and an empty `all_diffs` then genuinely + // means no rows matched. See + // `src/adapter/src/active_compute_sink.rs` for + // the emission order. + if ts > as_of && state.all_diffs.is_empty() { + return ProcessResult::NoRowsMatched; + } + } else { + let Some(diff_datum) = datums.next() else { + return ProcessResult::Error(AdapterError::Internal( + "missing mz_diff in subscribe output".into(), + )); + }; + let diff = match diff_datum { + mz_repr::Datum::Int64(d) => Diff::from(d), + other => { + return ProcessResult::Error(AdapterError::Internal(format!( + "unexpected mz_diff datum while processing read-then-write: {other:?}" + ))); + } + }; + + let data_row = Row::pack(datums); + + // Validate constraints for rows being added (positive diff) + if diff.is_positive() { + for (idx, datum) in data_row.iter().enumerate() { + if let Err(e) = table_desc.constraints_met(idx, &datum) { + return ProcessResult::Error(e.into()); + } + } + } + + state.byte_size = state + .byte_size + .saturating_add(u64::cast_from(data_row.byte_len())); + if state.byte_size > max_result_size { + return ProcessResult::Error(AdapterError::ResultSize(format!( + "result exceeds max size of {}", + max_result_size + ))); + } + state.all_diffs.push((data_row, ts, diff)); + } + } + + // We're ready to write once we've seen a progress + // message and have accumulated any diffs. Data rows can + // only arrive *after* the initial progress at `as_of` + // (see the note in the progress branch), so a non-empty + // `all_diffs` here implies we're past the initial + // progress. + let ready_to_write = saw_progress && !state.all_diffs.is_empty(); + ProcessResult::Continue { ready_to_write } + } + PeekResponseUnary::Error(e) => { + ProcessResult::Error(AdapterError::Unstructured(anyhow::anyhow!(e))) + } + PeekResponseUnary::DependencyDropped(dep) => ProcessResult::Error( + AdapterError::Unstructured(anyhow::anyhow!(dep.query_terminated_error())), + ), + PeekResponseUnary::Canceled => ProcessResult::Error(AdapterError::Canceled), + } +} + +/// Build the response returned when no rows matched the selection. +fn build_no_rows_response( + kind: &MutationKind, + returning: &[MirScalarExpr], +) -> Result { + if !returning.is_empty() { + let rows: Vec = vec![]; + return Ok(ExecuteResponse::SendingRowsImmediate { + rows: Box::new(rows.into_row_iter()), + }); + } + Ok(match kind { + MutationKind::Delete => ExecuteResponse::Deleted(0), + MutationKind::Update => ExecuteResponse::Updated(0), + MutationKind::Insert => ExecuteResponse::Inserted(0), + }) +} + +/// Transform a MIR expression to produce the appropriate diffs for a mutation. +/// +/// - DELETE: Negates the expression to produce `(row, -1)` diffs +/// - UPDATE: Unions negated old rows with mapped new rows to produce both +/// `(old_row, -1)` and `(new_row, +1)` diffs +fn apply_mutation_to_mir( + expr: MirRelationExpr, + kind: &MutationKind, + assignments: &BTreeMap, +) -> MirRelationExpr { + match kind { + MutationKind::Delete => MirRelationExpr::Negate { + input: Box::new(expr), + }, + MutationKind::Update => { + let arity = expr.arity(); + + // Find a fresh LocalId that won't conflict with any in the expression. + // + // Invariant: `Let` and `LetRec` are the only MIR nodes that *bind* + // LocalIds; `Get` references them but does not introduce new ones. + // So scanning just those two node kinds and picking `max + 1` is + // guaranteed to produce an id unused by the subtree. + let mut max_id = 0_u64; + expr.visit_pre(|e| match e { + MirRelationExpr::Let { id, .. } => { + max_id = std::cmp::max(max_id, id.into()); + } + MirRelationExpr::LetRec { ids, .. } => { + for id in ids { + max_id = std::cmp::max(max_id, id.into()); + } + } + _ => {} + }); + let binding_id = LocalId::new(max_id + 1); + + let get_binding = MirRelationExpr::Get { + id: Id::Local(binding_id), + typ: expr.typ(), + access_strategy: mz_expr::AccessStrategy::UnknownOrLocal, + }; + + // Build map expressions + let map_scalars: Vec = (0..arity) + .map(|i| { + assignments + .get(&i) + .cloned() + .unwrap_or_else(|| MirScalarExpr::column(i)) + }) + .collect(); + + let new_rows = get_binding + .clone() + .map(map_scalars) + .project((arity..2 * arity).collect()); + + let old_rows = MirRelationExpr::Negate { + input: Box::new(get_binding), + }; + + let body = new_rows.union(old_rows); + + MirRelationExpr::Let { + id: binding_id, + value: Box::new(expr), + body: Box::new(body), + } + } + // INSERT: rows pass through unchanged; the subscribe emits them with diff +1. + MutationKind::Insert => expr, + } +} diff --git a/src/adapter/src/lib.rs b/src/adapter/src/lib.rs index da031f0dcff39..55ec1e22c7573 100644 --- a/src/adapter/src/lib.rs +++ b/src/adapter/src/lib.rs @@ -44,6 +44,7 @@ mod coord; mod error; mod explain; mod frontend_peek; +mod frontend_read_then_write; mod notice; mod optimize; mod util; diff --git a/src/adapter/src/metrics.rs b/src/adapter/src/metrics.rs index c6ea0fa94e39d..4ccbe2dedcb1d 100644 --- a/src/adapter/src/metrics.rs +++ b/src/adapter/src/metrics.rs @@ -54,6 +54,7 @@ pub struct Metrics { pub apply_catalog_implications_seconds: Histogram, pub group_commit_catalog_upper_seconds: Histogram, pub group_commit_table_advancement_seconds: Histogram, + pub occ_retry_count: Histogram, } impl Metrics { @@ -246,6 +247,13 @@ impl Metrics { name: "mz_group_commit_table_advancement_seconds", help: "The time it takes to iterate over all catalog entries to find tables during group commit.", buckets: histogram_seconds_buckets(0.001, 32.0), + )), + occ_retry_count: registry.register(metric!( + name: "mz_occ_read_then_write_retry_count", + help: "Number of OCC retries per read-then-write operation.", + buckets: vec![ + 0., 1., 2., 3., 5., 10., 25., 50., 100., 200., 300., 500., 750., 1000., + ], )) } } diff --git a/src/adapter/src/optimize.rs b/src/adapter/src/optimize.rs index 26c2321ed538c..2a3f8249bac9c 100644 --- a/src/adapter/src/optimize.rs +++ b/src/adapter/src/optimize.rs @@ -85,9 +85,8 @@ use mz_transform::{MaybeShouldPanic, TransformCtx, TransformError}; /// A type for a [`DataflowDescription`] backed by `Mir~` plans. Used internally /// by the optimizer implementations. type MirDataflowDescription = DataflowDescription; -/// A type for a [`DataflowDescription`] backed by `Lir~` plans. Used internally -/// by the optimizer implementations. -type LirDataflowDescription = DataflowDescription; +/// A type for a [`DataflowDescription`] backed by `Lir~` plans. +pub type LirDataflowDescription = DataflowDescription; // Core API // -------- diff --git a/src/adapter/src/optimize/subscribe.rs b/src/adapter/src/optimize/subscribe.rs index 618e3a533cc68..613153192fa8c 100644 --- a/src/adapter/src/optimize/subscribe.rs +++ b/src/adapter/src/optimize/subscribe.rs @@ -17,8 +17,9 @@ use differential_dataflow::lattice::Lattice; use mz_compute_types::ComputeInstanceId; use mz_compute_types::plan::Plan; use mz_compute_types::sinks::{ComputeSinkConnection, ComputeSinkDesc, SubscribeSinkConnection}; +use mz_expr::MirRelationExpr; use mz_ore::soft_assert_or_log; -use mz_repr::{GlobalId, Timestamp}; +use mz_repr::{GlobalId, RelationDesc, Timestamp}; use mz_sql::optimizer_metrics::OptimizerMetrics; use mz_sql::plan::{HirToMirConfig, SubscribeFrom, SubscribePlan}; use mz_transform::TransformCtx; @@ -114,6 +115,86 @@ impl Optimizer { pub fn sink_id(&self) -> GlobalId { self.sink_id } + + /// Optimize a subscribe dataflow starting from a pre-lowered MIR + /// expression. Used by the frontend read-then-write path which applies + /// mutation transformations in MIR before optimization. + pub fn optimize_mir( + &mut self, + expr: MirRelationExpr, + desc: RelationDesc, + ) -> Result, OptimizerError> { + let time = Instant::now(); + + let mut df_builder = { + let compute = self.compute_instance.clone(); + DataflowBuilder::new(&*self.catalog, compute).with_config(&self.config) + }; + let mut df_desc = MirDataflowDescription::new(self.debug_name.clone()); + let mut df_meta = DataflowMetainfo::default(); + + // MIR ⇒ MIR optimization (local) + let mut transform_ctx = TransformCtx::local( + &self.config.features, + &self.typecheck_ctx, + &mut df_meta, + Some(&mut self.metrics), + Some(self.view_id), + ); + let expr = optimize_mir_local(expr, &mut transform_ctx)?; + + df_builder.import_view_into_dataflow( + &self.view_id, + &expr, + &mut df_desc, + &self.config.features, + )?; + df_builder.maybe_reoptimize_imported_views(&mut df_desc, &self.config)?; + + let sink_description = ComputeSinkDesc { + from: self.view_id, + from_desc: desc, + connection: ComputeSinkConnection::Subscribe(SubscribeSinkConnection { + // Read-then-write subscribes use raw diffs. + output: vec![], + }), + with_snapshot: self.with_snapshot, + up_to: self.up_to.map(Antichain::from_elem).unwrap_or_default(), + non_null_assertions: vec![], + refresh_schedule: None, + }; + df_desc.export_sink(self.sink_id, sink_description); + + // Prepare expressions in the assembled dataflow. + let style = ExprPrepMaintained; + df_desc.visit_children( + |r| style.prep_relation_expr(r), + |s| style.prep_scalar_expr(s), + )?; + + // Construct TransformCtx for global optimization. + let mut transform_ctx = TransformCtx::global( + &df_builder, + &mz_transform::EmptyStatisticsOracle, + &self.config.features, + &self.typecheck_ctx, + &mut df_meta, + Some(&mut self.metrics), + ); + mz_transform::optimize_dataflow(&mut df_desc, &mut transform_ctx, false)?; + + if self.config.mode == OptimizeMode::Explain { + trace_plan!(at: "global", &df_meta.used_indexes(&df_desc)); + } + + self.duration += time.elapsed(); + + Ok(GlobalMirPlan { + df_desc, + df_meta, + phantom: PhantomData::, + }) + } } /// The (sealed intermediate) result after: diff --git a/src/adapter/src/optimize/view.rs b/src/adapter/src/optimize/view.rs index a5f92b50f98cd..bc0a770e3f485 100644 --- a/src/adapter/src/optimize/view.rs +++ b/src/adapter/src/optimize/view.rs @@ -66,6 +66,23 @@ impl Optimizer { } impl Optimizer { + /// Creates an optimizer instance that takes an [`ExprPrep`] to handle + /// unmaterializable functions while preserving the usual constant-folding + /// size limit. + pub fn new_with_prep( + config: OptimizerConfig, + metrics: Option, + expr_prep_style: S, + ) -> Optimizer { + Self { + typecheck_ctx: empty_typechecking_context(), + config, + metrics, + expr_prep_style, + fold_constants_limit: true, + } + } + /// Creates an optimizer instance that takes an [`ExprPrep`] to handle /// unmaterializable functions. Additionally, this instance calls constant /// folding without a size limit. diff --git a/src/adapter/src/peek_client.rs b/src/adapter/src/peek_client.rs index f489862a22d14..66aa58c0978ed 100644 --- a/src/adapter/src/peek_client.rs +++ b/src/adapter/src/peek_client.rs @@ -24,22 +24,24 @@ use mz_repr::Timestamp; use mz_repr::global_id::TransientIdGen; use mz_repr::{RelationDesc, Row}; use mz_sql::optimizer_metrics::OptimizerMetrics; +use mz_sql::plan::Params; use mz_storage_types::sources::Timeline; use mz_timestamp_oracle::TimestampOracle; use prometheus::Histogram; +use qcell::QCell; use thiserror::Error; use timely::progress::Antichain; -use tokio::sync::oneshot; +use tokio::sync::{Semaphore, oneshot}; use uuid::Uuid; use crate::catalog::Catalog; -use crate::command::{CatalogSnapshot, Command}; -use crate::coord::Coordinator; +use crate::command::{CatalogSnapshot, Command, ExecuteResponse}; use crate::coord::peek::FastPathPlan; -use crate::statement_logging::WatchSetCreation; +use crate::coord::{Coordinator, ExecuteContextGuard}; +use crate::session::{LifecycleTimestamps, Session}; use crate::statement_logging::{ - FrontendStatementLoggingEvent, PreparedStatementEvent, StatementLoggingFrontend, - StatementLoggingId, + FrontendStatementLoggingEvent, PreparedStatementEvent, PreparedStatementLoggingInfo, + StatementLoggingFrontend, StatementLoggingId, WatchSetCreation, }; use crate::{AdapterError, Client, CollectionIdBundle, ReadHolds, statement_logging}; @@ -66,6 +68,12 @@ pub struct PeekClient { persist_client: PersistClient, /// Statement logging state for frontend peek sequencing. pub statement_logging_frontend: StatementLoggingFrontend, + /// Semaphore for limiting concurrent OCC (optimistic concurrency control) write operations. + pub occ_write_semaphore: Arc, + /// Whether frontend OCC read-then-write is enabled (determined once at process startup). + pub frontend_read_then_write_enabled: bool, + /// Whether the coordinator is in read-only mode. Mutations must be rejected. + pub read_only: bool, } impl PeekClient { @@ -77,6 +85,9 @@ impl PeekClient { optimizer_metrics: OptimizerMetrics, persist_client: PersistClient, statement_logging_frontend: StatementLoggingFrontend, + occ_write_semaphore: Arc, + frontend_read_then_write_enabled: bool, + read_only: bool, ) -> Self { Self { coordinator_client, @@ -87,6 +98,9 @@ impl PeekClient { statement_logging_frontend, oracles: Default::default(), // lazily populated persist_client, + occ_write_semaphore, + frontend_read_then_write_enabled, + read_only, } } @@ -151,6 +165,12 @@ impl PeekClient { .expect("if the coordinator is still alive, it shouldn't have dropped our call") } + /// Returns a clone of the coordinator client, for use in cleanup guards + /// that need to send fire-and-forget commands. + pub(crate) fn coordinator_client(&self) -> &crate::Client { + &self.coordinator_client + } + /// Acquire read holds on the given compute/storage collections, and /// determine the smallest common valid write frontier among the specified collections. /// @@ -416,7 +436,53 @@ impl PeekClient { }) } - // Statement logging helper methods + /// Set up statement logging for a frontend-sequenced operation. + /// + /// If `outer_ctx_extra` is `None`, begins a new statement execution log + /// entry. If `outer_ctx_extra` is `Some` (e.g. EXECUTE/FETCH), reuses and + /// retires the existing logging context. + /// + /// Returns a [`StatementLoggingGuard`]. Callers must retire the guard + /// (e.g. via [`StatementLoggingGuard::retire_with_result`]) on the + /// execution's terminal outcome, or [`defuse`](StatementLoggingGuard::defuse) + /// it when handing off logging responsibility. Dropping the guard without + /// retiring it emits an `Aborted` end-execution event. + pub(crate) fn begin_statement_logging( + &self, + session: &mut Session, + params: &Params, + logging: &Arc>, + catalog: &Catalog, + lifecycle_timestamps: Option, + outer_ctx_extra: &mut Option, + ) -> StatementLoggingGuard { + let id = if outer_ctx_extra.is_none() { + let result = self.statement_logging_frontend.begin_statement_execution( + session, + params, + logging, + catalog.system_config(), + lifecycle_timestamps, + ); + + if let Some((logging_id, began_execution, mseh_update, prepared_statement)) = result { + self.log_began_execution(began_execution, mseh_update, prepared_statement); + Some(logging_id) + } else { + None + } + } else { + outer_ctx_extra + .take() + .and_then(|guard| guard.defuse().retire()) + }; + + StatementLoggingGuard { + id, + coordinator_client: self.coordinator_client.clone(), + now: self.statement_logging_frontend.now.clone(), + } + } /// Log the beginning of statement execution. pub(crate) fn log_began_execution( @@ -483,7 +549,10 @@ impl PeekClient { )); } - /// Log the end of statement execution. + /// Emit a `FrontendStatementLoggingEvent::EndedExecution` for the given + /// logging id. Used by callers that manage the statement-logging + /// lifecycle explicitly (see `try_frontend_peek`), rather than via the + /// RAII [`StatementLoggingGuard`]. pub(crate) fn log_ended_execution( &self, id: StatementLoggingId, @@ -502,6 +571,79 @@ impl PeekClient { } } +/// RAII guard owning a frontend statement-logging lifecycle. +/// +/// Created by [`PeekClient::begin_statement_logging`]. The guard ensures that +/// every statement for which `BeganExecution` was logged also receives a +/// corresponding `EndedExecution`, even on early-return, panic, or mid-flight +/// drop of the enclosing future. If the guard is dropped without being +/// explicitly retired, it emits `StatementEndedExecutionReason::Aborted`. +/// +/// For non-sampled statements the guard still exists but carries no id, and +/// retirement / drop are no-ops. +#[must_use = "StatementLoggingGuard must be explicitly retired or handed off; \ + otherwise `Drop` will log the statement as Aborted"] +pub(crate) struct StatementLoggingGuard { + /// `None` if the statement was not sampled for logging. + id: Option, + coordinator_client: Client, + now: mz_ore::now::NowFn, +} + +impl StatementLoggingGuard { + /// Returns the logging id, if this statement is being logged. + pub(crate) fn id(&self) -> Option { + self.id + } + + /// Retires the guard with an explicit end-execution reason. + pub(crate) fn retire(mut self, reason: statement_logging::StatementEndedExecutionReason) { + self.emit(reason); + } + + /// Retires the guard using the standard mapping from an execution result. + pub(crate) fn retire_with_result(self, result: &Result) { + let reason = match result { + Ok(resp) => resp.into(), + Err(e) => statement_logging::StatementEndedExecutionReason::Errored { + error: e.to_string(), + }, + }; + self.retire(reason); + } + + /// Hands off logging responsibility without emitting an end-execution + /// event. Use when another component (e.g. the coordinator, for streaming + /// peek / subscribe responses) will log the end asynchronously. + pub(crate) fn defuse(mut self) { + self.id = None; + } + + fn emit(&mut self, reason: statement_logging::StatementEndedExecutionReason) { + let Some(id) = self.id.take() else { + return; + }; + let ended_at = (self.now)(); + let record = statement_logging::StatementEndedExecutionRecord { + id: id.0, + reason, + ended_at, + }; + self.coordinator_client + .send(Command::FrontendStatementLogging( + FrontendStatementLoggingEvent::EndedExecution(record), + )); + } +} + +impl Drop for StatementLoggingGuard { + fn drop(&mut self) { + if self.id.is_some() { + self.emit(statement_logging::StatementEndedExecutionReason::Aborted); + } + } +} + /// Errors arising during collection lookup in peek client operations. #[derive(Error, Debug)] pub enum CollectionLookupError { diff --git a/src/adapter/src/session.rs b/src/adapter/src/session.rs index fe6f68d90a34c..daec06ecc7f2f 100644 --- a/src/adapter/src/session.rs +++ b/src/adapter/src/session.rs @@ -1818,6 +1818,14 @@ impl GroupCommitWriteLocks { self.locks.extend(existing); } + /// Insert a single lock into this collection. + /// + /// Useful when a lock is acquired directly during group commit rather + /// than handed off from a session via [`Self::merge`]. + pub fn insert_lock(&mut self, id: CatalogItemId, lock: tokio::sync::OwnedMutexGuard<()>) { + self.locks.insert(id, lock); + } + /// Returns the collections we're missing locks for, if any. pub fn missing_locks( &self, diff --git a/src/environmentd/src/http/sql.rs b/src/environmentd/src/http/sql.rs index b46b58a1a4485..f9b54342cb91f 100644 --- a/src/environmentd/src/http/sql.rs +++ b/src/environmentd/src/http/sql.rs @@ -643,6 +643,12 @@ impl SqlResult { PeekResponseUnary::Error(e) => { return Ok(SqlResult::err(client, Error::Unstructured(anyhow!(e)))); } + PeekResponseUnary::DependencyDropped(dep) => { + return Ok(SqlResult::err( + client, + Error::Unstructured(anyhow!(dep.query_terminated_error())), + )); + } PeekResponseUnary::Canceled => { return Ok(SqlResult::err(client, AdapterError::Canceled)); } @@ -1063,6 +1069,16 @@ impl ResultSender for WebSocket { Some((StatementEndedExecutionReason::Errored { error }, ctx_extra)), ); } + Some(PeekResponseUnary::DependencyDropped(dep)) => { + let error = dep.query_terminated_error(); + break ( + true, + vec![WebSocketResponse::Error( + Error::Unstructured(anyhow!(error.clone())).into(), + )], + Some((StatementEndedExecutionReason::Errored { error }, ctx_extra)), + ); + } Some(PeekResponseUnary::Canceled) => { break ( true, diff --git a/src/environmentd/src/telemetry.rs b/src/environmentd/src/telemetry.rs index 5781ad38a8ecc..3752478d0e7e6 100644 --- a/src/environmentd/src/telemetry.rs +++ b/src/environmentd/src/telemetry.rs @@ -177,6 +177,9 @@ async fn report_loop( PeekResponseUnary::Rows(rows) => row_iters.push(rows), PeekResponseUnary::Canceled => bail!("query canceled"), PeekResponseUnary::Error(e) => bail!(e), + PeekResponseUnary::DependencyDropped(dep) => { + bail!("{}", dep.query_terminated_error()) + } } } diff --git a/src/environmentd/tests/server.rs b/src/environmentd/tests/server.rs index a363082e7f1ef..eba0cd7ed52f2 100644 --- a/src/environmentd/tests/server.rs +++ b/src/environmentd/tests/server.rs @@ -667,6 +667,122 @@ ORDER BY mseh.began_at", ); } +#[mz_ore::test] +fn test_statement_logging_frontend_constant_insert_sets_cluster() { + let harness = test_util::TestHarness::default().with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ); + let (server, mut client) = setup_statement_logging_core(1.0, 1.0, "", harness); + + client.execute("SET CLUSTER TO quickstart", &[]).unwrap(); + client + .execute( + "CREATE TABLE statement_logging_constant_insert_t (x INT)", + &[], + ) + .unwrap(); + client + .execute( + "INSERT INTO statement_logging_constant_insert_t VALUES (1)", + &[], + ) + .unwrap(); + + let mut client = server.connect_internal(postgres::NoTls).unwrap(); + let row = Retry::default() + .max_duration(Duration::from_secs(30)) + .retry(|_| { + let rows = client + .query( + "SELECT mseh.cluster_name, mseh.finished_status +FROM mz_internal.mz_statement_execution_history AS mseh +LEFT JOIN mz_internal.mz_prepared_statement_history AS mpsh + ON mseh.prepared_statement_id = mpsh.id +JOIN (SELECT DISTINCT sql, sql_hash FROM mz_internal.mz_sql_text) AS mst + ON mpsh.sql_hash = mst.sql_hash +WHERE mst.sql ~~ 'INSERT INTO statement_logging_constant_insert_t%' + AND mseh.finished_at IS NOT NULL +ORDER BY mseh.began_at DESC", + &[], + ) + .unwrap(); + + if let Some(row) = rows.into_iter().next() { + Ok(row) + } else { + Err(()) + } + }) + .expect("constant INSERT statement log entry should be recorded"); + + let cluster_name: Option = row.get(0); + let finished_status: String = row.get(1); + assert_eq!(cluster_name.as_deref(), Some("quickstart")); + assert_eq!(finished_status, "success"); +} + +// Regression test: the frontend OCC read-then-write path must set +// `execution_timestamp` on the statement's log entry. The old, coordinator +// path does this through `set_statement_execution_timestamp` during group +// commit; the frontend path needs to emit the equivalent signal once its +// write commits. +#[mz_ore::test] +fn test_statement_logging_frontend_read_then_write_sets_execution_timestamp() { + let harness = test_util::TestHarness::default().with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ); + let (server, mut client) = setup_statement_logging_core(1.0, 1.0, "", harness); + + client.execute("SET CLUSTER TO quickstart", &[]).unwrap(); + client + .execute("CREATE TABLE statement_logging_rtw_t (x INT)", &[]) + .unwrap(); + client + .execute("INSERT INTO statement_logging_rtw_t VALUES (1), (2)", &[]) + .unwrap(); + // DELETE goes through the frontend OCC read-then-write path. + client + .execute("DELETE FROM statement_logging_rtw_t WHERE x = 1", &[]) + .unwrap(); + + let mut client = server.connect_internal(postgres::NoTls).unwrap(); + let row = Retry::default() + .max_duration(Duration::from_secs(30)) + .retry(|_| { + let rows = client + .query( + "SELECT mseh.execution_timestamp, mseh.finished_status +FROM mz_internal.mz_statement_execution_history AS mseh +LEFT JOIN mz_internal.mz_prepared_statement_history AS mpsh + ON mseh.prepared_statement_id = mpsh.id +JOIN (SELECT DISTINCT sql, sql_hash FROM mz_internal.mz_sql_text) AS mst + ON mpsh.sql_hash = mst.sql_hash +WHERE mst.sql ~~ 'DELETE FROM statement_logging_rtw_t%' + AND mseh.finished_at IS NOT NULL +ORDER BY mseh.began_at DESC", + &[], + ) + .unwrap(); + + if let Some(row) = rows.into_iter().next() { + Ok(row) + } else { + Err(()) + } + }) + .expect("DELETE statement log entry should be recorded"); + + let execution_timestamp: Option = row.get(0); + let finished_status: String = row.get(1); + assert_eq!(finished_status, "success"); + assert!( + execution_timestamp.is_some(), + "frontend OCC read-then-write DELETE must set execution_timestamp, got NULL" + ); +} + fn run_throttling_test(use_prepared_statement: bool) { // The `target_data_rate` should be // - high enough so that the `SELECT 1` queries get throttled (even with high CPU load due to @@ -1219,6 +1335,363 @@ fn test_cancel_long_running_query() { .expect("simple query succeeds after cancellation"); } +// Test that frontend-sequenced read-then-write statements honor pgwire cancel +// requests and do not run to completion after cancellation. +#[mz_ore::test] +fn test_cancel_frontend_read_then_write_long_running_query() { + let server = test_util::TestHarness::default() + .unsafe_mode() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .start_blocking(); + server.enable_feature_flags(&["unsafe_enable_unsafe_functions"]); + + let mut client = server.connect(postgres::NoTls).unwrap(); + let cancel_token = client.cancel_token(); + + client + .batch_execute("CREATE TABLE t (a TEXT, ts INT)") + .unwrap(); + client + .batch_execute("INSERT INTO t VALUES ('hello', 10)") + .unwrap(); + + let (shutdown_tx, shutdown_rx) = std::sync::mpsc::channel(); + let cancel_thread = thread::spawn(move || { + loop { + thread::sleep(Duration::from_millis(200)); + match shutdown_rx.try_recv() { + Ok(()) => return, + Err(std::sync::mpsc::TryRecvError::Empty) => { + let _ = cancel_token.cancel_query(postgres::NoTls); + } + Err(std::sync::mpsc::TryRecvError::Disconnected) => return, + } + } + }); + + match client.batch_execute( + "INSERT INTO t SELECT a, CASE WHEN mz_unsafe.mz_sleep(ts) > 0 THEN 0 END AS ts FROM t", + ) { + Err(e) if e.code() == Some(&SqlState::QUERY_CANCELED) => {} + Err(e) => panic!("expected error SqlState::QUERY_CANCELED, but got {e:?}"), + Ok(_) => panic!("expected error SqlState::QUERY_CANCELED, but query succeeded"), + } + + shutdown_tx.send(()).unwrap(); + cancel_thread.join().unwrap(); + + let rows = client + .query_one("SELECT count(*) FROM t", &[]) + .unwrap() + .get::<_, i64>(0); + assert_eq!( + rows, 1, + "cancelled statement should not have committed writes" + ); + + // NOTE: mz_sleep with a constant ts gets evaluated differently. This gives + // us additional coverage for cancelling at different moments in the + // processing pipeline. + let cancel_token = client.cancel_token(); + let (shutdown_tx, shutdown_rx) = std::sync::mpsc::channel(); + let cancel_thread = thread::spawn(move || { + loop { + thread::sleep(Duration::from_millis(200)); + match shutdown_rx.try_recv() { + Ok(()) => return, + Err(std::sync::mpsc::TryRecvError::Empty) => { + let _ = cancel_token.cancel_query(postgres::NoTls); + } + Err(std::sync::mpsc::TryRecvError::Disconnected) => return, + } + } + }); + + match client.batch_execute( + "INSERT INTO t SELECT a, CASE WHEN mz_unsafe.mz_sleep(10) > 0 THEN 0 END AS ts FROM t", + ) { + Err(e) if e.code() == Some(&SqlState::QUERY_CANCELED) => {} + Err(e) => panic!("expected error SqlState::QUERY_CANCELED, but got {e:?}"), + Ok(_) => panic!("expected error SqlState::QUERY_CANCELED, but query succeeded"), + } + + shutdown_tx.send(()).unwrap(); + cancel_thread.join().unwrap(); + + let rows = client + .query_one("SELECT count(*) FROM t", &[]) + .unwrap() + .get::<_, i64>(0); + assert_eq!( + rows, 1, + "cancelled statement should not have committed writes" + ); +} + +#[mz_ore::test] +fn test_frontend_read_then_write_constant_insert_prepares_unmaterializable_functions() { + let server = test_util::TestHarness::default() + .unsafe_mode() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .start_blocking(); + + let mut client = server.connect(postgres::NoTls).unwrap(); + + client.batch_execute("CREATE TABLE t (u text)").unwrap(); + client.batch_execute("BEGIN").unwrap(); + client + .execute("INSERT INTO t VALUES (current_user())", &[]) + .unwrap(); + client.batch_execute("COMMIT").unwrap(); + + let inserted_matches_current_user = client + .query_one("SELECT u = current_user() FROM t", &[]) + .unwrap() + .get::<_, bool>(0); + assert!(inserted_matches_current_user); +} + +#[mz_ore::test] +fn test_frontend_read_then_write_constant_insert_respects_max_result_size() { + let server = test_util::TestHarness::default() + .unsafe_mode() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .with_system_parameter_default("max_result_size".to_string(), "1MB".to_string()) + .start_blocking(); + + let mut client = server.connect(postgres::NoTls).unwrap(); + + client + .batch_execute("CREATE TABLE t2 (a int4, b text)") + .unwrap(); + + let err = client + .execute( + "INSERT INTO t2 SELECT * FROM generate_series(1, 10001), repeat('a', 100)", + &[], + ) + .unwrap_err(); + let db_err = err.as_db_error().expect("expected db error"); + assert!( + db_err + .message() + .contains("result exceeds max size of 1.0 MiB"), + "unexpected error: {err}" + ); +} + +#[mz_ore::test] +fn test_frontend_read_then_write_constant_insert_mz_now_uses_legacy_error() { + let server = test_util::TestHarness::default() + .unsafe_mode() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .start_blocking(); + + let mut client = server.connect(postgres::NoTls).unwrap(); + + client + .batch_execute("CREATE TABLE dec (d mz_timestamp)") + .unwrap(); + + let err = client + .execute("INSERT INTO dec VALUES (mz_now())", &[]) + .unwrap_err(); + let db_err = err.as_db_error().expect("expected db error"); + assert!( + db_err + .message() + .contains("calls to mz_now in write statements"), + "unexpected error: {err}" + ); +} + +#[mz_ore::test] +fn test_frontend_read_then_write_returning_error_does_not_commit_write() { + let server = test_util::TestHarness::default() + .unsafe_mode() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .start_blocking(); + + let mut client = server.connect(postgres::NoTls).unwrap(); + + client + .batch_execute("CREATE TABLE t (a INT, b INT)") + .unwrap(); + + let err = client + .query("INSERT INTO t VALUES (7, 8) RETURNING 1/0", &[]) + .unwrap_err(); + let db_err = err.as_db_error().expect("expected db error"); + assert!( + db_err.message().contains("division by zero"), + "unexpected error message: {:?}", + db_err.message() + ); + + let rows = client + .query_one("SELECT count(*) FROM t", &[]) + .unwrap() + .get::<_, i64>(0); + assert_eq!(rows, 0, "failing RETURNING must not commit the write"); +} + +// Regression test for the empty-snapshot branch of the OCC loop. +// +// `ActiveSubscribe::initialize` emits a progress message at `as_of` before +// any data batch is processed, so the OCC loop must not conclude +// `NoRowsMatched` on that first progress — the snapshot hasn't been +// delivered yet. The check that distinguishes "initial progress" from +// "snapshot complete and empty" is `ts > as_of`; this test exercises both +// empty-match cases and asserts the operations return zero without +// hanging or writing. +#[mz_ore::test] +fn test_frontend_read_then_write_empty_snapshot_returns_zero() { + let server = test_util::TestHarness::default() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .start_blocking(); + + let mut client = server.connect(postgres::NoTls).unwrap(); + + client.batch_execute("CREATE TABLE t (x INT)").unwrap(); + + // DELETE on a completely empty table. + let deleted = client + .execute("DELETE FROM t", &[]) + .expect("DELETE on empty table should return 0 rows"); + assert_eq!(deleted, 0, "DELETE on empty table must report 0 rows"); + + // DELETE with a WHERE clause that matches no rows against a non-empty + // table. The snapshot is non-empty (contains row (1)) but the selection + // is empty after filtering. + client.batch_execute("INSERT INTO t VALUES (1)").unwrap(); + let deleted = client + .execute("DELETE FROM t WHERE x = 999", &[]) + .expect("DELETE with no matches should return 0 rows"); + assert_eq!(deleted, 0, "DELETE with no matches must report 0 rows"); + + // UPDATE with a WHERE clause that matches no rows. + let updated = client + .execute("UPDATE t SET x = 2 WHERE x = 999", &[]) + .expect("UPDATE with no matches should return 0 rows"); + assert_eq!(updated, 0, "UPDATE with no matches must report 0 rows"); + + // The original row is still there. + let rows = client + .query_one("SELECT count(*) FROM t", &[]) + .unwrap() + .get::<_, i64>(0); + assert_eq!(rows, 1); +} + +// End-to-end coverage of the OCC retry path: +// +// N concurrent connections each issue M `UPDATE counter SET v = v + 1` +// statements against the same single-row table. Without a working +// `TimestampPassed` retry loop this would lose updates (two writers reading +// `v = k` and both committing `v = k + 1`); the final value pinning down at +// `N * M` proves retries actually re-read fresh state and re-apply the diff. +// +// Also asserts the `mz_occ_read_then_write_retry_count` histogram observes +// every UPDATE and that at least one observation reports a retry, so the +// retry-count metric stays wired up to the OCC loop. +#[mz_ore::test] +fn test_frontend_read_then_write_concurrent_updates_retry() { + const NUM_WORKERS: usize = 4; + const UPDATES_PER_WORKER: usize = 25; + + let server = test_util::TestHarness::default() + .with_system_parameter_default( + "enable_adapter_frontend_occ_read_then_write".to_string(), + "true".to_string(), + ) + .start_blocking(); + + let mut setup = server.connect(postgres::NoTls).unwrap(); + setup + .batch_execute("CREATE TABLE counter (id INT, v INT)") + .unwrap(); + setup + .batch_execute("INSERT INTO counter VALUES (1, 0)") + .unwrap(); + + let mut handles = Vec::with_capacity(NUM_WORKERS); + for _ in 0..NUM_WORKERS { + let mut client = server.connect(postgres::NoTls).unwrap(); + handles.push(thread::spawn(move || { + for _ in 0..UPDATES_PER_WORKER { + client + .execute("UPDATE counter SET v = v + 1 WHERE id = 1", &[]) + .expect("UPDATE under contention should succeed via OCC retry"); + } + })); + } + for handle in handles { + handle.join().expect("worker thread panicked"); + } + + let final_v: i32 = setup + .query_one("SELECT v FROM counter WHERE id = 1", &[]) + .unwrap() + .get(0); + let expected = i32::try_from(NUM_WORKERS * UPDATES_PER_WORKER).unwrap(); + assert_eq!( + final_v, expected, + "concurrent OCC UPDATEs lost updates: expected {expected}, got {final_v}", + ); + + // Inspect the OCC retry-count histogram. Every UPDATE that took the + // frontend OCC path should produce exactly one observation, so + // sample_count must be >= NUM_WORKERS * UPDATES_PER_WORKER. Same-row + // contention essentially guarantees at least one observation lands above + // the 0-retry bucket, so we assert that too. + let metrics = server.metrics_registry().gather(); + let retry_metric = metrics + .iter() + .find(|m| m.name() == "mz_occ_read_then_write_retry_count") + .expect("mz_occ_read_then_write_retry_count metric should be registered"); + let metric = retry_metric.get_metric(); + assert_eq!(metric.len(), 1, "expected a single histogram series"); + let histogram = metric[0].get_histogram(); + + let total_updates = u64::try_from(NUM_WORKERS * UPDATES_PER_WORKER).unwrap(); + assert!( + histogram.get_sample_count() >= total_updates, + "expected at least {} OCC observations, got {}", + total_updates, + histogram.get_sample_count(), + ); + + let zero_retry_bucket = histogram + .get_bucket() + .iter() + .find(|b| b.upper_bound() == 0.0) + .expect("histogram should have a 0-retry bucket"); + assert!( + zero_retry_bucket.cumulative_count() < histogram.get_sample_count(), + "expected at least one UPDATE to retry under contention; \ + all {} observations landed in the 0-retry bucket", + histogram.get_sample_count(), + ); +} + fn test_cancellation_cancels_dataflows(query: &str) { // Query that returns how many dataflows are currently installed. // Ignores introspection subscribe dataflows. @@ -3311,11 +3784,11 @@ fn test_github_20262() { } } -// Test that the server properly handles cancellation requests of read-then-write queries. +// Test that a timed-out read-then-write query does not commit its writes. // See database-issues#6134. #[mz_ore::test] #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `epoll_wait` on OS `linux` -fn test_cancel_read_then_write() { +fn test_timeout_read_then_write() { let server = test_util::TestHarness::default() .unsafe_mode() .start_blocking(); @@ -3325,53 +3798,27 @@ fn test_cancel_read_then_write() { client .batch_execute("CREATE TABLE foo (a TEXT, ts INT)") .unwrap(); + client + .batch_execute("INSERT INTO foo VALUES ('hello', 10)") + .unwrap(); - // Lots of races here, so try this whole thing in a loop. - Retry::default() - .clamp_backoff(Duration::ZERO) - .retry(|_state| { - let mut client1 = server.connect(postgres::NoTls).unwrap(); - let mut client2 = server.connect(postgres::NoTls).unwrap(); - let cancel_token = client2.cancel_token(); + client + .batch_execute("SET statement_timeout = '5s'") + .unwrap(); - client1.batch_execute("DELETE FROM foo").unwrap(); - client1.batch_execute("SET statement_timeout = '5s'").unwrap(); - client1 - .batch_execute("INSERT INTO foo VALUES ('hello', 10)") - .unwrap(); + let err = client + .batch_execute("INSERT INTO foo SELECT a, CASE WHEN mz_unsafe.mz_sleep(ts) > 0 THEN 0 END AS ts FROM foo") + .unwrap_err(); + assert_contains!(err.to_string_with_causes(), "statement timeout"); - let handle1 = thread::spawn(move || { - let err = client1 - .batch_execute("insert into foo select a, case when mz_unsafe.mz_sleep(ts) > 0 then 0 end as ts from foo") - .unwrap_err(); - assert_contains!( - err.to_string_with_causes(), - "statement timeout" - ); - client1 - }); - std::thread::sleep(Duration::from_millis(100)); - let handle2 = thread::spawn(move || { - let err = client2 - .batch_execute("insert into foo values ('blah', 1);") - .unwrap_err(); - assert_contains!( - err.to_string_with_causes(), - "canceling statement" - ); - }); - std::thread::sleep(Duration::from_millis(100)); - cancel_token.cancel_query(postgres::NoTls)?; - let mut client1 = handle1.join().unwrap(); - handle2.join().unwrap(); - let rows:i64 = client1.query_one ("SELECT count(*) FROM foo", &[]).unwrap().get(0); - // We ran 3 inserts. First succeeded. Second timedout. Third cancelled. - if rows !=1 { - anyhow::bail!("unexpected row count: {rows}"); - } - Ok::<_, anyhow::Error>(()) - }) - .unwrap(); + let rows: i64 = client + .query_one("SELECT count(*) FROM foo", &[]) + .unwrap() + .get(0); + assert_eq!( + rows, 1, + "timed-out statement should not have committed writes" + ); } #[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 1))] diff --git a/src/environmentd/tests/sql.rs b/src/environmentd/tests/sql.rs index 3779e344c4c5c..e70262984e644 100644 --- a/src/environmentd/tests/sql.rs +++ b/src/environmentd/tests/sql.rs @@ -1690,8 +1690,12 @@ fn test_subscribe_outlive_cluster() { .batch_execute("CREATE CLUSTER newcluster REPLICAS (r1 (size 'scale=1,workers=1'))") .unwrap(); client2_cancel.cancel_query(postgres::NoTls).unwrap(); - client2 - .batch_execute("ROLLBACK; SET CLUSTER = default") + // The cancel is asynchronous and might race with subsequent commands. + // Retry ROLLBACK in a loop in case it gets canceled. + Retry::default() + .max_tries(5) + .clamp_backoff(Duration::from_millis(100)) + .retry(|_| client2.batch_execute("ROLLBACK; SET CLUSTER = default")) .unwrap(); assert_eq!( client2 @@ -2529,7 +2533,7 @@ fn test_subscribe_on_dropped_source() { assert!( res.unwrap_db_error() .message() - .contains("subscribe has been terminated because underlying relation") + .contains("query could not complete because relation") ); } @@ -2585,7 +2589,7 @@ fn test_dont_drop_sinks_twice() { let err = out.read_to_end(&mut vec![]).unwrap_err(); assert!( err.to_string_with_causes() - .contains("subscribe has been terminated") + .contains("copy has been terminated") ); drop(out); diff --git a/src/pgwire/src/protocol.rs b/src/pgwire/src/protocol.rs index 35c832f85ea39..730ed5516f40b 100644 --- a/src/pgwire/src/protocol.rs +++ b/src/pgwire/src/protocol.rs @@ -2422,6 +2422,9 @@ where None => FetchResult::Rows(None), Some(PeekResponseUnary::Rows(rows)) => FetchResult::Rows(Some(rows)), Some(PeekResponseUnary::Error(err)) => FetchResult::Error(err), + Some(PeekResponseUnary::DependencyDropped(dep)) => { + FetchResult::Error(dep.query_terminated_error()) + } Some(PeekResponseUnary::Canceled) => FetchResult::Canceled, }, notice = notice_fut => { @@ -2649,6 +2652,15 @@ where .await .map(|state| (state, SendRowsEndedReason::Errored { error: text })); } + Some(PeekResponseUnary::DependencyDropped(dep)) => { + let text = dep.copy_terminated_error(); + let err = + ErrorResponse::error(SqlState::INTERNAL_ERROR, text.clone()); + return self + .send_error_and_get_state(err) + .await + .map(|state| (state, SendRowsEndedReason::Errored { error: text })); + } Some(PeekResponseUnary::Canceled) => { return self.send_error_and_get_state(ErrorResponse::error( SqlState::QUERY_CANCELED, diff --git a/src/sql/src/session/vars.rs b/src/sql/src/session/vars.rs index a28ce1edf0331..695a989a0006a 100644 --- a/src/sql/src/session/vars.rs +++ b/src/sql/src/session/vars.rs @@ -1104,6 +1104,8 @@ impl SystemVars { &MAX_RESULT_SIZE, &MAX_COPY_FROM_ROW_SIZE, &ALLOWED_CLUSTER_REPLICA_SIZES, + &MAX_CONCURRENT_OCC_WRITES, + &MAX_OCC_RETRIES, &upsert_rocksdb::UPSERT_ROCKSDB_COMPACTION_STYLE, &upsert_rocksdb::UPSERT_ROCKSDB_OPTIMIZE_COMPACTION_MEMTABLE_BUDGET, &upsert_rocksdb::UPSERT_ROCKSDB_LEVEL_COMPACTION_DYNAMIC_LEVEL_BYTES, @@ -1657,6 +1659,16 @@ impl SystemVars { .collect() } + /// Returns the value of the `max_concurrent_occ_writes` configuration parameter. + pub fn max_concurrent_occ_writes(&self) -> u32 { + *self.expect_value(&MAX_CONCURRENT_OCC_WRITES) + } + + /// Returns the value of the `max_occ_retries` configuration parameter. + pub fn max_occ_retries(&self) -> u32 { + *self.expect_value(&MAX_OCC_RETRIES) + } + /// Returns the value of the `default_cluster_replication_factor` configuration parameter. pub fn default_cluster_replication_factor(&self) -> u32 { *self.expect_value::(&DEFAULT_CLUSTER_REPLICATION_FACTOR) diff --git a/src/sql/src/session/vars/definitions.rs b/src/sql/src/session/vars/definitions.rs index fbf53e96ca358..38defca57f5f6 100644 --- a/src/sql/src/session/vars/definitions.rs +++ b/src/sql/src/session/vars/definitions.rs @@ -633,6 +633,20 @@ pub static ALLOWED_CLUSTER_REPLICA_SIZES: VarDefinition = VarDefinition::new( true, ); +pub static MAX_CONCURRENT_OCC_WRITES: VarDefinition = VarDefinition::new( + "max_concurrent_occ_writes", + value!(u32; 4), + "Maximum number of concurrent read-then-write (DELETE/UPDATE) operations using OCC. Read at startup; changes require an environmentd restart (Materialize).", + false, +); + +pub static MAX_OCC_RETRIES: VarDefinition = VarDefinition::new( + "max_occ_retries", + value!(u32; 1000), + "Maximum number of OCC retry attempts per read-then-write operation before giving up (Materialize).", + false, +); + pub static PERSIST_FAST_PATH_LIMIT: VarDefinition = VarDefinition::new( "persist_fast_path_limit", value!(usize; 25), diff --git a/test/cluster/mzcompose.py b/test/cluster/mzcompose.py index 049ac3ddd7317..3d2f8fc5bf92d 100644 --- a/test/cluster/mzcompose.py +++ b/test/cluster/mzcompose.py @@ -3852,7 +3852,7 @@ def subscribe(): # Expected msg = str(e) if ("cached plan must not change result type" in msg) or ( - "subscribe has been terminated because underlying relation" in msg + "query could not complete because relation" in msg ): continue raise e @@ -3987,7 +3987,7 @@ def subscribe(): except InternalError_ as e: assert 'query could not complete because relation "materialize.public.counter_tbl" was dropped' in str( e - ) or 'subscribe has been terminated because underlying relation "materialize.public.counter_tbl" was dropped' in str( + ) or 'query could not complete because relation "materialize.public.counter_tbl" was dropped' in str( e ) for thread in threads: @@ -4775,7 +4775,7 @@ def subscribe(): except DatabaseError as exc: assert ( exc.diag.message_primary - == 'subscribe has been terminated because underlying relation "materialize.public.mv2" was dropped' + == 'query could not complete because relation "materialize.public.mv2" was dropped' ) subscribe_thread = Thread(target=subscribe) @@ -6504,7 +6504,7 @@ def subscribe(): except DatabaseError as exc: assert ( exc.diag.message_primary - == 'subscribe has been terminated because underlying relation "materialize.public.mv" was dropped' + == 'query could not complete because relation "materialize.public.mv" was dropped' ), exc subscribe_thread = Thread(target=subscribe) diff --git a/test/sqllogictest/ct_various.slt b/test/sqllogictest/ct_various.slt index 5a829f1c2e01c..9ca28cc9a1e0a 100644 --- a/test/sqllogictest/ct_various.slt +++ b/test/sqllogictest/ct_various.slt @@ -187,7 +187,7 @@ DROP CONTINUAL TASK drop_ct CASCADE; ---- COMPLETE 0 -statement error subscribe has been terminated because underlying relation +statement error query could not complete because relation "materialize\.public\.drop_ct" was dropped FETCH 1 c WITH (TIMEOUT = '5s') statement ok