diff --git a/Cargo.lock b/Cargo.lock index 7e083c579be0d..296955621634a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5756,6 +5756,34 @@ dependencies = [ "uuid", ] +[[package]] +name = "mz-bogo-consensus" +version = "0.0.0" +dependencies = [ + "anyhow", + "async-trait", + "axum", + "bytes", + "clap", + "futures", + "mz-build-info", + "mz-build-tools", + "mz-http-util", + "mz-ore", + "prometheus", + "prost", + "prost-build", + "serde", + "socket2 0.6.3", + "tokio", + "tokio-stream", + "tonic", + "tonic-prost", + "tonic-prost-build", + "tracing", + "tracing-subscriber", +] + [[package]] name = "mz-build-info" version = "0.0.0" @@ -5790,6 +5818,7 @@ dependencies = [ "derivative", "differential-dataflow", "futures", + "imbl", "insta", "ipnet", "itertools 0.14.0", @@ -7131,6 +7160,7 @@ dependencies = [ "itertools 0.14.0", "md-5", "mz-aws-util", + "mz-bogo-consensus", "mz-build-tools", "mz-dyncfg", "mz-foundationdb", @@ -7157,6 +7187,8 @@ dependencies = [ "timely", "tokio", "tokio-postgres", + "tokio-stream", + "tonic", "tracing", "turmoil", "url", @@ -8034,6 +8066,7 @@ dependencies = [ "chrono", "differential-dataflow", "futures", + "imbl", "itertools 0.14.0", "mz-build-info", "mz-ccsr", diff --git a/Cargo.toml b/Cargo.toml index 8ba97cb61b290..745ad8cdb94f1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -12,6 +12,7 @@ members = [ "src/aws-secrets-controller", "src/aws-util", "src/balancerd", + "src/bogo-consensus", "src/build-info", "src/build-tools", "src/catalog", @@ -137,6 +138,7 @@ default-members = [ "src/aws-secrets-controller", "src/aws-util", "src/balancerd", + "src/bogo-consensus", "src/build-info", "src/build-tools", "src/catalog", diff --git a/bin/bogo-consensus b/bin/bogo-consensus new file mode 100755 index 0000000000000..8cc433378ff59 --- /dev/null +++ b/bin/bogo-consensus @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +# Copyright Materialize, Inc. and contributors. All rights reserved. +# +# Use of this software is governed by the Business Source License +# included in the LICENSE file at the root of this repository. +# +# 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. +# +# bogo-consensus — build and run the in-memory persist Consensus gRPC server. + +set -euo pipefail + +cd "$(dirname "$0")/.." + +exec cargo run --bin mz-bogo-consensus -- "$@" diff --git a/ci/release-qualification/pipeline.template.yml b/ci/release-qualification/pipeline.template.yml index a337cfe38531b..feb56ce668049 100644 --- a/ci/release-qualification/pipeline.template.yml +++ b/ci/release-qualification/pipeline.template.yml @@ -694,8 +694,8 @@ steps: agents: queue: linux-aarch64-small - - id: cluster-spec-sheet-environmentd - label: "Cluster spec sheet: Environmentd (against Staging)" + - id: cluster-spec-sheet-envd-qps-scalability + label: "Cluster spec sheet: Envd QPS Scalability (against Staging)" depends_on: build-aarch64 timeout_in_minutes: 3600 concurrency: 1 @@ -704,6 +704,34 @@ steps: - ./ci/plugins/mzcompose: composition: cluster-spec-sheet run: default - args: [--cleanup, --target=cloud-staging, environmentd] + args: [--cleanup, --target=cloud-staging, envd_qps_scalability] + agents: + queue: linux-aarch64-small + + - id: cluster-spec-sheet-envd-objects-scalability + label: "Cluster spec sheet: Envd Objects Scalability (against Staging)" + depends_on: build-aarch64 + timeout_in_minutes: 3600 + concurrency: 1 + concurrency_group: 'cluster-spec-sheet-cluster' + plugins: + - ./ci/plugins/mzcompose: + composition: cluster-spec-sheet + run: default + args: [--cleanup, --target=cloud-staging, envd_objects_scalability] + agents: + queue: linux-aarch64-small + + - id: cluster-spec-sheet-cluster-object-limits + label: "Cluster spec sheet: Cluster Object Limits (against Staging)" + depends_on: build-aarch64 + timeout_in_minutes: 3600 + concurrency: 1 + concurrency_group: 'cluster-spec-sheet-cluster' + plugins: + - ./ci/plugins/mzcompose: + composition: cluster-spec-sheet + run: default + args: [--cleanup, --target=cloud-staging, cluster_object_limits] agents: queue: linux-aarch64-small diff --git a/doc/developer/design/20260515_ddl_catalog_o_delta.md b/doc/developer/design/20260515_ddl_catalog_o_delta.md new file mode 100644 index 0000000000000..37f35bfcbe034 --- /dev/null +++ b/doc/developer/design/20260515_ddl_catalog_o_delta.md @@ -0,0 +1,398 @@ +# Catalog transactions in O(delta) instead of O(catalog size) + +- Associated: branch `envd-ddl-scalability`; audit harness and findings + in `test/envd-ddl-scalability/`. + +## The Problem + +DDL latency in Materialize grows with the number of catalog objects. +On the `envd-ddl-scalability` audit harness (warm local envd, post-fix +state), padding the catalog with 5000 trivial tables raises p50 +single-statement DDL latency by 17-31 ms versus an empty catalog: + +| op | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| `CREATE TABLE` | 30.0 ms | 47.3 ms | +17.3 | +| `DROP TABLE` | 18.6 ms | 42.6 ms | +24.0 | +| `ALTER TABLE … ADD COLUMN` | 22.3 ms | 53.0 ms | +30.7 | +| `RENAME TABLE` | 17.4 ms | 43.6 ms | +26.2 | + +The slope is ~3-6 μs/object and is shared by every DDL op — including +RENAME and CREATE VIEW, which do no controller-side work. That tells +us the cost is in the catalog/coordinator infrastructure paid by every +DDL, not in any op-specific path. + +When padding objects share dependencies (e.g. many MVs reading the +same base table), the cost goes super-linear and dominates everything +else — see `test/envd-ddl-scalability/NOTES.md` "Storage-side blow-up" +for that workstream. This document does not address the storage-side +cost. + +### Where the time goes + +Trace ranking (post-fix envd, N=0 → N=5000, top growers): + +| span | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| `snapshot` (catalog durable) | 0.5 ms | 4.1 ms | +3.6 | +| `transaction` (catalog durable) | 0.4 ms | 3.0 ms | +2.6 | +| `consolidate` (catalog durable) | 0.4 ms | 2.1 ms | +1.7 | +| `PersistTableWriteCmd::Append` (catalog shard) | 5.0 ms | 7.6 ms | +2.6 | +| `apply_catalog_implications_inner` | <0.5 ms | 1.2 ms | +1.2 | +| `apply_updates` (in-memory) | 0.5 ms | 1.4 ms | +0.9 | + +The first three add ~8 ms per DDL at N=5000 ≈ **1.5 μs/object**, the +single largest source of the slope. Their cost is structural: + +- `with_snapshot` (`src/catalog/src/durable/persist.rs:766`) walks the + consolidated trace and constructs a fresh + `Snapshot { databases: BTreeMap, schemas: BTreeMap, items: BTreeMap, … }` + per transaction. For N items, that's N inserts into the items + BTreeMap alone (plus all the other tables). +- `Transaction::new` (`src/catalog/src/durable/transaction.rs:128`) + takes that snapshot and walks every row a second time, calling + `TableTransaction::new(initial.into_iter().map(RustType::from_proto) + .collect())` — a full O(N) proto→Rust deserialisation into owned + BTreeMaps that the transaction then mutates. +- `consolidate` (`persist.rs:706`) re-consolidates the in-memory + trace via differential dataflow when the catalog reaches its + consolidation threshold. Trace size grows with N → cost grows + with N. + +So **every single-statement DDL materializes the full catalog state +twice**: once into `Snapshot`, once into `TableTransaction.initial`. +Adapter serialises DDL so these are paid sequentially — there is no +parallelism to hide them. + +Three prior fixes on this branch removed named O(N) loops in +`validate_resource_limits` (`f69d91c977`), `allocate_oids` +(`9fca09ff8a`), and `TableTransaction::insert` (`293a243e6b`), but +the slope is essentially unchanged — those loops contributed below +measurement noise at N=5000 relative to the structural cost above. + +## Success Criteria + +1. Single-statement DDL latency is roughly flat as catalog size grows. + Concretely, on the `envd-ddl-scalability` harness with tables + padding, the Δ between N=0 and N=5000 for the four ops above + should drop from +17-31 ms to ≤ +5 ms. +2. The fix is invariant under add-only growth — N=10 000 and beyond + should remain flat to the same tolerance, modulo persist-shard + append cost (which is out of scope here, see below). +3. Multi-statement DDL transactions stop re-doing O(prior_statements) + work per added statement. +4. No regression on DDL throughput on a small catalog. + +## Out of Scope + +- **Persist-side scaling of the catalog shard.** + `PersistTableWriteCmd::Append` and `compare_and_append` on the + catalog shard grow with shard history. Smaller per-DDL batches from + the durable-overlay change should reduce this somewhat, but the + remainder belongs in a separate workstream owned by storage/persist. +- **Storage controller `open_data_handles` super-linearity.** The + shared-dependency MV-padding regression is dominated by per-shard + persist init in the storage controller; tracked separately in + `NOTES.md` and not addressed here. +- **Schema/format changes to what is durably stored.** The fix is + about how transactions read and write the catalog in memory; the + protobuf state-update encoding on persist is unchanged. + +## Solution Proposal + +The full catalog already exists once as in-memory shared state in +`PersistCatalogState`. The fix is to **stop re-materialising it per +transaction**. + +Replace the per-transaction `Snapshot` construction with shared +indexed catalog data, owned by `PersistCatalogState` and maintained +incrementally on every commit. Transactions hold an overlay over +that shared data: reads probe the overlay then fall through to the +base, writes only touch the overlay, and commits emit only the delta. + +### Shared durable state + +In `src/catalog/src/durable/persist.rs`, give `PersistCatalogState` a +new field: + +```rust +struct DurableCatalogData { + databases: Arc>, + schemas: Arc>, + roles: Arc>, + role_auth: Arc>, + items: Arc>, + comments: Arc>, + clusters: Arc>, + network_policies: Arc>, + cluster_replicas: Arc>, + introspection_sources: Arc>, + id_allocator: Arc>, + configs: Arc>, + settings: Arc>, + source_references: Arc>, + system_object_mappings: Arc>, + system_configurations: Arc>, + default_privileges: Arc>, + system_privileges: Arc>, + storage_collection_metadata: Arc>, + unfinalized_shards: Arc>, + txn_wal_shard: Arc>, + + indexes: DurableCatalogIndexes, + counts: CatalogResourceCounts, + allocated_oids: Arc>, +} +``` + +`imbl::OrdMap` gives O(1) structural cloning, so passing it into a +transaction is essentially free. The `Arc` is belt-and-suspenders and +makes ownership explicit at the borrow level. + +`DurableCatalogData` is maintained by `PersistHandle::sync` / the +trace-apply path: each `StateUpdate` becomes an insert or remove on +the relevant map, plus index/count updates. The data structure +replaces the per-transaction `Snapshot` entirely; the trace +(`self.snapshot: Vec<(StateUpdateKind, Timestamp, Diff)>`) can be +dropped in favour of consolidating directly into the maps as updates +arrive. We keep the persist read-path machinery +(`sync_to_current_upper`, `update_applier`) as-is — only the +materialised view of the state changes shape. + +### Indexes maintained alongside + +`DurableCatalogIndexes` holds the lookups that today require scans: + +```rust +struct DurableCatalogIndexes { + database_by_name: imbl::OrdMap, + schema_by_parent_name: imbl::OrdMap<(Option, String), SchemaKey>, + item_by_namespace: imbl::OrdMap, + role_by_name: imbl::OrdMap, + cluster_by_name: imbl::OrdMap, + replica_by_cluster_and_name: imbl::OrdMap<(ClusterId, String), ClusterReplicaKey>, + oid_owner: imbl::OrdMap, +} +``` + +These are updated in lock-step with the main maps in the trace-apply +path. They turn the uniqueness checks in `TableTransaction::insert` +and the OID allocator's full-scan-for-membership into O(log N) point +lookups. + +### Per-transaction overlay + +In `src/catalog/src/durable/transaction.rs`, change `TableTransaction` +from owning its own `BTreeMap` of initial values to overlaying: + +```rust +struct TableTransaction { + base: Arc>, + pending: BTreeMap>>, + uniqueness_violation: Option bool>, +} +``` + +Construction is `Arc::clone(&data.items)` — no walk, no +deserialisation. Reads check `pending` first (overlay semantics), +then `base`. Writes only touch `pending`. Commit emits a delta: +exactly the keys in `pending`. + +`Transaction::new` becomes proportional to the number of `Arc::clone`s +(constant in the number of tables, not in N). The work `Transaction:: +new` does today — building `initial_oids` by walking every OID-bearing +table's values — moves into the shared `DurableCatalogData`. The +allocator at the transaction level uses +`data.allocated_oids.clone()` plus a per-transaction +delta-applied overlay; both lookup and update remain O(log N). + +### Adapter-side: durable overlay travels with the DDL transaction + +In `src/adapter/src/coord/`, today every statement in a multi- +statement DDL transaction (`BEGIN; ALTER … ; CREATE … ; COMMIT`) ends +up calling `catalog.transact_op_in_session_transaction` which +**replays the accumulated ops** against a fresh durable state. With +the overlay design this is unnecessary. + +Replace the in-flight DDL representation with an owned durable +overlay: + +```rust +enum TransactionOps { + None, + Reads, + Writes(…), + DDL { + durable_txn: DurableCatalogTransaction, + state: CatalogState, + side_effects: Vec<…>, + revision: u64, + }, + … +} +``` + +This requires `Transaction<'a>` to stop borrowing `&'a mut dyn +DurableCatalogState`. Today the borrow exists because the transaction +holds the `durable_catalog` reference for things like +`get_and_increment_id`. Most of those calls can become methods that +read from the shared data + overlay; the remaining ones (allocator +bumps) can be modeled as overlay-only mutations until commit. Once +the borrow is gone, the transaction can be moved into +`TransactionOps::DDL` and held across statements, with each +subsequent statement adding to the same overlay. + +Result: a 50-statement DDL transaction does one durable read at +`BEGIN` instead of 50, and commits once at `COMMIT`. + +### What about `consolidate`? + +In the current implementation, the trace vector +`PersistHandle::snapshot` is periodically consolidated via +`differential_dataflow::consolidation::consolidate_updates`. With the +proposed design, the trace can be applied incrementally into the +shared `DurableCatalogData` maps — there is no consolidated trace +vector to maintain on the read path. The `Vec` can stay as a write- +side buffer of recent updates if useful, but the materialised state +is the `imbl::OrdMap` view. + +That eliminates the `consolidate` span entirely from the per-DDL +path. + +### `apply_catalog_implications_inner` and `apply_updates` audit + +These two together added +2 ms at N=5000. The structural fix doesn't +touch them; they are a separate audit pass. + +- `apply_updates` (`src/adapter/src/catalog/apply.rs:102`): iterates + the per-DDL delta of `StateUpdate`s, so should be O(delta). The + growth implies something inside the apply handlers reads catalog- + wide state. Audit each `apply_*_update` branch. +- `apply_catalog_implications_inner` + (`src/adapter/src/coord/catalog_implications.rs:182`): handlers + per implication kind. Same shape; one of the handlers is reading + catalog-wide state. + +Fix as small, focused changes after the structural fix lands. + +### Storage point APIs + +`StorageTxn::get_collection_metadata() -> BTreeMap<…>` forces full +scans on callers. Add: + +```rust +fn get_collection_shard(&self, id: GlobalId) -> Option; +fn collection_metadata_contains_shard(&self, shard: ShardId) -> bool; +``` + +so the catalog transaction's storage-side prep stays proportional to +the affected collections. Small, mechanical, ride along with the +durable-overlay change. + +## Minimal Viable Prototype + +Land in this order so each step is independently shippable and +measurable. Each step has a clear regression signal in the audit +harness (`test/envd-ddl-scalability/audit.py`). + +1. **`DurableCatalogData` + overlay reads** — no ownership changes + to `Transaction<'a>` yet. `with_snapshot` returns an + `Arc::clone` of the shared maps; `Transaction::new` consumes + them as `Arc` rather than owned BTreeMaps. + `TableTransaction` becomes overlay-based. + + Regression signal: `snapshot`, `transaction`, and `consolidate` + span self-times drop to near-zero. Slope should drop by ~1.5 + μs/object. + +2. **Indexes** (`database_by_name`, `item_by_namespace`, etc.) + maintained on the trace-apply path. Wire callers in + `TableTransaction::insert`, `verify_keys`, the OID allocator + membership probe, and any name-lookup paths. + + Regression signal: the secondary growth contributors from the + trace ranking flatten. + +3. **Maintained resource counts** (already in `CatalogState`, branch + commit `f69d91c977`). Re-validate that this still works after + step 1. + +4. **OID allocator backed by `Arc>` in shared + state**, with per-transaction delta. Removes the per-txn O(N) + `initial_oids` construction from `Transaction::new`. The existing + `initial_oids` cache (`9fca09ff8a`) becomes obsolete and is + removed. + + Regression signal: `Transaction::new` self-time at high N drops + further. + +5. **Split `Transaction` ownership.** Stop borrowing + `&mut dyn DurableCatalogState`; allow the transaction to be + owned. Add `TransactionOps::DDL` carrying the durable overlay + across statements. Multi-statement DDL transactions stop + replaying ops. + + Regression signal: latency of a 10-statement DDL transaction + should be flat in catalog size and drop dramatically vs. today's + replay cost. + +6. **`apply_catalog_implications_inner` / `apply_updates` audit.** + Identify and fix the remaining per-DDL O(N) loops. + +7. **Storage point APIs.** Mechanical. + +The first step alone should close roughly half the slope on the +audit harness. Each subsequent step is a smaller win. + +## Alternatives + +### Keep `Snapshot`, optimise `Snapshot` construction + +Make `Snapshot` lazily backed by the trace rather than eagerly +materialised. This preserves the existing transaction API but is a +bigger change to the snapshot type and still pays the proto→Rust +conversion when the transaction reads any row. The proposed design +sidesteps both costs by sharing the materialised state. + +### A purely incremental `Snapshot` cache in `PersistCatalogState` + +Cache the most recent `Snapshot`, mutate it incrementally as updates +arrive, hand out clones to transactions. This is closer to the +proposed design but uses owned BTreeMaps, so handing a Snapshot to a +transaction still costs O(N) for the clone unless the BTreeMap is +swapped for a structurally-shared map. At which point we have the +proposed design. + +### Use `Arc` instead of `imbl::OrdMap` + +`Arc` plus a separate `pending: BTreeMap>` +overlay is workable. `imbl::OrdMap` gives the same overlay semantics +for free (the data structure is itself Cow-on-write with O(log N) +mutations) and matches what `CatalogState` already uses. Choosing +`imbl` reduces the amount of bespoke overlay code. + +### Skip the durable overlay; only fix the in-memory side + +The trace shows the durable side dominates. An in-memory-only fix +would leave most of the slope intact. + +## Open questions + +- Are there callers of `Snapshot` outside the transaction path that + need a fully materialised owned snapshot (e.g. dumps, debug tools, + upgrade paths)? If so, expose a helper that materialises one from + the shared maps on demand — only those callers pay the O(N) cost. +- Audit-log updates today live on the `Transaction`. They are + append-only and don't need the overlay treatment; confirm they + stay correct under the new transaction shape. +- The `verify` / `verify_keys` walks in `TableTransaction` use + uniqueness-violation predicates. For the cases handled by named + indexes (name-based uniqueness), the index does the work. For any + predicate that isn't reducible to a key, fall back to the existing + walk over `pending` only (skip `base`), and require the caller to + supply an explicit "this predicate is index-backed" capability. +- The `revision: u64` field on `TransactionOps::DDL` is a hook for + optimistic concurrency between in-flight readers and the + serialised DDL writer. Defer the policy decision (re-check at + commit vs. fail-fast on conflict) until step 5; the field is + cheap to carry. diff --git a/misc/monitoring/tempo.yml b/misc/monitoring/tempo.yml index c8c9f35cb220d..c5c537efb504e 100644 --- a/misc/monitoring/tempo.yml +++ b/misc/monitoring/tempo.yml @@ -45,3 +45,5 @@ storage: overrides: metrics_generator_processors: [service-graphs, span-metrics] + # Local debugging emits more than the default 10k live traces; bump. + max_traces_per_user: 200000 diff --git a/misc/python/materialize/cli/run.py b/misc/python/materialize/cli/run.py index 6b277a4c455af..33262afda57ab 100644 --- a/misc/python/materialize/cli/run.py +++ b/misc/python/materialize/cli/run.py @@ -120,6 +120,12 @@ def main() -> int: help="Blob storage connection string", default=os.getenv("MZDEV_BLOB", DEFAULT_BLOB), ) + parser.add_argument( + "--persist-consensus-url", + help="Override the persist Consensus URL (e.g. bogo://127.0.0.1:6882). " + "Defaults to the postgres URL with `?options=--search_path=consensus`.", + default=os.getenv("MZDEV_PERSIST_CONSENSUS_URL"), + ) parser.add_argument( "--release", help="Build artifacts in release mode, with optimizations", @@ -292,6 +298,11 @@ def main() -> int: environment_id = f"local-az1-{uuid.uuid4()}-0" environment_file.write_text(environment_id) + persist_consensus_url = ( + args.persist_consensus_url + or f"{args.postgres}?options=--search_path=consensus" + ) + print(f"persist-consensus-url: {persist_consensus_url}") print(f"persist-blob-url: {args.blob}") print(f"listeners config path: {args.listeners_config_path}") command += [ @@ -302,7 +313,7 @@ def main() -> int: f"--orchestrator-process-prometheus-service-discovery-directory={MZDATA}/prometheus", f"--orchestrator-process-scratch-directory={scratch}", "--secrets-controller=local-file", - f"--persist-consensus-url={args.postgres}?options=--search_path=consensus", + f"--persist-consensus-url={persist_consensus_url}", f"--persist-blob-url={args.blob}", f"--timestamp-oracle-url={args.postgres}?options=--search_path=tsoracle", f"--environment-id={environment_id}", diff --git a/src/adapter/src/catalog.rs b/src/adapter/src/catalog.rs index c161785e44a03..2c119a9169e46 100644 --- a/src/adapter/src/catalog.rs +++ b/src/adapter/src/catalog.rs @@ -88,7 +88,7 @@ use uuid::Uuid; // DO NOT add any more imports from `crate` outside of `crate::catalog`. pub use crate::catalog::builtin_table_updates::BuiltinTableUpdate; pub use crate::catalog::open::{InitializeStateResult, OpenCatalogResult}; -pub use crate::catalog::state::CatalogState; +pub use crate::catalog::state::{CatalogState, UserConnectionKind}; pub use crate::catalog::transact::{ DropObjectInfo, InjectedAuditEvent, Op, ReplicaCreateDropReason, TransactionResult, }; @@ -138,6 +138,11 @@ pub struct Catalog { expr_cache_handle: Option, storage: Arc>>, transient_revision: u64, + /// Set by the Coordinator after construction. `None` in tests / debug + /// catalogs that don't bother wiring a metrics registry through. + transact_phase_metrics: Option, + apply_updates_phase_metrics: Option, + apply_update_kind_metrics: Option, } // Implement our own Clone because derive can't unless S is Clone, which it's @@ -149,11 +154,35 @@ impl Clone for Catalog { expr_cache_handle: self.expr_cache_handle.clone(), storage: Arc::clone(&self.storage), transient_revision: self.transient_revision, + transact_phase_metrics: self.transact_phase_metrics.clone(), + apply_updates_phase_metrics: self.apply_updates_phase_metrics.clone(), + apply_update_kind_metrics: self.apply_update_kind_metrics.clone(), } } } impl Catalog { + /// Install a `HistogramVec` (one label: `phase`) for phase-level timing + /// inside `Catalog::transact`. The Coordinator calls this once at startup. + /// Without it, phase timing is silently skipped. + pub fn set_transact_phase_metrics(&mut self, phase_metrics: prometheus::HistogramVec) { + self.transact_phase_metrics = Some(phase_metrics); + } + + /// Install a `HistogramVec` (label: `phase`) for sub-phase timing inside + /// `CatalogState::apply_updates`. + pub fn set_apply_updates_phase_metrics(&mut self, phase_metrics: prometheus::HistogramVec) { + self.state.set_apply_updates_phase_metrics(phase_metrics); + self.apply_updates_phase_metrics = self.state.apply_updates_phase_metrics().cloned(); + } + + /// Install a `HistogramVec` (label: `kind`) for per-update-kind timing + /// inside `CatalogState::apply_updates_inner`. + pub fn set_apply_update_kind_metrics(&mut self, phase_metrics: prometheus::HistogramVec) { + self.state.set_apply_update_kind_metrics(phase_metrics); + self.apply_update_kind_metrics = self.state.apply_update_kind_metrics().cloned(); + } + /// Set the optimized plan for the item identified by `id`. /// /// # Panics @@ -628,7 +657,7 @@ impl Catalog { .into_element(); // Drain transaction. let _ = txn.get_and_commit_op_updates(); - txn.commit(commit_ts).await?; + txn.commit(&mut **storage, commit_ts).await?; Ok(id) } @@ -1121,6 +1150,43 @@ impl Catalog { .filter(|entry| entry.is_secret() && entry.id().is_user()) } + /// Number of user items of the given [`SqlCatalogItemType`]. O(log N) on + /// the bucket map, vs. O(N) for the iterator-based `user_*()` methods. + pub fn user_item_count(&self, typ: SqlCatalogItemType) -> usize { + self.state.user_item_counts.get(&typ).copied().unwrap_or(0) + } + + pub fn user_tables_count(&self) -> usize { + self.user_item_count(SqlCatalogItemType::Table) + } + + pub fn user_sinks_count(&self) -> usize { + self.user_item_count(SqlCatalogItemType::Sink) + } + + pub fn user_materialized_views_count(&self) -> usize { + self.user_item_count(SqlCatalogItemType::MaterializedView) + } + + pub fn user_secrets_count(&self) -> usize { + self.user_item_count(SqlCatalogItemType::Secret) + } + + /// Sum of `user_controllable_persist_shard_count()` across all user + /// sources. This is what the `max_sources` limit is computed against. + pub fn user_source_shard_count(&self) -> usize { + self.state.user_source_shard_count + } + + /// Number of user connections of a given sub-kind. + pub fn user_connection_count(&self, kind: crate::catalog::state::UserConnectionKind) -> usize { + self.state + .user_connection_counts + .get(&kind) + .copied() + .unwrap_or(0) + } + pub fn get_network_policy(&self, network_policy_id: NetworkPolicyId) -> &NetworkPolicy { self.state.get_network_policy(&network_policy_id) } diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index 184eb14a8dcf7..dd32748845426 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -110,49 +110,69 @@ impl CatalogState { let mut builtin_table_updates = Vec::with_capacity(updates.len()); let mut catalog_updates = Vec::with_capacity(updates.len()); + let phase_metrics = self.apply_updates_phase_metrics.clone(); + // First, consolidate updates. The code that applies parsed state // updates _requires_ that the given updates are consolidated. There // must be at most one addition and/or one retraction for a given item, // as identified by that items ID type. - let updates = Self::consolidate_updates(updates); + let updates = { + let _t = phase_metrics + .as_ref() + .map(|m| m.with_label_values(&["consolidate_initial"]).start_timer()); + Self::consolidate_updates(updates) + }; // Apply updates in groups, according to their timestamps. let mut groups: Vec> = Vec::new(); - for (_, updates) in &updates.into_iter().chunk_by(|update| update.ts) { - // Bring the updates into the pseudo-topological order that we need - // for updating our in-memory state and generating builtin table - // updates. - let updates = sort_updates(updates.collect()); - groups.push(updates); + { + let _t = phase_metrics + .as_ref() + .map(|m| m.with_label_values(&["sort_per_group"]).start_timer()); + for (_, updates) in &updates.into_iter().chunk_by(|update| update.ts) { + // Bring the updates into the pseudo-topological order that we need + // for updating our in-memory state and generating builtin table + // updates. + let updates = sort_updates(updates.collect()); + groups.push(updates); + } } for updates in groups { let mut apply_state = ApplyState::Updates(Vec::new()); let mut retractions = InProgressRetractions::default(); - for update in updates { - let (next_apply_state, (builtin_table_update, catalog_update)) = apply_state - .step( - ApplyState::new(update), - self, - &mut retractions, - local_expression_cache, - ) + { + let _t = phase_metrics + .as_ref() + .map(|m| m.with_label_values(&["apply_updates_inner"]).start_timer()); + for update in updates { + let (next_apply_state, (builtin_table_update, catalog_update)) = apply_state + .step( + ApplyState::new(update), + self, + &mut retractions, + local_expression_cache, + ) + .await; + apply_state = next_apply_state; + builtin_table_updates.extend(builtin_table_update); + catalog_updates.extend(catalog_update); + } + + // Apply remaining state. + let (builtin_table_update, catalog_update) = apply_state + .apply(self, &mut retractions, local_expression_cache) .await; - apply_state = next_apply_state; builtin_table_updates.extend(builtin_table_update); catalog_updates.extend(catalog_update); } - // Apply remaining state. - let (builtin_table_update, catalog_update) = apply_state - .apply(self, &mut retractions, local_expression_cache) - .await; - builtin_table_updates.extend(builtin_table_update); - catalog_updates.extend(catalog_update); - // Clean up plans and optimizer notices for items that // were retracted but not replaced (i.e., truly dropped). + let _t = phase_metrics + .as_ref() + .map(|m| m.with_label_values(&["cleanup_notices"]).start_timer()); let dropped_entries: Vec = retractions .items .into_values() @@ -223,11 +243,18 @@ impl CatalogState { let mut builtin_table_updates = Vec::with_capacity(updates.len()); let mut catalog_updates = Vec::new(); + let kind_metrics = self.apply_update_kind_metrics.clone(); + for state_update in updates { if matches!(state_update.kind, StateUpdateKind::SystemConfiguration(_)) { update_system_config = true; } + let kind_label = state_update_kind_label(&state_update.kind); + let _kind_timer = kind_metrics + .as_ref() + .map(|m| m.with_label_values(&[kind_label]).start_timer()); + match state_update.diff { StateDiff::Retraction => { // We want the parsed catalog updates to match the state of @@ -1339,20 +1366,23 @@ impl CatalogState { ) { match diff { StateDiff::Addition => { - let newly_inserted = Arc::make_mut(&mut self.storage_metadata) + // `imbl::OrdSet::insert` returns the previous value as `Option` + // (`Some` if it was already present), unlike `BTreeSet::insert` + // which returns `bool` (`true` if newly inserted). + let prev = Arc::make_mut(&mut self.storage_metadata) .unfinalized_shards .insert(unfinalized_shard.shard); assert!( - newly_inserted, + prev.is_none(), "values must be explicitly retracted before inserting a new value: {unfinalized_shard:?}", ); } StateDiff::Retraction => { - let removed = Arc::make_mut(&mut self.storage_metadata) + let prev = Arc::make_mut(&mut self.storage_metadata) .unfinalized_shards .remove(&unfinalized_shard.shard); assert!( - removed, + prev.is_some(), "retraction does not match existing value: {unfinalized_shard:?}" ); } @@ -1908,9 +1938,28 @@ impl CatalogState { .expect("catalog out of sync") .bound_objects .insert(entry.id); + // `imbl::OrdSet::insert` returns the previous value as + // `Option`; we don't check it here because the + // pre-existing `BTreeSet` version didn't either. }; } + if entry.id.is_user() { + *self.user_item_counts.entry(entry.item.typ()).or_insert(0) += 1; + if let CatalogItem::Source(source) = &entry.item { + let shards: usize = source + .user_controllable_persist_shard_count() + .try_into() + .expect("non-negative shard count"); + self.user_source_shard_count += shards; + } + if let CatalogItem::Connection(connection) = &entry.item { + let kind = + crate::catalog::state::UserConnectionKind::from_details(&connection.details); + *self.user_connection_counts.entry(kind).or_insert(0) += 1; + } + } + for u in entry.references().items() { match self.entry_by_id.get_mut(u) { Some(metadata) => metadata.referenced_by.push(entry.id()), @@ -2036,14 +2085,51 @@ impl CatalogState { if !id.is_system() { if let Some(cluster_id) = metadata.item().cluster_id() { - assert!( - self.clusters_by_id - .get_mut(&cluster_id) - .expect("catalog out of sync") - .bound_objects - .remove(&id), - "catalog out of sync" - ); + let prev = self + .clusters_by_id + .get_mut(&cluster_id) + .expect("catalog out of sync") + .bound_objects + .remove(&id); + assert!(prev.is_some(), "catalog out of sync"); + } + } + + if id.is_user() { + let typ = metadata.item().typ(); + let count = self + .user_item_counts + .get_mut(&typ) + .expect("catalog out of sync: user item count missing"); + *count = count + .checked_sub(1) + .expect("catalog out of sync: count underflow"); + if *count == 0 { + self.user_item_counts.remove(&typ); + } + if let CatalogItem::Source(source) = metadata.item() { + let shards: usize = source + .user_controllable_persist_shard_count() + .try_into() + .expect("non-negative shard count"); + self.user_source_shard_count = self + .user_source_shard_count + .checked_sub(shards) + .expect("catalog out of sync: source shard count underflow"); + } + if let CatalogItem::Connection(connection) = metadata.item() { + let kind = + crate::catalog::state::UserConnectionKind::from_details(&connection.details); + let count = self + .user_connection_counts + .get_mut(&kind) + .expect("catalog out of sync: user connection count missing"); + *count = count + .checked_sub(1) + .expect("catalog out of sync: count underflow"); + if *count == 0 { + self.user_connection_counts.remove(&kind); + } } } @@ -2141,6 +2227,32 @@ impl CatalogState { /// /// # Panics /// +/// Stable static label per `StateUpdateKind` variant, for per-kind Prometheus +/// histogram labels. +fn state_update_kind_label(kind: &StateUpdateKind) -> &'static str { + match kind { + StateUpdateKind::Role(_) => "role", + StateUpdateKind::RoleAuth(_) => "role_auth", + StateUpdateKind::Database(_) => "database", + StateUpdateKind::Schema(_) => "schema", + StateUpdateKind::DefaultPrivilege(_) => "default_privilege", + StateUpdateKind::SystemPrivilege(_) => "system_privilege", + StateUpdateKind::SystemConfiguration(_) => "system_configuration", + StateUpdateKind::Cluster(_) => "cluster", + StateUpdateKind::NetworkPolicy(_) => "network_policy", + StateUpdateKind::IntrospectionSourceIndex(_) => "introspection_source_index", + StateUpdateKind::ClusterReplica(_) => "cluster_replica", + StateUpdateKind::SystemObjectMapping(_) => "system_object_mapping", + StateUpdateKind::TemporaryItem(_) => "temporary_item", + StateUpdateKind::Item(_) => "item", + StateUpdateKind::Comment(_) => "comment", + StateUpdateKind::SourceReferences(_) => "source_references", + StateUpdateKind::AuditLog(_) => "audit_log", + StateUpdateKind::StorageCollectionMetadata(_) => "storage_collection_metadata", + StateUpdateKind::UnfinalizedShard(_) => "unfinalized_shard", + } +} + /// This function assumes that all provided `updates` have the same timestamp and will panic /// otherwise. It also requires that the provided `updates` are consolidated, i.e. all contained /// `StateUpdateKinds` are unique. diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 62b2157c907f4..90fbb715d6c82 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -55,22 +55,22 @@ use crate::coord::catalog_implications::parsed_state_updates::ParsedStateUpdate; /// in writable mode, before any migrations have run. const MIGRATION_VERSION_KEY: &str = "migration_version"; -pub(crate) fn get_migration_version(txn: &Transaction<'_>) -> Option { +pub(crate) fn get_migration_version(txn: &Transaction) -> Option { txn.get_setting(MIGRATION_VERSION_KEY.into()) .map(|s| s.parse().expect("valid migration version")) } pub(crate) fn set_migration_version( - txn: &mut Transaction<'_>, + txn: &mut Transaction, version: Version, ) -> Result<(), mz_catalog::durable::CatalogError> { txn.set_setting(MIGRATION_VERSION_KEY.into(), Some(version.to_string())) } -fn rewrite_ast_items(tx: &mut Transaction<'_>, mut f: F) -> Result<(), anyhow::Error> +fn rewrite_ast_items(tx: &mut Transaction, mut f: F) -> Result<(), anyhow::Error> where F: for<'a> FnMut( - &'a mut Transaction<'_>, + &'a mut Transaction, CatalogItemId, &'a mut Statement, ) -> Result<(), anyhow::Error>, @@ -90,13 +90,13 @@ where } fn rewrite_items( - tx: &mut Transaction<'_>, + tx: &mut Transaction, cat: &ConnCatalog<'_>, mut f: F, ) -> Result<(), anyhow::Error> where F: for<'a> FnMut( - &'a mut Transaction<'_>, + &'a mut Transaction, &'a &ConnCatalog<'_>, CatalogItemId, &'a mut Statement, @@ -128,7 +128,7 @@ pub(crate) struct MigrateResult { /// Returns the builtin updates corresponding to all user items. pub(crate) async fn migrate( state: &mut CatalogState, - tx: &mut Transaction<'_>, + tx: &mut Transaction, local_expr_cache: &mut LocalExpressionCache, item_updates: Vec, _now: NowFn, @@ -338,7 +338,7 @@ pub(crate) async fn migrate( /// points to the original `source_name` but with the altered global id (which is now /// `progress_id`). fn rewrite_sources_to_tables( - tx: &mut Transaction<'_>, + tx: &mut Transaction, catalog: &ConnCatalog<'_>, ) -> Result<(), anyhow::Error> { use mz_sql::ast::{ @@ -995,7 +995,7 @@ fn ast_rewrite_sql_server_constraints(stmt: &mut Statement) -> Result<(), a /// Add missing item IDs to the ON clauses of CREATE INDEX statements. fn ast_rewrite_add_missing_index_ids( - tx: &Transaction<'_>, + tx: &Transaction, stmt: &mut Statement, ) -> Result<(), anyhow::Error> { let Statement::CreateIndex(stmt) = stmt else { diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 4107e26c9b97d..d65e6baccf3a9 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -134,6 +134,9 @@ impl Catalog { database_by_id: imbl::OrdMap::new(), entry_by_id: imbl::OrdMap::new(), entry_by_global_id: imbl::OrdMap::new(), + user_item_counts: imbl::OrdMap::new(), + user_source_shard_count: 0, + user_connection_counts: imbl::OrdMap::new(), notices_by_dep_id: imbl::OrdMap::new(), ambient_schemas_by_name: imbl::OrdMap::new(), ambient_schemas_by_id: imbl::OrdMap::new(), @@ -170,6 +173,8 @@ impl Catalog { aws_privatelink_availability_zones: config.aws_privatelink_availability_zones, http_host_name: config.http_host_name, license_key: config.license_key, + apply_updates_phase_metrics: None, + apply_update_kind_metrics: None, }; let deploy_generation = storage.get_deployment_generation().await?; @@ -510,7 +515,7 @@ impl Catalog { // Bump the migration version immediately before committing. set_migration_version(&mut txn, config.build_info.semver_version())?; - txn.commit(config.boot_ts).await?; + txn.commit(&mut **storage, config.boot_ts).await?; // Now that the migration is durable, run any requested deferred cleanup. schema_migration_result.cleanup_action.await; @@ -565,6 +570,9 @@ impl Catalog { expr_cache_handle, transient_revision: 1, storage: Arc::new(tokio::sync::Mutex::new(storage)), + transact_phase_metrics: None, + apply_updates_phase_metrics: None, + apply_update_kind_metrics: None, }; // Operators aren't stored in the catalog, but we would like them in @@ -643,12 +651,12 @@ impl Catalog { // `MZ_CATALOG_RAW` builtin source. let item_id = self.resolve_builtin_storage_collection(&MZ_CATALOG_RAW); let global_id = self.get_entry(&item_id).latest_global_id(); - match txn.get_collection_metadata().get(&global_id) { + match txn.get_collection_shard(global_id) { None => { txn.insert_collection_metadata([(global_id, shard_id)].into()) .map_err(mz_catalog::durable::DurableCatalogError::from)?; } - Some(id) => assert_eq!(*id, shard_id), + Some(id) => assert_eq!(id, shard_id), } storage_collections @@ -669,7 +677,7 @@ impl Catalog { "storage is not allowed to generate catalog changes that would change the catalog or controller state" ); let commit_ts = txn.upper(); - txn.commit(commit_ts).await?; + txn.commit(&mut **storage, commit_ts).await?; drop(storage); // Save updated state. @@ -699,7 +707,7 @@ impl Catalog { "initializing controller should not produce updates: {updates:?}" ); let commit_ts = tx.upper(); - tx.commit(commit_ts).await?; + tx.commit(&mut **storage, commit_ts).await?; let read_only_tx = storage.transaction().await?; @@ -743,7 +751,7 @@ impl CatalogState { /// /// Returns the list of new builtin [`GlobalId`]s. fn add_new_remove_old_builtin_items_migration( - txn: &mut mz_catalog::durable::Transaction<'_>, + txn: &mut mz_catalog::durable::Transaction, ) -> Result, mz_catalog::durable::CatalogError> { let mut new_builtin_mappings = Vec::new(); // Used to validate unique descriptions. @@ -957,7 +965,7 @@ fn add_new_remove_old_builtin_items_migration( } fn add_new_remove_old_builtin_clusters_migration( - txn: &mut mz_catalog::durable::Transaction<'_>, + txn: &mut mz_catalog::durable::Transaction, builtin_cluster_config_map: &BuiltinBootstrapClusterConfigMap, boot_ts: Timestamp, ) -> Result<(), mz_catalog::durable::CatalogError> { @@ -1032,7 +1040,7 @@ fn add_new_remove_old_builtin_clusters_migration( } fn add_new_remove_old_builtin_introspection_source_migration( - txn: &mut mz_catalog::durable::Transaction<'_>, + txn: &mut mz_catalog::durable::Transaction, ) -> Result<(), AdapterError> { let mut new_indexes = Vec::new(); let mut removed_indexes = BTreeSet::new(); @@ -1067,7 +1075,7 @@ fn add_new_remove_old_builtin_introspection_source_migration( } fn add_new_remove_old_builtin_roles_migration( - txn: &mut mz_catalog::durable::Transaction<'_>, + txn: &mut mz_catalog::durable::Transaction, ) -> Result<(), mz_catalog::durable::CatalogError> { let mut durable_roles: BTreeMap<_, _> = txn .get_roles() @@ -1097,7 +1105,7 @@ fn add_new_remove_old_builtin_roles_migration( } fn add_new_remove_old_builtin_cluster_replicas_migration( - txn: &mut Transaction<'_>, + txn: &mut Transaction, builtin_cluster_config_map: &BuiltinBootstrapClusterConfigMap, boot_ts: Timestamp, ) -> Result<(), AdapterError> { @@ -1217,7 +1225,7 @@ fn add_new_remove_old_builtin_cluster_replicas_migration( /// input is no longer valid. For example if we remove a configuration parameter or change the /// accepted set of values. fn remove_invalid_config_param_role_defaults_migration( - txn: &mut Transaction<'_>, + txn: &mut Transaction, ) -> Result<(), AdapterError> { static BUILD_INFO: mz_build_info::BuildInfo = mz_build_info::build_info!(); diff --git a/src/adapter/src/catalog/open/builtin_schema_migration.rs b/src/adapter/src/catalog/open/builtin_schema_migration.rs index bca1c1baf91c7..e77e08ddfb199 100644 --- a/src/adapter/src/catalog/open/builtin_schema_migration.rs +++ b/src/adapter/src/catalog/open/builtin_schema_migration.rs @@ -253,7 +253,7 @@ impl Default for MigrationResult { pub(super) async fn run( build_info: &BuildInfo, deploy_generation: u64, - txn: &mut Transaction<'_>, + txn: &mut Transaction, config: BuiltinItemMigrationConfig, ) -> Result { // Sanity check to ensure we're not touching durable state in read-only mode. @@ -271,13 +271,12 @@ pub(super) async fn run( }; let build_version = build_info.semver_version(); - let collection_metadata = txn.get_collection_metadata(); let system_objects = txn .get_system_object_mappings() .map(|m| { let object = m.description; let global_id = m.unique_identifier.global_id; - let shard_id = collection_metadata.get(&global_id).copied(); + let shard_id = txn.get_collection_shard(global_id); let Some((_, builtin)) = BUILTIN_LOOKUP.get(&object) else { panic!("missing builtin {object:?}"); }; @@ -346,7 +345,7 @@ impl Default for MigrationRunResult { impl MigrationRunResult { /// Apply this migration result to the given transaction. - fn apply(&self, txn: &mut Transaction<'_>) { + fn apply(&self, txn: &mut Transaction) { // Update collection metadata. let replaced_ids = self.new_shards.keys().copied().collect(); let old_metadata = txn.delete_collection_metadata(replaced_ids); diff --git a/src/adapter/src/catalog/state.rs b/src/adapter/src/catalog/state.rs index 27dd43fc1469f..dd8eeb67d0652 100644 --- a/src/adapter/src/catalog/state.rs +++ b/src/adapter/src/catalog/state.rs @@ -109,6 +109,43 @@ use crate::session::Session; /// /// [`Serialize`] is implemented to create human readable dumps of the in-memory state, not for /// storing the contents of this struct on disk. +/// +/// # A note on persistent collections (`imbl::OrdMap` / `imbl::OrdSet` / `imbl::Vector`) +/// +/// Almost every collection field on `CatalogState` — and on the value +/// types stored inside those collections (`Database`, `Schema`, `Cluster`, +/// `Role`, `CatalogEntry`, `SourceReferences`, `StorageMetadata`, …) — +/// is an `imbl::*` persistent collection rather than a `std::collections` +/// one. This is load-bearing: `Catalog::transact_inner` builds two +/// `Cow` handles (`preliminary_state` and `state`) and +/// calls `to_mut()` on each, so all `imbl::OrdMap`s end up shared +/// across two `CatalogState`s during a single DDL. Any later +/// `get_mut(...)` on a shared `imbl::OrdMap` triggers a leaf path-copy +/// that **clones every value in the affected B-tree leaf**, not just +/// the targeted one. If any of those values embeds a non-persistent +/// inner collection (`BTreeMap`, `BTreeSet`, `Vec`, …), that inner +/// collection gets deep-cloned in O(M) — and if M scales with the +/// workload (e.g. items in a schema), the per-DDL cost scales too. +/// +/// Concretely: the audit_pad-scalability bench (`test/envd-ddl-scalability/`) +/// caught two ~5 ms/DDL N-dependent slopes at N=15k tables that were +/// nothing more than `Arc::make_mut`/`imbl` leaf-copies deep-cloning +/// non-persistent inner collections (`Schema.items: BTreeMap` and +/// `StorageMetadata.collection_metadata: BTreeMap`). Switching those +/// inners to `imbl::OrdMap` made the leaf-copy O(1) (persistent +/// tree-root refcount bump) and erased the slope. +/// +/// **Rule:** when adding a field whose value type lives inside any +/// of these `imbl::OrdMap`s (or behind an `Arc` we `make_mut`), use +/// a persistent collection (`imbl::OrdMap`, `imbl::OrdSet`, +/// `imbl::Vector`) for any sub-field that can grow with the +/// workload. Two intentional holdouts exist: +/// * `Cluster.log_indexes` (`BTreeMap`) — has +/// a tight `arranged_logs: BTreeMap` API +/// contract with the compute controller and is bounded by the +/// number of log variants (~10). +/// * `*_by_name_` legacy fields elsewhere — see comments at the +/// call sites. #[derive(Debug, Clone, Serialize)] pub struct CatalogState { // State derived from the durable catalog. These fields should only be mutated in `open.rs` or @@ -123,6 +160,15 @@ pub struct CatalogState { pub(super) entry_by_id: imbl::OrdMap, #[serde(serialize_with = "mz_ore::serde::map_key_to_string")] pub(super) entry_by_global_id: imbl::OrdMap, + // Bucketed counts derived from `entry_by_id`, maintained by + // `insert_entry`/`drop_item`. Used by `validate_resource_limits` to avoid + // scanning the full entry map on every DDL. + #[serde(skip)] + pub(super) user_item_counts: imbl::OrdMap, + #[serde(skip)] + pub(super) user_source_shard_count: usize, + #[serde(skip)] + pub(super) user_connection_counts: imbl::OrdMap, pub(super) ambient_schemas_by_name: imbl::OrdMap, #[serde(serialize_with = "mz_ore::serde::map_key_to_string")] pub(super) ambient_schemas_by_id: imbl::OrdMap, @@ -178,6 +224,59 @@ pub struct CatalogState { // Read-only not derived from the durable catalog. #[serde(skip)] pub(super) license_key: ValidatedLicenseKey, + + /// Installed by the Coordinator at startup to time sub-phases of + /// `apply_updates`. `None` in tests / debug catalogs. + #[serde(skip)] + pub(super) apply_updates_phase_metrics: Option, + /// Installed by the Coordinator at startup to time individual update kinds + /// inside `apply_updates_inner`. `None` in tests / debug catalogs. + #[serde(skip)] + pub(super) apply_update_kind_metrics: Option, +} + +impl CatalogState { + pub fn set_apply_updates_phase_metrics(&mut self, phase_metrics: prometheus::HistogramVec) { + self.apply_updates_phase_metrics = Some(phase_metrics); + } + pub fn apply_updates_phase_metrics(&self) -> Option<&prometheus::HistogramVec> { + self.apply_updates_phase_metrics.as_ref() + } + pub fn set_apply_update_kind_metrics(&mut self, phase_metrics: prometheus::HistogramVec) { + self.apply_update_kind_metrics = Some(phase_metrics); + } + pub fn apply_update_kind_metrics(&self) -> Option<&prometheus::HistogramVec> { + self.apply_update_kind_metrics.as_ref() + } +} + +/// Sub-classification of user connections that we maintain bucketed counts +/// for, mirroring the variants of [`mz_sql::plan::ConnectionDetails`] that +/// `validate_resource_limits` checks. Variants we don't limit (Csr, Ssh, Aws, +/// IcebergCatalog) are folded into `Other` so the bucket map covers all +/// connection items. +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub enum UserConnectionKind { + Kafka, + Postgres, + MySql, + SqlServer, + AwsPrivatelink, + Other, +} + +impl UserConnectionKind { + pub(super) fn from_details(details: &mz_sql::plan::ConnectionDetails) -> Self { + use mz_sql::plan::ConnectionDetails::*; + match details { + Kafka(_) => UserConnectionKind::Kafka, + Postgres(_) => UserConnectionKind::Postgres, + MySql(_) => UserConnectionKind::MySql, + SqlServer(_) => UserConnectionKind::SqlServer, + AwsPrivatelink(_) => UserConnectionKind::AwsPrivatelink, + Csr(_) | Ssh { .. } | Aws(_) | IcebergCatalog(_) => UserConnectionKind::Other, + } + } } /// Keeps track of what expressions are cached or not during startup. @@ -293,6 +392,9 @@ impl CatalogState { database_by_id: Default::default(), entry_by_id: Default::default(), entry_by_global_id: Default::default(), + user_item_counts: Default::default(), + user_source_shard_count: 0, + user_connection_counts: Default::default(), notices_by_dep_id: Default::default(), ambient_schemas_by_name: Default::default(), ambient_schemas_by_id: Default::default(), @@ -331,6 +433,8 @@ impl CatalogState { storage_metadata: Arc::new(StorageMetadata::default()), license_key: ValidatedLicenseKey::for_tests(), mock_authentication_nonce: Default::default(), + apply_updates_phase_metrics: None, + apply_update_kind_metrics: None, } } @@ -1831,9 +1935,9 @@ impl CatalogState { }, id: SchemaSpecifier::Temporary, oid, - items: BTreeMap::new(), - functions: BTreeMap::new(), - types: BTreeMap::new(), + items: imbl::OrdMap::new(), + functions: imbl::OrdMap::new(), + types: imbl::OrdMap::new(), owner_id, privileges: PrivilegeMap::from_mz_acl_items(vec![rbac::owner_privilege( mz_sql::catalog::ObjectType::Schema, @@ -2143,7 +2247,7 @@ impl CatalogState { #[allow(clippy::useless_let_if_seq)] pub fn resolve( &self, - get_schema_entries: fn(&Schema) -> &BTreeMap, + get_schema_entries: fn(&Schema) -> &imbl::OrdMap, current_database: Option<&DatabaseId>, search_path: &Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)>, name: &PartialItemName, diff --git a/src/adapter/src/catalog/transact.rs b/src/adapter/src/catalog/transact.rs index 1ddd26068e549..1a5c7daa78ae5 100644 --- a/src/adapter/src/catalog/transact.rs +++ b/src/adapter/src/catalog/transact.rs @@ -27,7 +27,7 @@ use mz_audit_log::{ }; use mz_catalog::SYSTEM_CONN_ID; use mz_catalog::builtin::BuiltinLog; -use mz_catalog::durable::{NetworkPolicy, Snapshot, Transaction}; +use mz_catalog::durable::{DurableCatalogData, NetworkPolicy, Transaction}; use mz_catalog::expr_cache::LocalExpressions; use mz_catalog::memory::error::{AmbiguousRename, Error, ErrorKind}; use mz_catalog::memory::objects::{ @@ -469,35 +469,47 @@ impl Catalog { .await .unwrap_or_terminate("starting catalog transaction"); - let new_state = Self::transact_inner( - TransactInnerMode::Commit, - storage_collections, - oracle_write_ts, - session, - ops, - temporary_ids, - &mut builtin_table_updates, - &mut catalog_updates, - &mut audit_events, - &mut tx, - &self.state, - ) - .await?; + let phase_metrics = self.transact_phase_metrics.as_ref(); + + let new_state = { + let _t = phase_metrics.map(|m| m.with_label_values(&["transact_inner"]).start_timer()); + Self::transact_inner( + TransactInnerMode::Commit, + storage_collections, + oracle_write_ts, + session, + ops, + temporary_ids, + &mut builtin_table_updates, + &mut catalog_updates, + &mut audit_events, + &mut tx, + &self.state, + phase_metrics, + ) + .await? + }; // The user closure was successful, apply the updates. Terminate the // process if this fails, because we have to restart envd due to // indeterminate catalog state, which we only reconcile during catalog // init. - tx.commit(oracle_write_ts) - .await - .unwrap_or_terminate("catalog storage transaction commit must succeed"); + { + let _t = phase_metrics.map(|m| m.with_label_values(&["tx_commit"]).start_timer()); + tx.commit(&mut **storage, oracle_write_ts) + .await + .unwrap_or_terminate("catalog storage transaction commit must succeed"); + } // Dropping here keeps the mutable borrow on self, preventing us accidentally // mutating anything until after f is executed. drop(storage); - if let Some(new_state) = new_state { - self.transient_revision += 1; - self.state = new_state; + { + let _t = phase_metrics.map(|m| m.with_label_values(&["assign_state"]).start_timer()); + if let Some(new_state) = new_state { + self.transient_revision += 1; + self.state = new_state; + } } Ok(TransactionResult { @@ -514,21 +526,21 @@ impl Catalog { /// The durable transaction is intentionally never committed and no storage /// controller prepare-state side effects are run. /// - /// If `prev_snapshot` is `Some`, the transaction is initialized from that - /// snapshot (which represents the tx state after the previous dry run), - /// ensuring it starts in sync with `base_state`. If `None` (first - /// statement), a fresh transaction is loaded from durable storage. + /// If `prev_durable_data` is `Some`, the transaction is initialized from + /// it (representing the tx state after the previous dry run), ensuring it + /// starts in sync with `base_state`. If `None` (first statement), a fresh + /// transaction is loaded from durable storage. /// - /// Returns the new accumulated state and a snapshot of the transaction's - /// state for use in subsequent incremental dry runs. + /// Returns the new accumulated state and the transaction's current + /// durable data for use in subsequent incremental dry runs. pub async fn transact_incremental_dry_run( &self, base_state: &CatalogState, ops: Vec, session: Option<&ConnMeta>, - prev_snapshot: Option, + prev_durable_data: Option, oracle_write_ts: mz_repr::Timestamp, - ) -> Result<(CatalogState, Snapshot), AdapterError> { + ) -> Result<(CatalogState, DurableCatalogData), AdapterError> { // For DDL transactions, items are not temporary (CREATE TABLE FROM SOURCE, etc.) // but we still need to check for collisions. let temporary_ids = self.temporary_ids(&ops, BTreeSet::new())?; @@ -537,12 +549,12 @@ impl Catalog { let mut catalog_updates = vec![]; let mut audit_events = vec![]; let mut storage = self.storage().await; - let mut tx = if let Some(snapshot) = prev_snapshot { - // Restore transaction from saved snapshot so it starts in sync + let mut tx = if let Some(data) = prev_durable_data { + // Restore transaction from saved durable data so it starts in sync // with the accumulated CatalogState from previous dry runs. storage - .transaction_from_snapshot(snapshot) - .unwrap_or_terminate("starting catalog transaction from snapshot") + .transaction_from_durable_data(data) + .unwrap_or_terminate("starting catalog transaction from durable data") } else { // First statement: fresh transaction from durable storage, which // is in sync with the real catalog state. @@ -565,19 +577,22 @@ impl Catalog { &mut audit_events, &mut tx, base_state, + // Dry runs are a different code path and we don't want them + // polluting the bench measurements; skip phase metrics. + None, ) .await?; - // Save the transaction's current state as a snapshot for the next + // Save the transaction's current state as durable data for the next // incremental dry run. - let new_snapshot = tx.current_snapshot(); + let new_durable_data = tx.current_durable_data(); // Transaction is NOT committed — drop it. drop(storage); // transact_inner returns Some(state) when ops produced changes. let state = new_state.unwrap_or_else(|| base_state.clone()); - Ok((state, new_snapshot)) + Ok((state, new_durable_data)) } /// Extracts optimized expressions from `Op::CreateItem` operations for views @@ -644,8 +659,9 @@ impl Catalog { builtin_table_updates: &mut Vec, parsed_catalog_updates: &mut Vec, audit_events: &mut Vec, - tx: &mut Transaction<'_>, + tx: &mut Transaction, state: &CatalogState, + phase_metrics: Option<&prometheus::HistogramVec>, ) -> Result, AdapterError> { // We come up with new catalog state, builtin state updates, and parsed // catalog updates (for deriving catalog implications) in two phases: @@ -693,6 +709,7 @@ impl Catalog { let mut updates = Vec::new(); + let _op_loop_timer = phase_metrics.map(|m| m.with_label_values(&["op_loop"]).start_timer()); for op in ops { let (weird_builtin_table_update, temporary_item_updates) = Self::transact_op( oracle_write_ts, @@ -747,53 +764,67 @@ impl Catalog { } updates.append(&mut op_updates); } + drop(_op_loop_timer); - if !updates.is_empty() { - let mut local_expr_cache = LocalExpressionCache::new(cached_exprs.clone()); - let (op_builtin_table_updates, op_catalog_updates) = state - .to_mut() - .apply_updates(updates.clone(), &mut local_expr_cache) - .await; - let op_builtin_table_updates = state - .to_mut() - .resolve_builtin_table_updates(op_builtin_table_updates); - builtin_table_updates.extend(op_builtin_table_updates); - parsed_catalog_updates.extend(op_catalog_updates); + { + let _t = + phase_metrics.map(|m| m.with_label_values(&["final_apply_updates"]).start_timer()); + if !updates.is_empty() { + let mut local_expr_cache = LocalExpressionCache::new(cached_exprs.clone()); + let (op_builtin_table_updates, op_catalog_updates) = state + .to_mut() + .apply_updates(updates.clone(), &mut local_expr_cache) + .await; + let op_builtin_table_updates = state + .to_mut() + .resolve_builtin_table_updates(op_builtin_table_updates); + builtin_table_updates.extend(op_builtin_table_updates); + parsed_catalog_updates.extend(op_catalog_updates); + } } - match mode { - TransactInnerMode::Commit => { - // `storage_collections` can be `None` in tests. - if let Some(c) = storage_collections { - c.prepare_state( - tx, - storage_collections_to_create, - storage_collections_to_drop, - storage_collections_to_register, - ) - .await?; + { + let _t = phase_metrics.map(|m| m.with_label_values(&["prepare_state"]).start_timer()); + match mode { + TransactInnerMode::Commit => { + // `storage_collections` can be `None` in tests. + if let Some(c) = storage_collections { + c.prepare_state( + tx, + storage_collections_to_create, + storage_collections_to_drop, + storage_collections_to_register, + ) + .await?; + } + } + TransactInnerMode::DryRun => { + debug_assert!( + storage_collections.is_none(), + "dry-run mode must not prepare storage state" + ); } - } - TransactInnerMode::DryRun => { - debug_assert!( - storage_collections.is_none(), - "dry-run mode must not prepare storage state" - ); } } - let updates = tx.get_and_commit_op_updates(); - if !updates.is_empty() { - let mut local_expr_cache = LocalExpressionCache::new(cached_exprs.clone()); - let (op_builtin_table_updates, op_catalog_updates) = state - .to_mut() - .apply_updates(updates.clone(), &mut local_expr_cache) - .await; - let op_builtin_table_updates = state - .to_mut() - .resolve_builtin_table_updates(op_builtin_table_updates); - builtin_table_updates.extend(op_builtin_table_updates); - parsed_catalog_updates.extend(op_catalog_updates); + { + let _t = phase_metrics.map(|m| { + m.with_label_values(&["post_prepare_apply_updates"]) + .start_timer() + }); + let updates = tx.get_and_commit_op_updates(); + if !updates.is_empty() { + let mut local_expr_cache = LocalExpressionCache::new(cached_exprs.clone()); + let (op_builtin_table_updates, op_catalog_updates) = state + .to_mut() + .apply_updates(updates.clone(), &mut local_expr_cache) + .await; + let op_builtin_table_updates = state + .to_mut() + .resolve_builtin_table_updates(op_builtin_table_updates); + builtin_table_updates.extend(op_builtin_table_updates); + parsed_catalog_updates.extend(op_catalog_updates); + } } match state { @@ -816,7 +847,7 @@ impl Catalog { op: Op, temporary_ids: &BTreeSet, audit_events: &mut Vec, - tx: &mut Transaction<'_>, + tx: &mut Transaction, state: &CatalogState, storage_collections_to_create: &mut BTreeSet, storage_collections_to_drop: &mut BTreeSet, @@ -2788,7 +2819,7 @@ impl Catalog { /// for catalog items. We currently think that there are no use cases that require this assumption, /// but no way to know for sure. fn tx_replace_item( - tx: &mut Transaction<'_>, + tx: &mut Transaction, state: &CatalogState, id: CatalogItemId, new_entry: CatalogEntry, @@ -3225,8 +3256,8 @@ mod tests { // --- Path A: Incremental (two separate dry-run calls) --- - // First call: only op_t1, no previous snapshot. - let (state_after_t1, snapshot_after_t1) = catalog + // First call: only op_t1, no previous durable data. + let (state_after_t1, data_after_t1) = catalog .transact_incremental_dry_run( &base_state, vec![op_t1.clone()], @@ -3255,13 +3286,13 @@ mod tests { "t2 should NOT exist after first dry run" ); - // Second call: only op_t2, using state/snapshot from first call. + // Second call: only op_t2, using state/durable_data from first call. let (state_incremental, _) = catalog .transact_incremental_dry_run( &state_after_t1, vec![op_t2.clone()], None, - Some(snapshot_after_t1), + Some(data_after_t1), oracle_write_ts, ) .await diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 97ff33ad5f5f9..c8cc3df0d6ca9 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -4688,6 +4688,14 @@ pub fn serve( handle.block_on(epoch_millis_oracle.apply_write(boot_ts)); } + catalog.set_transact_phase_metrics(metrics.catalog_transact_phase_seconds.clone()); + catalog.set_apply_updates_phase_metrics( + metrics.catalog_apply_updates_phase_seconds.clone(), + ); + catalog.set_apply_update_kind_metrics( + metrics.catalog_apply_update_kind_seconds.clone(), + ); + let catalog = Arc::new(catalog); let caching_secrets_reader = CachingSecretsReader::new(secrets_controller.reader()); diff --git a/src/adapter/src/coord/appends.rs b/src/adapter/src/coord/appends.rs index dc9db6c7deb47..0c325dcf66100 100644 --- a/src/adapter/src/coord/appends.rs +++ b/src/adapter/src/coord/appends.rs @@ -504,15 +504,6 @@ impl Coordinator { } } - // Add table advancements for all tables. - let table_advancement_start = Instant::now(); - for table in self.catalog().entries().filter(|entry| entry.is_table()) { - appends.entry(table.id()).or_default(); - } - self.metrics - .group_commit_table_advancement_seconds - .observe(table_advancement_start.elapsed().as_secs_f64()); - // Consolidate all Rows for a given table. We do not consolidate the // staged batches, that's up to whoever staged them. let mut all_appends = Vec::with_capacity(appends.len()); @@ -556,8 +547,15 @@ impl Coordinator { "Appending to tables, {modified_tables:?}, at {timestamp}, advancing to {advance_to}" ); } + // Instrument our table writes since they can block the coordinator. let histogram = self.metrics.append_table_duration_seconds.clone(); + + // NOTE: It is important that we append, even when there are no actual + // appends. This makes sure we periodically bump the upper of all + // tables, which is required to make them readable at the latest oracle + // read ts. + let append_fut = self .controller .storage diff --git a/src/adapter/src/coord/catalog_implications.rs b/src/adapter/src/coord/catalog_implications.rs index 67a5f48ba1a96..1f70ab9a1b7c9 100644 --- a/src/adapter/src/coord/catalog_implications.rs +++ b/src/adapter/src/coord/catalog_implications.rs @@ -87,6 +87,13 @@ impl Coordinator { catalog_updates: Vec, ) -> Result<(), AdapterError> { let start = Instant::now(); + let phase_metric = self + .metrics + .apply_catalog_implications_phase_seconds + .clone(); + let _absorb_timer = phase_metric + .with_label_values(&["absorb_updates"]) + .start_timer(); let mut catalog_implications: BTreeMap = BTreeMap::new(); let mut cluster_commands: BTreeMap = BTreeMap::new(); @@ -162,6 +169,11 @@ impl Coordinator { } } + drop(_absorb_timer); + + let _inner_timer = phase_metric + .with_label_values(&["inner_total"]) + .start_timer(); self.apply_catalog_implications_inner( ctx, catalog_implications.into_iter().collect_vec(), @@ -170,6 +182,7 @@ impl Coordinator { introspection_source_indexes, ) .await?; + drop(_inner_timer); self.metrics .apply_catalog_implications_seconds @@ -187,6 +200,10 @@ impl Coordinator { cluster_replica_commands: Vec<((ClusterId, ReplicaId), CatalogImplication)>, mut introspection_source_indexes: BTreeMap>, ) -> Result<(), AdapterError> { + let phase_metric = self + .metrics + .apply_catalog_implications_phase_seconds + .clone(); let mut tables_to_drop = BTreeSet::new(); let mut sources_to_drop = vec![]; let mut replication_slots_to_drop: Vec<(PostgresConnection, String)> = vec![]; @@ -235,6 +252,9 @@ impl Coordinator { // just a log message. Over the next couple of PRs all of these will go // away. + let _item_loop_timer = phase_metric + .with_label_values(&["inner_item_loop"]) + .start_timer(); for (catalog_id, implication) in implications { tracing::trace!(?implication, "have to apply catalog implication"); @@ -550,7 +570,11 @@ impl Coordinator { } } } + drop(_item_loop_timer); + let _cluster_loop_timer = phase_metric + .with_label_values(&["inner_cluster_loops"]) + .start_timer(); for (cluster_id, command) in cluster_commands { tracing::trace!(?command, "have cluster command to apply!"); @@ -670,7 +694,11 @@ impl Coordinator { } } } + drop(_cluster_loop_timer); + let _controller_setup_timer = phase_metric + .with_label_values(&["inner_controller_setup"]) + .start_timer(); if !source_collections_to_create.is_empty() { self.create_source_collections(source_collections_to_create) .await?; @@ -740,7 +768,11 @@ impl Coordinator { .await .unwrap_or_terminate("cannot fail to alter ingestion source desc"); } + drop(_controller_setup_timer); + let _dep_scan_timer = phase_metric + .with_label_values(&["inner_dependency_scan"]) + .start_timer(); // Apply source drop overwrites. sources_to_drop.retain(|(_, gid)| !source_gids_to_keep.contains(gid)); @@ -825,54 +857,62 @@ impl Coordinator { .chain(tables_to_drop.iter().map(|(_id, gid)| gid)) .copied() .collect(); - let compute_gids_to_drop: Vec<_> = indexes_to_drop + let compute_gids_to_drop: BTreeSet<_> = indexes_to_drop .iter() .chain(compute_sinks_to_drop.iter()) .copied() .collect(); + let clusters_to_drop_set: BTreeSet<_> = clusters_to_drop.iter().copied().collect(); // Gather resources that we have to remove from timeline state and // pre-check if any Timelines become empty, when we drop the specified // storage and compute resources. // // Note: We only apply these changes below. - let mut timeline_id_bundles = BTreeMap::new(); + let mut timeline_associations = BTreeMap::new(); for (timeline, TimelineState { read_holds, .. }) in &self.global_timelines { let mut id_bundle = CollectionIdBundle::default(); - for storage_id in read_holds.storage_ids() { - if storage_gids_to_drop.contains(&storage_id) { - id_bundle.storage_ids.insert(storage_id); + // Probe the (small) drop sets against read_holds, which can be O(N). + for storage_id in &storage_gids_to_drop { + if read_holds.storage_holds.contains_key(storage_id) { + id_bundle.storage_ids.insert(*storage_id); } } - - for (instance_id, id) in read_holds.compute_ids() { - if compute_gids_to_drop.contains(&(instance_id, id)) - || clusters_to_drop.contains(&instance_id) - { + for (instance_id, gid) in &compute_gids_to_drop { + if read_holds.compute_holds.contains_key(&(*instance_id, *gid)) { id_bundle .compute_ids - .entry(instance_id) + .entry(*instance_id) .or_default() - .insert(id); + .insert(*gid); + } + } + // Whole-cluster drops are rare, so the linear scan here is fine. + if !clusters_to_drop_set.is_empty() { + for (instance_id, gid) in read_holds.compute_ids() { + if clusters_to_drop_set.contains(&instance_id) { + id_bundle + .compute_ids + .entry(instance_id) + .or_default() + .insert(gid); + } } } - timeline_id_bundles.insert(timeline.clone(), id_bundle); - } - - let mut timeline_associations = BTreeMap::new(); - for (timeline, id_bundle) in timeline_id_bundles.into_iter() { - let TimelineState { read_holds, .. } = self - .global_timelines - .get(&timeline) - .expect("all timelines have a timestamp oracle"); - - let empty = read_holds.id_bundle().difference(&id_bundle).is_empty(); - timeline_associations.insert(timeline, (empty, id_bundle)); + let bundle_compute_len: usize = + id_bundle.compute_ids.values().map(|ids| ids.len()).sum(); + let empty = read_holds.storage_holds.len() == id_bundle.storage_ids.len() + && read_holds.compute_holds.len() == bundle_compute_len; + timeline_associations.insert(timeline.clone(), (empty, id_bundle)); } + drop(_dep_scan_timer); + let _finalize_timer = phase_metric + .with_label_values(&["inner_finalize"]) + .start_timer(); // No error returns are allowed after this point. Enforce this at compile time // by using this odd structure so we don't accidentally add a stray `?`. let _: () = async { @@ -931,7 +971,7 @@ impl Coordinator { } if !compute_gids_to_drop.is_empty() { - self.drop_compute_collections(compute_gids_to_drop); + self.drop_compute_collections(compute_gids_to_drop.into_iter().collect()); } if !vpc_endpoints_to_drop.is_empty() { @@ -1040,8 +1080,18 @@ impl Coordinator { table_collections_to_create: BTreeMap, execution_timestamps_to_set: BTreeSet, ) -> Result<(), AdapterError> { + let phase_metric = self + .metrics + .apply_catalog_implications_phase_seconds + .clone(); + // If we have tables, determine the initial validity for the table. - let write_ts = self.get_local_write_ts().await; + let write_ts = { + let _t = phase_metric + .with_label_values(&["create_table_write_ts"]) + .start_timer(); + self.get_local_write_ts().await + }; let register_ts = write_ts.timestamp; // After acquiring `register_ts` but before using it, we need to @@ -1053,10 +1103,15 @@ impl Coordinator { // is readable at the oracle read ts after we bump it to the // `register_ts` below. Both of these needs are served by calling // `advance_upper`. - self.catalog - .advance_upper(write_ts.advance_to) - .await - .unwrap_or_terminate("unable to advance catalog upper"); + { + let _t = phase_metric + .with_label_values(&["create_table_advance_upper"]) + .start_timer(); + self.catalog + .advance_upper(write_ts.advance_to) + .await + .unwrap_or_terminate("unable to advance catalog upper"); + } for id in execution_timestamps_to_set { self.set_statement_execution_timestamp(id, register_ts); @@ -1064,17 +1119,27 @@ impl Coordinator { let storage_metadata = self.catalog.state().storage_metadata(); - self.controller - .storage - .create_collections( - storage_metadata, - Some(register_ts), - table_collections_to_create.into_iter().collect_vec(), - ) - .await - .unwrap_or_terminate("cannot fail to create collections"); + { + let _t = phase_metric + .with_label_values(&["create_table_storage_create_collections"]) + .start_timer(); + self.controller + .storage + .create_collections( + storage_metadata, + Some(register_ts), + table_collections_to_create.into_iter().collect_vec(), + ) + .await + .unwrap_or_terminate("cannot fail to create collections"); + } - self.apply_local_write(register_ts).await; + { + let _t = phase_metric + .with_label_values(&["create_table_apply_local_write"]) + .start_timer(); + self.apply_local_write(register_ts).await; + } Ok(()) } diff --git a/src/adapter/src/coord/command_handler.rs b/src/adapter/src/coord/command_handler.rs index 629f9b0f7867f..be752558be806 100644 --- a/src/adapter/src/coord/command_handler.rs +++ b/src/adapter/src/coord/command_handler.rs @@ -1267,7 +1267,7 @@ impl Coordinator { state, revision, side_effects: vec![], - snapshot: None, + durable_data: None, }) { return ctx.retire(Err(err)); } diff --git a/src/adapter/src/coord/ddl.rs b/src/adapter/src/coord/ddl.rs index dfd1cc0dc72e1..46c7746bed10a 100644 --- a/src/adapter/src/coord/ddl.rs +++ b/src/adapter/src/coord/ddl.rs @@ -50,7 +50,9 @@ use serde_json::json; use tracing::{Instrument, Level, event, info_span, warn}; use crate::active_compute_sink::{ActiveComputeSink, ActiveComputeSinkRetireReason}; -use crate::catalog::{DropObjectInfo, Op, ReplicaCreateDropReason, TransactionResult}; +use crate::catalog::{ + DropObjectInfo, Op, ReplicaCreateDropReason, TransactionResult, UserConnectionKind, +}; use crate::coord::Coordinator; use crate::coord::appends::BuiltinTableAppendNotify; use crate::coord::catalog_implications::parsed_state_updates::ParsedStateUpdate; @@ -226,7 +228,7 @@ impl Coordinator { ops: txn_ops, revision: txn_revision, state: txn_state, - snapshot: txn_snapshot, + durable_data: txn_durable_data, side_effects: _, }, .. @@ -254,7 +256,7 @@ impl Coordinator { // Clone what we need from the session before taking &mut below. let txn_ops_clone = txn_ops.clone(); let txn_state_clone = txn_state.clone(); - let prev_snapshot = txn_snapshot.clone(); + let prev_durable_data = txn_durable_data.clone(); // Validate resource limits with all accumulated + new ops (cheap O(N) counting). let mut combined_ops = txn_ops_clone; @@ -273,13 +275,13 @@ impl Coordinator { // initialize the transaction so it starts in sync with the accumulated // state. Otherwise (first statement), the fresh durable transaction is // already in sync with the real catalog state. - let (new_state, new_snapshot) = self + let (new_state, new_durable_data) = self .catalog() .transact_incremental_dry_run( &txn_state_clone, ops.clone(), conn, - prev_snapshot, + prev_durable_data, oracle_write_ts, ) .await?; @@ -293,7 +295,7 @@ impl Coordinator { state: new_state, side_effects: vec![Box::new(side_effect)], revision: self.catalog().transient_revision(), - snapshot: Some(new_snapshot), + durable_data: Some(new_durable_data), }); self.metrics @@ -313,6 +315,17 @@ impl Coordinator { conn_id: Option<&ConnectionId>, ops: Vec, ) -> Result<(BuiltinTableAppendNotify, Vec), AdapterError> { + let _coord_inner_timer = self + .metrics + .catalog_transact_phase_seconds + .with_label_values(&["coord_inner_total"]) + .start_timer(); + let _coord_pre_transact_timer = self + .metrics + .catalog_transact_phase_seconds + .with_label_values(&["coord_pre_transact"]) + .start_timer(); + if self.controller.read_only() { return Err(AdapterError::ReadOnly); } @@ -465,6 +478,9 @@ impl Coordinator { // regress or pause for 10s. let oracle_write_ts = self.get_local_write_ts().await.timestamp; + // Cloned out so we can keep using it after the split-borrow below. + let phase_metric = self.metrics.catalog_transact_phase_seconds.clone(); + let Coordinator { catalog, active_conns, @@ -472,8 +488,14 @@ impl Coordinator { cluster_replica_statuses, .. } = self; - let catalog = Arc::make_mut(catalog); + let catalog = { + let _t = phase_metric + .with_label_values(&["coord_arc_make_mut"]) + .start_timer(); + Arc::make_mut(catalog) + }; let conn = conn_id.map(|id| active_conns.get(id).expect("connection must exist")); + drop(_coord_pre_transact_timer); let TransactionResult { builtin_table_updates, @@ -488,6 +510,10 @@ impl Coordinator { ) .await?; + let _coord_post_transact_timer = phase_metric + .with_label_values(&["coord_post_transact"]) + .start_timer(); + for (cluster_id, replica_id) in &cluster_replicas_to_drop { cluster_replica_statuses.remove_cluster_replica_statuses(cluster_id, replica_id); } @@ -513,10 +539,18 @@ impl Coordinator { // Append our builtin table updates, then return the notify so we can run other tasks in // parallel. - let (builtin_update_notify, _) = self - .builtin_table_update() - .execute(builtin_table_updates) - .await; + let (builtin_update_notify, _) = { + let _t = phase_metric + .with_label_values(&["coord_builtin_table_execute"]) + .start_timer(); + self.builtin_table_update() + .execute(builtin_table_updates) + .await + }; + + let _coord_finalize_timer = phase_metric + .with_label_values(&["coord_finalize"]) + .start_timer(); // No error returns are allowed after this point. Enforce this at compile time // by using this odd structure so we don't accidentally add a stray `?`. @@ -1292,96 +1326,69 @@ impl Coordinator { } } - let mut current_aws_privatelink_connections = 0; - let mut current_postgres_connections = 0; - let mut current_mysql_connections = 0; - let mut current_sql_server_connections = 0; - let mut current_kafka_connections = 0; - for c in self.catalog().user_connections() { - let connection = c - .connection() - .expect("`user_connections()` only returns connection objects"); - - match connection.details { - ConnectionDetails::AwsPrivatelink(_) => current_aws_privatelink_connections += 1, - ConnectionDetails::Postgres(_) => current_postgres_connections += 1, - ConnectionDetails::MySql(_) => current_mysql_connections += 1, - ConnectionDetails::SqlServer(_) => current_sql_server_connections += 1, - ConnectionDetails::Kafka(_) => current_kafka_connections += 1, - ConnectionDetails::Csr(_) - | ConnectionDetails::Ssh { .. } - | ConnectionDetails::Aws(_) - | ConnectionDetails::IcebergCatalog(_) => {} - } - } self.validate_resource_limit( - current_kafka_connections, + self.catalog() + .user_connection_count(UserConnectionKind::Kafka), new_kafka_connections, SystemVars::max_kafka_connections, "Kafka Connection", MAX_KAFKA_CONNECTIONS.name(), )?; self.validate_resource_limit( - current_postgres_connections, + self.catalog() + .user_connection_count(UserConnectionKind::Postgres), new_postgres_connections, SystemVars::max_postgres_connections, "PostgreSQL Connection", MAX_POSTGRES_CONNECTIONS.name(), )?; self.validate_resource_limit( - current_mysql_connections, + self.catalog() + .user_connection_count(UserConnectionKind::MySql), new_mysql_connections, SystemVars::max_mysql_connections, "MySQL Connection", MAX_MYSQL_CONNECTIONS.name(), )?; self.validate_resource_limit( - current_sql_server_connections, + self.catalog() + .user_connection_count(UserConnectionKind::SqlServer), new_sql_server_connections, SystemVars::max_sql_server_connections, "SQL Server Connection", MAX_SQL_SERVER_CONNECTIONS.name(), )?; self.validate_resource_limit( - current_aws_privatelink_connections, + self.catalog() + .user_connection_count(UserConnectionKind::AwsPrivatelink), new_aws_privatelink_connections, SystemVars::max_aws_privatelink_connections, "AWS PrivateLink Connection", MAX_AWS_PRIVATELINK_CONNECTIONS.name(), )?; self.validate_resource_limit( - self.catalog().user_tables().count(), + self.catalog().user_tables_count(), new_tables, SystemVars::max_tables, "table", MAX_TABLES.name(), )?; - - let current_sources: usize = self - .catalog() - .user_sources() - .filter_map(|source| source.source()) - .map(|source| source.user_controllable_persist_shard_count()) - .sum::() - .try_into() - .expect("non-negative sum of sources"); - self.validate_resource_limit( - current_sources, + self.catalog().user_source_shard_count(), new_sources, SystemVars::max_sources, "source", MAX_SOURCES.name(), )?; self.validate_resource_limit( - self.catalog().user_sinks().count(), + self.catalog().user_sinks_count(), new_sinks, SystemVars::max_sinks, "sink", MAX_SINKS.name(), )?; self.validate_resource_limit( - self.catalog().user_materialized_views().count(), + self.catalog().user_materialized_views_count(), new_materialized_views, SystemVars::max_materialized_views, "materialized view", @@ -1458,7 +1465,7 @@ impl Coordinator { )?; } self.validate_resource_limit( - self.catalog().user_secrets().count(), + self.catalog().user_secrets_count(), new_secrets, SystemVars::max_secrets, "secret", diff --git a/src/adapter/src/coord/group_sync.rs b/src/adapter/src/coord/group_sync.rs index 52685143e0e25..35bbf890f7f8c 100644 --- a/src/adapter/src/coord/group_sync.rs +++ b/src/adapter/src/coord/group_sync.rs @@ -14,7 +14,7 @@ //! operations. Only memberships granted by the `MZ_JWT_SYNC_ROLE_ID` sentinel //! are managed; manually-granted memberships are never touched. -use std::collections::{BTreeMap, BTreeSet}; +use std::collections::BTreeSet; use mz_adapter_types::dyncfgs::{OIDC_GROUP_ROLE_SYNC_ENABLED, OIDC_GROUP_ROLE_SYNC_STRICT}; use mz_repr::role_id::RoleId; @@ -54,7 +54,7 @@ pub struct GroupSyncDiff { /// manual grant takes precedence and we don't overwrite the grantor. pub fn compute_group_sync_diff( member_id: RoleId, - current_membership: &BTreeMap, + current_membership: &imbl::OrdMap, target_role_ids: &BTreeSet, ) -> GroupSyncDiff { // Partition current memberships into sync-managed vs manually-granted. diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index 39609ffa84bb6..4784ab4252cf6 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -2139,7 +2139,7 @@ impl Coordinator { state: _, side_effects, revision, - snapshot: _, + durable_data: _, } => { // Make sure our catalog hasn't changed. if *revision != self.catalog().transient_revision() { diff --git a/src/adapter/src/metrics.rs b/src/adapter/src/metrics.rs index 2333fe4543dfd..cf0116cc70ea6 100644 --- a/src/adapter/src/metrics.rs +++ b/src/adapter/src/metrics.rs @@ -53,9 +53,12 @@ pub struct Metrics { pub catalog_snapshot_seconds: HistogramVec, pub pgwire_recv_scheduling_delay_ms: HistogramVec, pub catalog_transact_seconds: HistogramVec, + pub catalog_transact_phase_seconds: HistogramVec, + pub catalog_apply_updates_phase_seconds: HistogramVec, + pub catalog_apply_update_kind_seconds: HistogramVec, pub apply_catalog_implications_seconds: Histogram, + pub apply_catalog_implications_phase_seconds: HistogramVec, pub group_commit_catalog_upper_seconds: Histogram, - pub group_commit_table_advancement_seconds: Histogram, } impl Metrics { @@ -243,21 +246,48 @@ impl Metrics { var_labels: ["method"], buckets: histogram_seconds_buckets(0.001, 32.0), )), + catalog_transact_phase_seconds: registry.register(metric!( + name: "mz_catalog_transact_phase_seconds", + help: "The time spent in each phase of a catalog::transact call \ + (op_loop, final_apply_updates, prepare_state, \ + post_prepare_apply_updates, tx_commit, assign_state).", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.001, 32.0), + )), + catalog_apply_updates_phase_seconds: registry.register(metric!( + name: "mz_catalog_apply_updates_phase_seconds", + help: "The time spent in each sub-phase of a single \ + CatalogState::apply_updates call \ + (consolidate_initial, sort_per_group, \ + apply_updates_inner, cleanup_notices).", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.0001, 32.0), + )), + catalog_apply_update_kind_seconds: registry.register(metric!( + name: "mz_catalog_apply_update_kind_seconds", + help: "The time spent applying a single durable state update \ + inside CatalogState::apply_updates_inner, broken out by \ + update kind. One observation per update.", + var_labels: ["kind"], + buckets: histogram_seconds_buckets(0.00001, 32.0), + )), apply_catalog_implications_seconds: registry.register(metric!( name: "mz_apply_catalog_implications_seconds", help: "The time it takes to apply catalog implications.", buckets: histogram_seconds_buckets(0.001, 32.0), )), + apply_catalog_implications_phase_seconds: registry.register(metric!( + name: "mz_apply_catalog_implications_phase_seconds", + help: "The time spent in each phase of a single \ + apply_catalog_implications call.", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.0001, 32.0), + )), group_commit_catalog_upper_seconds: registry.register(metric!( name: "mz_group_commit_catalog_upper_seconds", help: "The time it takes to advance the catalog shard upper during group commit.", buckets: histogram_seconds_buckets(0.001, 32.0), )), - group_commit_table_advancement_seconds: registry.register(metric!( - 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), - )) } } diff --git a/src/adapter/src/session.rs b/src/adapter/src/session.rs index 3961e48d25075..04f5ca4b2066e 100644 --- a/src/adapter/src/session.rs +++ b/src/adapter/src/session.rs @@ -63,7 +63,7 @@ use crate::error::AdapterError; use crate::metrics::{Metrics, SessionMetrics}; use crate::statement_logging::PreparedStatementLoggingInfo; use crate::{AdapterNotice, ExecuteContext}; -use mz_catalog::durable::Snapshot; +use mz_catalog::durable::DurableCatalogData; const DUMMY_CONNECTION_ID: ConnectionId = ConnectionId::Static(0); @@ -1396,14 +1396,14 @@ impl TransactionStatus { revision: og_revision, state: og_state, side_effects, - snapshot: og_snapshot, + durable_data: og_durable_data, } => match add_ops { TransactionOps::DDL { ops: new_ops, revision: new_revision, side_effects: mut net_new_side_effects, state: new_state, - snapshot: new_snapshot, + durable_data: new_durable_data, } => { if *og_revision != new_revision { return Err(AdapterError::DDLTransactionRace); @@ -1412,7 +1412,7 @@ impl TransactionStatus { if !new_ops.is_empty() { *og_ops = new_ops; *og_state = new_state; - *og_snapshot = new_snapshot; + *og_durable_data = new_durable_data; } side_effects.append(&mut net_new_side_effects); } @@ -1603,11 +1603,12 @@ pub enum TransactionOps { >, /// Transient revision of the `Catalog` when this transaction started. revision: u64, - /// Snapshot of the durable transaction state after the last dry run. - /// Used to initialize the next dry run's transaction so it starts - /// in sync with the accumulated `state`. `None` for the first - /// statement in the transaction (before any dry run). - snapshot: Option, + /// Durable transaction state after the last dry run, carried across + /// statements in a multi-statement DDL transaction. Used to initialize + /// the next dry run's transaction so it starts in sync with the + /// accumulated `state`. `None` for the first statement in the + /// transaction (before any dry run). + durable_data: Option, }, } diff --git a/src/bogo-consensus/Cargo.toml b/src/bogo-consensus/Cargo.toml new file mode 100644 index 0000000000000..02a2ac12c703d --- /dev/null +++ b/src/bogo-consensus/Cargo.toml @@ -0,0 +1,46 @@ +[package] +name = "mz-bogo-consensus" +description = "In-memory gRPC backend for the persist Consensus trait. For performance testing only." +version = "0.0.0" +edition.workspace = true +rust-version.workspace = true +publish = false + +[lints] +workspace = true + +[dependencies] +anyhow.workspace = true +async-trait.workspace = true +axum.workspace = true +bytes.workspace = true +clap = { workspace = true, features = ["env"] } +futures.workspace = true +mz-build-info = { path = "../build-info" } +mz-http-util = { path = "../http-util" } +mz-ore = { path = "../ore", features = ["async", "cli", "metrics", "tracing"], default-features = false } +prometheus.workspace = true +prost.workspace = true +serde.workspace = true +socket2.workspace = true +tokio = { workspace = true, features = ["macros", "net", "rt-multi-thread", "signal", "sync"] } +tokio-stream = { workspace = true, features = ["net"] } +tonic = { workspace = true, features = ["transport"] } +tonic-prost.workspace = true +tracing.workspace = true +tracing-subscriber.workspace = true + +[build-dependencies] +mz-build-tools = { path = "../build-tools", default-features = false, features = ["protobuf-src"] } +prost-build.workspace = true +tonic-prost-build.workspace = true + +[[bin]] +name = "mz-bogo-consensus" +path = "src/bin/mz-bogo-consensus.rs" + +[features] +default = ["mz-build-tools/default"] + +[package.metadata.cargo-udeps.ignore] +normal = ["mz-build-info"] diff --git a/src/bogo-consensus/build.rs b/src/bogo-consensus/build.rs new file mode 100644 index 0000000000000..ff74d3dc7e70a --- /dev/null +++ b/src/bogo-consensus/build.rs @@ -0,0 +1,28 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use std::path::PathBuf; + +fn main() { + let mut config = prost_build::Config::new(); + config + .protoc_executable(mz_build_tools::protoc()) + .btree_map(["."]) + .bytes([".mz_bogo_consensus.VersionedData"]); + + println!("cargo:rerun-if-changed=src/"); + tonic_prost_build::configure() + .emit_rerun_if_changed(false) + .compile_with_config( + config, + &[PathBuf::from("bogo-consensus/src/bogo_consensus.proto")], + &[PathBuf::from(".."), mz_build_tools::protoc_include()], + ) + .unwrap_or_else(|e| panic!("{e}")); +} diff --git a/src/bogo-consensus/src/bin/mz-bogo-consensus.rs b/src/bogo-consensus/src/bin/mz-bogo-consensus.rs new file mode 100644 index 0000000000000..495fb40069841 --- /dev/null +++ b/src/bogo-consensus/src/bin/mz-bogo-consensus.rs @@ -0,0 +1,166 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +//! `mz-bogo-consensus`: a fast, in-memory persist Consensus backend served +//! over gRPC. +//! +//! Pair with `--persist-consensus-url=bogo://host:port` on environmentd (or +//! testdrive) to bypass CRDB/Postgres entirely during performance testing. + +use std::net::SocketAddr; +use std::sync::Arc; + +use axum::Router; +use axum::extract::State; +use axum::response::IntoResponse; +use axum::routing; +use mz_bogo_consensus::metrics::BogoMetrics; +use mz_bogo_consensus::proto::bogo_consensus_server::BogoConsensusServer as BogoGrpcServer; +use mz_bogo_consensus::server::BogoConsensusServer; +use mz_ore::cli::{self, CliConfig}; +use mz_ore::error::ErrorExt; +use mz_ore::metrics::MetricsRegistry; +use tokio::net::TcpListener; +use tokio::signal; +use tokio_stream::wrappers::TcpListenerStream; +use tonic::transport::Server; +use tracing::info; + +/// Run an in-memory persist Consensus backend over gRPC. +#[derive(Debug, clap::Parser)] +#[clap(name = "mz-bogo-consensus")] +struct Args { + /// The address on which to listen for gRPC requests. + #[clap( + long, + env = "BOGO_CONSENSUS_LISTEN_ADDR", + value_name = "HOST:PORT", + default_value = "127.0.0.1:6882" + )] + listen_addr: SocketAddr, + + /// The address on which to listen for Prometheus scrapes at `/metrics`. + /// If unset, no metrics endpoint is started. + #[clap( + long, + env = "BOGO_CONSENSUS_METRICS_LISTEN_ADDR", + value_name = "HOST:PORT" + )] + metrics_listen_addr: Option, + + /// Tracing filter directive (passed to `EnvFilter`). + #[clap(long, env = "BOGO_CONSENSUS_LOG_FILTER", default_value = "info")] + log_filter: String, +} + +#[tokio::main] +async fn main() { + let args: Args = cli::parse_args(CliConfig { + env_prefix: Some("MZ_BOGO_CONSENSUS_"), + enable_version_flag: false, + }); + + let env_filter = tracing_subscriber::EnvFilter::try_new(&args.log_filter) + .unwrap_or_else(|_| tracing_subscriber::EnvFilter::new("info")); + tracing_subscriber::fmt() + .with_ansi(false) + .with_env_filter(env_filter) + .init(); + + if let Err(err) = run(args).await { + eprintln!("mz-bogo-consensus: fatal: {}", err.display_with_causes()); + std::process::exit(1); + } +} + +async fn run(args: Args) -> Result<(), anyhow::Error> { + let registry = MetricsRegistry::new(); + let metrics = Arc::new(BogoMetrics::new(®istry)); + + if let Some(metrics_addr) = args.metrics_listen_addr { + spawn_metrics_server(metrics_addr, registry.clone()).await?; + } + + let server = BogoConsensusServer::new(Arc::clone(&metrics)); + // Catalog-shard scans grow with shard history; raise the gRPC message + // size cap well above tonic's 4 MiB default. + let grpc = BogoGrpcServer::new(server) + .max_decoding_message_size(256 * 1024 * 1024) + .max_encoding_message_size(256 * 1024 * 1024); + + let listener = TcpListener::bind(args.listen_addr).await?; + let local_addr = listener.local_addr()?; + info!(addr = %local_addr, "bogo-consensus gRPC server listening"); + let incoming = TcpListenerStream::new(listener); + + Server::builder() + // Loopback bench: every gRPC call is small but frequent. Defaults + // would Nagle the response trailers and stall on HTTP/2 flow-control + // window updates. See `BogoConsensusClient::connect` for the matching + // client-side knobs and the rationale. + .tcp_nodelay(true) + .initial_stream_window_size(8 * 1024 * 1024) + .initial_connection_window_size(16 * 1024 * 1024) + .add_service(grpc) + .serve_with_incoming_shutdown(incoming, shutdown_signal()) + .await?; + + info!("bogo-consensus gRPC server shut down"); + Ok(()) +} + +async fn spawn_metrics_server( + addr: SocketAddr, + registry: MetricsRegistry, +) -> Result<(), anyhow::Error> { + let listener = TcpListener::bind(addr).await?; + let local_addr = listener.local_addr()?; + info!(addr = %local_addr, "bogo-consensus metrics server listening"); + + let app = Router::new() + .route("/metrics", routing::get(prometheus_handler)) + .route("/api/livez", routing::get(|| async { "ok" })) + .with_state(registry); + + mz_ore::task::spawn(|| "bogo_consensus::metrics_http_server", async move { + if let Err(e) = axum::serve(listener, app.into_make_service()).await { + tracing::error!(error = %e, "metrics server exited"); + } + }); + Ok(()) +} + +async fn prometheus_handler(State(registry): State) -> impl IntoResponse { + mz_http_util::handle_prometheus(®istry).await +} + +async fn shutdown_signal() { + let ctrl_c = async { + let _ = signal::ctrl_c().await; + }; + #[cfg(unix)] + let terminate = async { + match signal::unix::signal(signal::unix::SignalKind::terminate()) { + Ok(mut s) => { + s.recv().await; + } + Err(e) => { + tracing::warn!(error = %e, "failed to install SIGTERM handler"); + std::future::pending::<()>().await; + } + } + }; + #[cfg(not(unix))] + let terminate = std::future::pending::<()>(); + + tokio::select! { + _ = ctrl_c => info!("received Ctrl-C, shutting down"), + _ = terminate => info!("received SIGTERM, shutting down"), + } +} diff --git a/src/bogo-consensus/src/bogo_consensus.proto b/src/bogo-consensus/src/bogo_consensus.proto new file mode 100644 index 0000000000000..2629902899b60 --- /dev/null +++ b/src/bogo-consensus/src/bogo_consensus.proto @@ -0,0 +1,73 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +syntax = "proto3"; + +package mz_bogo_consensus; + +message VersionedData { + uint64 seqno = 1; + bytes data = 2; +} + +enum CaSResult { + UNSPECIFIED = 0; + COMMITTED = 1; + EXPECTATION_MISMATCH = 2; +} + +message HeadRequest { + string key = 1; +} + +message HeadResponse { + optional VersionedData data = 1; +} + +message CompareAndSetRequest { + string key = 1; + VersionedData new = 2; +} + +message CompareAndSetResponse { + CaSResult result = 1; +} + +message ScanRequest { + string key = 1; + uint64 from = 2; + uint64 limit = 3; +} + +message ScanResponse { + repeated VersionedData data = 1; +} + +message TruncateRequest { + string key = 1; + uint64 seqno = 2; +} + +message TruncateResponse { + optional uint64 deleted = 1; +} + +message ListKeysRequest {} + +message ListKeysResponse { + string key = 1; +} + +service BogoConsensus { + rpc Head(HeadRequest) returns (HeadResponse); + rpc CompareAndSet(CompareAndSetRequest) returns (CompareAndSetResponse); + rpc Scan(ScanRequest) returns (ScanResponse); + rpc Truncate(TruncateRequest) returns (TruncateResponse); + rpc ListKeys(ListKeysRequest) returns (stream ListKeysResponse); +} diff --git a/src/bogo-consensus/src/client.rs b/src/bogo-consensus/src/client.rs new file mode 100644 index 0000000000000..be28bf59a711f --- /dev/null +++ b/src/bogo-consensus/src/client.rs @@ -0,0 +1,181 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +//! A thin gRPC client for the bogo-consensus service. +//! +//! The client exposes the raw protobuf types; the `mz_persist::bogo` module +//! wraps it and adapts to the persist `Consensus` trait. Keeping the +//! conversion out of this crate avoids a dependency cycle with `mz-persist`. + +use std::pin::Pin; +use std::sync::atomic::{AtomicUsize, Ordering}; + +use futures::Stream; +use futures::stream::TryStreamExt; +use tonic::Status; +use tonic::transport::{Channel, Endpoint}; + +use crate::proto::bogo_consensus_client::BogoConsensusClient as TonicClient; +use crate::proto::{ + CompareAndSetRequest, HeadRequest, ListKeysRequest, ScanRequest, TruncateRequest, VersionedData, +}; + +pub use crate::proto::CaSResult; + +/// Default fan-out of independent gRPC connections from a single +/// [`BogoConsensusClient`] to the server. We open this many TCP +/// connections and round-robin RPCs across them so a multi-MB scan +/// response on one connection does not head-of-line-block the small +/// CAS responses queued on others. +/// +/// Empirically (envd DDL audit, `persist_external_op_latency` for +/// `consensus_cas`): with `1` channel ≈ 5-10% of CAS calls land in +/// the 32-64 ms bucket and dominate the mean; bumping to a small +/// pool clears that tail without measurable overhead at idle. +/// +/// Matches the default `persist_consensus_connection_pool_max_size` +/// (50) for the Postgres/CRDB backend — that's the size persist is +/// already designed to drive concurrently against a consensus +/// backend, so it's the right shape for an apples-to-apples bogo. +const DEFAULT_CHANNELS: usize = 50; + +/// A handle to a bogo-consensus server. +/// +/// Internally holds a fixed pool of independent gRPC channels (TCP +/// connections) and round-robins each RPC across them. Cloning is +/// cheap — every field is `Arc`-like under the hood. +#[derive(Debug, Clone)] +pub struct BogoConsensusClient { + channels: std::sync::Arc>>, + next: std::sync::Arc, +} + +impl BogoConsensusClient { + /// Connects to a bogo-consensus server at the given URL. + /// + /// `url` must be a tonic-acceptable URI (typically `http://host:port`). + /// The persist `bogo://` URL scheme is translated to `http://` before + /// being passed here. + pub async fn connect(url: String) -> Result { + Self::connect_with_channels(url, DEFAULT_CHANNELS).await + } + + /// Like [`connect`](Self::connect) but with an explicit number of + /// underlying gRPC connections. Exposed mostly so the conformance test + /// can run with `channels = 1` and exercise the single-connection path. + pub async fn connect_with_channels( + url: String, + channels: usize, + ) -> Result { + let channels = channels.max(1); + let endpoint = Endpoint::from_shared(url)? + // No TLS — bogo is for local perf testing. + .tcp_nodelay(true) + // Bump HTTP/2 flow control windows well above tonic's 65 KiB + // default. With the default, per-call CAS latency on a loopback + // bench rises linearly with payload size (~130 ns/byte, ≈ 7 MB/s + // throughput) because every other request stalls waiting for the + // server's WINDOW_UPDATE round-trip. Bumping the windows up + // amortises that across many requests. + .initial_stream_window_size(8 * 1024 * 1024) + .initial_connection_window_size(16 * 1024 * 1024); + let mut conns = Vec::with_capacity(channels); + for _ in 0..channels { + let channel = endpoint.clone().connect().await?; + // Catalog-shard scans grow with shard history; raise the gRPC + // message size cap well above tonic's 4 MiB default to avoid + // retry storms. + let client = TonicClient::new(channel) + .max_decoding_message_size(256 * 1024 * 1024) + .max_encoding_message_size(256 * 1024 * 1024); + conns.push(client); + } + Ok(Self { + channels: std::sync::Arc::new(conns), + next: std::sync::Arc::new(AtomicUsize::new(0)), + }) + } + + /// Picks the next connection in round-robin order and returns an owned + /// clone of its tonic client. Cloning the `TonicClient` just clones the + /// underlying `Channel`'s `Arc`, so this is cheap. + fn pick(&self) -> TonicClient { + let i = self.next.fetch_add(1, Ordering::Relaxed) % self.channels.len(); + self.channels[i].clone() + } + + pub async fn head(&self, key: &str) -> Result, Status> { + let resp = self + .pick() + .head(HeadRequest { + key: key.to_owned(), + }) + .await? + .into_inner(); + Ok(resp.data) + } + + pub async fn compare_and_set( + &self, + key: &str, + new: VersionedData, + ) -> Result { + let resp = self + .pick() + .compare_and_set(CompareAndSetRequest { + key: key.to_owned(), + new: Some(new), + }) + .await? + .into_inner(); + CaSResult::try_from(resp.result) + .map_err(|_| Status::internal(format!("unknown CaSResult: {}", resp.result))) + } + + pub async fn scan( + &self, + key: &str, + from: u64, + limit: u64, + ) -> Result, Status> { + let resp = self + .pick() + .scan(ScanRequest { + key: key.to_owned(), + from, + limit, + }) + .await? + .into_inner(); + Ok(resp.data) + } + + pub async fn truncate(&self, key: &str, seqno: u64) -> Result, Status> { + let resp = self + .pick() + .truncate(TruncateRequest { + key: key.to_owned(), + seqno, + }) + .await? + .into_inner(); + Ok(resp.deleted) + } + + pub async fn list_keys( + &self, + ) -> Result> + Send>>, Status> { + let stream = self + .pick() + .list_keys(ListKeysRequest {}) + .await? + .into_inner(); + Ok(Box::pin(stream.map_ok(|resp| resp.key))) + } +} diff --git a/src/bogo-consensus/src/lib.rs b/src/bogo-consensus/src/lib.rs new file mode 100644 index 0000000000000..fe92067b8225d --- /dev/null +++ b/src/bogo-consensus/src/lib.rs @@ -0,0 +1,37 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +//! In-memory gRPC backend for the persist `Consensus` trait. +//! +//! `bogo-consensus` is a fast, non-durable consensus backend used for +//! performance testing. It runs as a standalone server that keeps all state in +//! memory (mirroring [`mz_persist::mem::MemConsensus`]) and exposes it over +//! gRPC. The client side, in [`client`], implements just enough surface to be +//! plugged into `mz_persist::cfg::ConsensusConfig` via the `bogo://` URL +//! scheme. +//! +//! It is **not durable, not HA, not multi-writer-safe across crashes** — the +//! name is a deliberate nod to bogosort. Use it only when you need to take +//! Postgres/CRDB out of the loop for benchmarking. + +#![allow(missing_docs)] + +pub mod client; +pub mod metrics; +pub mod server; + +#[allow( + clippy::clone_on_ref_ptr, + clippy::as_conversions, + clippy::enum_variant_names +)] +pub mod proto { + //! Generated protobuf types and tonic stubs. + tonic::include_proto!("mz_bogo_consensus"); +} diff --git a/src/bogo-consensus/src/metrics.rs b/src/bogo-consensus/src/metrics.rs new file mode 100644 index 0000000000000..a53c541f7256f --- /dev/null +++ b/src/bogo-consensus/src/metrics.rs @@ -0,0 +1,73 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +//! Server-side metrics for the bogo-consensus gRPC service. + +use mz_ore::metric; +use mz_ore::metrics::MetricsRegistry; +use mz_ore::stats::histogram_seconds_buckets; +use prometheus::{HistogramVec, IntCounterVec, IntGauge, IntGauge as UIntGauge}; + +/// Prometheus metrics registered on the server side. +/// +/// Histograms are labeled by RPC name (`head`, `compare_and_set`, `scan`, +/// `truncate`, `list_keys`) and outcome (`ok`, `mismatch`, `err`). Gauges +/// expose the size of the in-memory state so a scraper can correlate request +/// latency with how large the store has grown during a run. +#[derive(Debug, Clone)] +pub struct BogoMetrics { + pub rpc_started: IntCounterVec, + pub rpc_completed: IntCounterVec, + pub rpc_seconds: HistogramVec, + pub rpc_bytes_in: IntCounterVec, + pub rpc_bytes_out: IntCounterVec, + pub shards_total: IntGauge, + pub versions_total: UIntGauge, +} + +impl BogoMetrics { + pub fn new(registry: &MetricsRegistry) -> Self { + Self { + rpc_started: registry.register(metric!( + name: "mz_bogo_consensus_rpc_started_total", + help: "Count of RPCs received by the bogo-consensus server.", + var_labels: ["rpc"], + )), + rpc_completed: registry.register(metric!( + name: "mz_bogo_consensus_rpc_completed_total", + help: "Count of RPCs that finished, labeled by outcome.", + var_labels: ["rpc", "outcome"], + )), + rpc_seconds: registry.register(metric!( + name: "mz_bogo_consensus_rpc_seconds", + help: "Wall time spent serving each RPC.", + var_labels: ["rpc"], + buckets: histogram_seconds_buckets(0.000_050, 16.0), + )), + rpc_bytes_in: registry.register(metric!( + name: "mz_bogo_consensus_rpc_bytes_in_total", + help: "Total bytes of payload data accepted (CAS data field).", + var_labels: ["rpc"], + )), + rpc_bytes_out: registry.register(metric!( + name: "mz_bogo_consensus_rpc_bytes_out_total", + help: "Total bytes of payload data returned (head/scan).", + var_labels: ["rpc"], + )), + shards_total: registry.register(metric!( + name: "mz_bogo_consensus_shards_total", + help: "Number of distinct keys currently held in memory.", + )), + versions_total: registry.register(metric!( + name: "mz_bogo_consensus_versions_total", + help: "Total number of VersionedData entries currently held in memory.", + )), + } + } +} diff --git a/src/bogo-consensus/src/server.rs b/src/bogo-consensus/src/server.rs new file mode 100644 index 0000000000000..a8dd4a3252f85 --- /dev/null +++ b/src/bogo-consensus/src/server.rs @@ -0,0 +1,299 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +//! Server-side implementation of the bogo-consensus gRPC service. +//! +//! The state model intentionally mirrors `mz_persist::mem::MemConsensus`: each +//! key maps to an append-only `Vec` ordered by sequence number, +//! and a compare-and-set succeeds iff the proposed seqno is exactly one more +//! than the current head. + +use std::collections::BTreeMap; +use std::pin::Pin; +use std::sync::{Arc, Mutex}; +use std::time::Instant; + +use futures::Stream; +use mz_ore::cast::CastFrom; +use tokio::sync::mpsc; +use tokio_stream::wrappers::ReceiverStream; +use tonic::{Request, Response, Status}; +use tracing::warn; + +use crate::metrics::BogoMetrics; +use crate::proto::bogo_consensus_server::BogoConsensus; +use crate::proto::{ + CaSResult, CompareAndSetRequest, CompareAndSetResponse, HeadRequest, HeadResponse, + ListKeysRequest, ListKeysResponse, ScanRequest, ScanResponse, TruncateRequest, + TruncateResponse, VersionedData, +}; + +/// In-memory consensus state, exposed via gRPC. +/// +/// The state is held behind a [`std::sync::Mutex`] (not `tokio::sync::Mutex`) +/// to match MemConsensus's documented avoidance of intermittent deadlocks under +/// the persist concurrency tests. The lock is held only for short, bounded +/// operations on the `BTreeMap`, so blocking the runtime is acceptable. +#[derive(Debug)] +pub struct BogoConsensusServer { + data: Arc>>>, + metrics: Arc, +} + +impl BogoConsensusServer { + pub fn new(metrics: Arc) -> Self { + Self { + data: Arc::new(Mutex::new(BTreeMap::new())), + metrics, + } + } + + /// Update the `shards_total` and `versions_total` gauges incrementally. + /// Called from the hot path while holding `self.data`'s mutex, so the + /// per-call cost must be O(1) — iterating the BTreeMap here adds ~100 µs + /// per CAS at 10k shards and head-of-line-blocks every other op behind + /// the same mutex. + /// + /// `shards_delta` is +1 for a CAS that created a new key, 0 otherwise. + /// `versions_delta` is the net change to the total number of stored + /// versions: +1 for a successful CAS append, -N for a truncate that + /// removed N entries. + fn bump_state_gauges(&self, shards_delta: i64, versions_delta: i64) { + if shards_delta != 0 { + self.metrics.shards_total.add(shards_delta); + } + if versions_delta != 0 { + self.metrics.versions_total.add(versions_delta); + } + } +} + +// i64::MAX as a u64; written this way to avoid an `as` conversion. +const SEQNO_MAX: u64 = u64::MAX >> 1; + +fn observe( + metrics: &BogoMetrics, + rpc: &'static str, + start: Instant, + result: Result, +) -> Result { + let elapsed = start.elapsed().as_secs_f64(); + metrics + .rpc_seconds + .with_label_values(&[rpc]) + .observe(elapsed); + let outcome = match &result { + Ok(_) => "ok", + Err(status) => match status.code() { + tonic::Code::FailedPrecondition => "mismatch", + _ => "err", + }, + }; + metrics + .rpc_completed + .with_label_values(&[rpc, outcome]) + .inc(); + if let Err(status) = &result { + warn!(rpc, code = ?status.code(), message = %status.message(), "rpc error"); + } + result +} + +#[async_trait::async_trait] +impl BogoConsensus for BogoConsensusServer { + async fn head(&self, request: Request) -> Result, Status> { + let start = Instant::now(); + self.metrics.rpc_started.with_label_values(&["head"]).inc(); + let HeadRequest { key } = request.into_inner(); + + let result = (|| { + let store = self + .data + .lock() + .map_err(|e| Status::internal(format!("lock poisoned: {e}")))?; + let data = store.get(&key).and_then(|v| v.last()).cloned(); + if let Some(d) = &data { + self.metrics + .rpc_bytes_out + .with_label_values(&["head"]) + .inc_by(u64::cast_from(d.data.len())); + } + Ok(Response::new(HeadResponse { data })) + })(); + + observe(&self.metrics, "head", start, result) + } + + async fn compare_and_set( + &self, + request: Request, + ) -> Result, Status> { + let start = Instant::now(); + self.metrics + .rpc_started + .with_label_values(&["compare_and_set"]) + .inc(); + let CompareAndSetRequest { key, new } = request.into_inner(); + let new = new.ok_or_else(|| Status::invalid_argument("missing `new` field"))?; + + let result = (|| { + if new.seqno > SEQNO_MAX { + return Err(Status::invalid_argument(format!( + "sequence numbers must fit within [0, i64::MAX], received: {}", + new.seqno + ))); + } + // CAS: `new.seqno` must equal previous.seqno + 1 (or 0 if empty). + // The trait phrases this as `new.seqno == previous.seqno.next()`, + // but here we work with the raw u64 so we compute `expected` as + // `new.seqno.checked_sub(1)` — `None` means "must be empty". + let expected = new.seqno.checked_sub(1); + let data_len = u64::cast_from(new.data.len()); + + let mut store = self + .data + .lock() + .map_err(|e| Status::internal(format!("lock poisoned: {e}")))?; + + let current = store.get(&key).and_then(|v| v.last()).map(|d| d.seqno); + if current != expected { + return Ok(Response::new(CompareAndSetResponse { + result: i32::from(CaSResult::ExpectationMismatch), + })); + } + let is_new_key = !store.contains_key(&key); + store.entry(key).or_default().push(new); + drop(store); + self.metrics + .rpc_bytes_in + .with_label_values(&["compare_and_set"]) + .inc_by(data_len); + self.bump_state_gauges(if is_new_key { 1 } else { 0 }, 1); + Ok(Response::new(CompareAndSetResponse { + result: i32::from(CaSResult::Committed), + })) + })(); + + observe(&self.metrics, "compare_and_set", start, result) + } + + async fn scan(&self, request: Request) -> Result, Status> { + let start = Instant::now(); + self.metrics.rpc_started.with_label_values(&["scan"]).inc(); + let ScanRequest { key, from, limit } = request.into_inner(); + let limit = usize::try_from(limit).unwrap_or(usize::MAX); + + let result = (|| { + let store = self + .data + .lock() + .map_err(|e| Status::internal(format!("lock poisoned: {e}")))?; + let data = if let Some(values) = store.get(&key) { + let from_idx = values.partition_point(|x| x.seqno < from); + let slice = &values[from_idx..]; + let end = usize::min(limit, slice.len()); + slice[..end].to_vec() + } else { + Vec::new() + }; + let bytes_out: u64 = data.iter().map(|d| u64::cast_from(d.data.len())).sum(); + self.metrics + .rpc_bytes_out + .with_label_values(&["scan"]) + .inc_by(bytes_out); + Ok(Response::new(ScanResponse { data })) + })(); + + observe(&self.metrics, "scan", start, result) + } + + async fn truncate( + &self, + request: Request, + ) -> Result, Status> { + let start = Instant::now(); + self.metrics + .rpc_started + .with_label_values(&["truncate"]) + .inc(); + let TruncateRequest { key, seqno } = request.into_inner(); + + let result = (|| { + let mut store = self + .data + .lock() + .map_err(|e| Status::internal(format!("lock poisoned: {e}")))?; + // Mirror MemConsensus: error if seqno > current head, or if no data + // exists for the key. Both map to FailedPrecondition (Determinate + // on the client side). + let current_head = store.get(&key).and_then(|v| v.last()).map(|d| d.seqno); + let too_high = match current_head { + None => true, + Some(head) => head < seqno, + }; + if too_high { + return Err(Status::failed_precondition(format!( + "upper bound too high for truncate: {seqno}" + ))); + } + let mut deleted: u64 = 0; + if let Some(values) = store.get_mut(&key) { + let count_before = values.len(); + values.retain(|val| val.seqno >= seqno); + deleted = u64::cast_from(count_before - values.len()); + } + drop(store); + let versions_delta = -i64::try_from(deleted).unwrap_or(i64::MAX); + self.bump_state_gauges(0, versions_delta); + Ok(Response::new(TruncateResponse { + deleted: Some(deleted), + })) + })(); + + observe(&self.metrics, "truncate", start, result) + } + + type ListKeysStream = + Pin> + Send + 'static>>; + + async fn list_keys( + &self, + _request: Request, + ) -> Result, Status> { + let start = Instant::now(); + self.metrics + .rpc_started + .with_label_values(&["list_keys"]) + .inc(); + + let keys: Vec = { + let store = self + .data + .lock() + .map_err(|e| Status::internal(format!("lock poisoned: {e}")))?; + store.keys().cloned().collect() + }; + + // We collect under lock then drop it before streaming. The persist + // trait documents list_keys as administrative-only, so the + // collect-then-stream shape is fine. + let (tx, rx) = mpsc::channel(64); + mz_ore::task::spawn(|| "bogo_consensus::list_keys", async move { + for key in keys { + if tx.send(Ok(ListKeysResponse { key })).await.is_err() { + break; + } + } + }); + + let stream: Self::ListKeysStream = Box::pin(ReceiverStream::new(rx)); + let response = Response::new(stream); + observe(&self.metrics, "list_keys", start, Ok(response)) + } +} diff --git a/src/catalog/Cargo.toml b/src/catalog/Cargo.toml index 6704bd79d8b06..cf5481f887728 100644 --- a/src/catalog/Cargo.toml +++ b/src/catalog/Cargo.toml @@ -21,6 +21,7 @@ clap.workspace = true derivative.workspace = true differential-dataflow.workspace = true futures.workspace = true +imbl.workspace = true ipnet.workspace = true itertools.workspace = true mz-adapter-types = { path = "../adapter-types" } diff --git a/src/catalog/src/durable.rs b/src/catalog/src/durable.rs index fac877b89a297..5299eceb0737a 100644 --- a/src/catalog/src/durable.rs +++ b/src/catalog/src/durable.rs @@ -41,6 +41,7 @@ pub use crate::durable::objects::{ StorageCollectionMetadata, SystemConfiguration, SystemObjectDescription, SystemObjectMapping, UnfinalizedShard, }; +pub use crate::durable::persist::DurableCatalogData; pub use crate::durable::persist::shard_id; use crate::durable::persist::{Timestamp, UnopenedPersistCatalogState}; pub use crate::durable::transaction::Transaction; @@ -310,6 +311,15 @@ pub trait DurableCatalogState: ReadOnlyDurableCatalogState { snapshot: Snapshot, ) -> Result; + /// Like [`Self::transaction_from_snapshot`], but takes a [`DurableCatalogData`] + /// (the shared, structurally-cloned view of the catalog tables) instead of + /// the proto-typed [`Snapshot`]. Avoids the per-statement O(N) proto→Rust + /// rebuild that `transaction_from_snapshot` pays. + fn transaction_from_durable_data( + &mut self, + data: DurableCatalogData, + ) -> Result; + /// Commits a durable catalog state transaction. The transaction will be committed at /// `commit_ts`. /// @@ -345,7 +355,8 @@ pub trait DurableCatalogState: ReadOnlyDurableCatalogState { } let mut txn = self.transaction().await?; let ids = txn.get_and_increment_id_by(id_type.to_string(), amount)?; - txn.commit_internal(commit_ts).await?; + let txn_batch = txn.into_parts(); + self.commit_transaction(txn_batch, commit_ts).await?; self.metrics() .allocate_id_seconds .observe(start.elapsed().as_secs_f64()); diff --git a/src/catalog/src/durable/initialize.rs b/src/catalog/src/durable/initialize.rs index a2b79847fbb2f..2f5463221adfc 100644 --- a/src/catalog/src/durable/initialize.rs +++ b/src/catalog/src/durable/initialize.rs @@ -213,7 +213,7 @@ static SYSTEM_SCHEMAS: LazyLock> = LazyLock::new(|| { /// Initializes the Catalog with some default objects. #[mz_ore::instrument] pub(crate) async fn initialize( - tx: &mut Transaction<'_>, + tx: &mut Transaction, options: &BootstrapArgs, initial_ts: EpochMillis, catalog_content_version: String, diff --git a/src/catalog/src/durable/metrics.rs b/src/catalog/src/durable/metrics.rs index fc3eedaa400eb..c9a5d0601379a 100644 --- a/src/catalog/src/durable/metrics.rs +++ b/src/catalog/src/durable/metrics.rs @@ -12,7 +12,7 @@ use mz_ore::metric; use mz_ore::metrics::{IntCounter, MetricsRegistry}; use mz_ore::stats::histogram_seconds_buckets; -use prometheus::{Counter, Histogram, IntGauge, IntGaugeVec}; +use prometheus::{Counter, Histogram, HistogramVec, IntGauge, IntGaugeVec}; #[derive(Debug, Clone)] pub struct Metrics { @@ -27,6 +27,8 @@ pub struct Metrics { pub allocate_id_seconds: Histogram, pub snapshot_consolidations: IntCounter, pub snapshot_max_entries: IntGauge, + pub commit_transaction_phase_seconds: HistogramVec, + pub sync_phase_seconds: HistogramVec, } impl Metrics { @@ -80,6 +82,20 @@ impl Metrics { help: "High-water mark of entries in the unconsolidated in-memory \ snapshot since process start.", )), + commit_transaction_phase_seconds: registry.register(metric!( + name: "mz_catalog_commit_transaction_phase_seconds", + help: "The time spent in each phase of a single \ + PersistCatalogState::commit_transaction call.", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.0001, 32.0), + )), + sync_phase_seconds: registry.register(metric!( + name: "mz_catalog_sync_phase_seconds", + help: "The time spent in each phase of a single \ + PersistCatalogState::sync call.", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.0001, 32.0), + )), } } } diff --git a/src/catalog/src/durable/persist.rs b/src/catalog/src/durable/persist.rs index c93830e38d7e3..d62f05ae5a518 100644 --- a/src/catalog/src/durable/persist.rs +++ b/src/catalog/src/durable/persist.rs @@ -22,6 +22,7 @@ use differential_dataflow::lattice::Lattice; use futures::{FutureExt, StreamExt}; use itertools::Itertools; use mz_audit_log::VersionedEvent; +use mz_controller_types::ClusterId; use mz_ore::metrics::MetricsFutureExt; use mz_ore::now::EpochMillis; use mz_ore::{ @@ -38,6 +39,8 @@ use mz_persist_client::{Diagnostics, PersistClient, ShardId}; use mz_persist_types::codec_impls::UnitSchema; use mz_proto::{RustType, TryFromProtoError}; use mz_repr::Diff; +use mz_sql::catalog::CatalogItemType; +use mz_sql::names::{DatabaseId, SchemaId}; use mz_storage_client::controller::PersistEpoch; use mz_storage_types::StorageDiff; use mz_storage_types::sources::SourceData; @@ -57,7 +60,17 @@ use crate::durable::objects::state_update::{ IntoStateUpdateKindJson, StateUpdate, StateUpdateKind, StateUpdateKindJson, TryIntoStateUpdateKind, }; -use crate::durable::objects::{AuditLogKey, FenceToken, Snapshot}; +use crate::durable::objects::{ + AuditLogKey, ClusterIntrospectionSourceIndexKey, ClusterIntrospectionSourceIndexValue, + ClusterKey, ClusterReplicaKey, ClusterReplicaValue, ClusterValue, CommentKey, CommentValue, + ConfigKey, ConfigValue, DatabaseKey, DatabaseValue, DefaultPrivilegesKey, + DefaultPrivilegesValue, FenceToken, GidMappingKey, GidMappingValue, IdAllocKey, IdAllocValue, + ItemKey, ItemValue, NetworkPolicyKey, NetworkPolicyValue, RoleAuthKey, RoleAuthValue, RoleKey, + RoleValue, SchemaKey, SchemaValue, ServerConfigurationKey, ServerConfigurationValue, + SettingKey, SettingValue, Snapshot, SourceReferencesKey, SourceReferencesValue, + StorageCollectionMetadataKey, StorageCollectionMetadataValue, SystemPrivilegesKey, + SystemPrivilegesValue, TxnWalShardValue, UnfinalizedShardKey, +}; use crate::durable::transaction::TransactionBatch; use crate::durable::upgrade::upgrade; use crate::durable::{ @@ -395,6 +408,8 @@ impl> PersistHandle { updates: Vec<(S, Diff)>, commit_ts: Timestamp, ) -> Result { + let phase = self.metrics.commit_transaction_phase_seconds.clone(); + let _fence_timer = phase.with_label_values(&["caa_fence_check"]).start_timer(); // The fencing check is expensive, so run it only with soft assertions enabled. let contains_fence = if mz_ore::assert::soft_assertions_enabled() { let parsed_updates: Vec<_> = updates @@ -417,6 +432,8 @@ impl> PersistHandle { None }; + drop(_fence_timer); + let _encode_timer = phase.with_label_values(&["caa_encode"]).start_timer(); let updates = updates.into_iter().map(|(kind, diff)| { let kind: StateUpdateKindJson = kind.into(); ( @@ -426,6 +443,10 @@ impl> PersistHandle { ) }); let next_upper = commit_ts.step_forward(); + drop(_encode_timer); + let _inner_timer = phase + .with_label_values(&["caa_persist_caa_inner"]) + .start_timer(); self.compare_and_append_inner(updates, next_upper) .await .inspect_err(|e| { @@ -440,6 +461,8 @@ impl> PersistHandle { ); } })?; + drop(_inner_timer); + let _sync_timer = phase.with_label_values(&["caa_post_sync"]).start_timer(); self.sync(next_upper).await?; Ok(next_upper) @@ -466,6 +489,10 @@ impl> PersistHandle { self.upper, ); + let phase = self.metrics.commit_transaction_phase_seconds.clone(); + let _persist_caa = phase + .with_label_values(&["caa_persist_compare_and_append"]) + .start_timer(); let res = self .write_handle .compare_and_append( @@ -475,6 +502,7 @@ impl> PersistHandle { ) .await .expect("invalid usage"); + drop(_persist_caa); if let Err(e @ UpperMismatch { .. }) = res { // Most likely we were fenced out. @@ -483,6 +511,9 @@ impl> PersistHandle { return Err(e.into()); } + let _downgrade = phase + .with_label_values(&["caa_since_downgrade"]) + .start_timer(); // Lag the shard's upper by 1 to keep it readable. let downgrade_to = Antichain::from_elem(next_upper.saturating_sub(1)); @@ -564,12 +595,25 @@ impl> PersistHandle { let mut updates: BTreeMap<_, Vec<_>> = BTreeMap::new(); - // Reset the amortized consolidation tracker so it picks up the - // current snapshot size as its baseline. - self.size_at_last_consolidation = None; + // We deliberately do NOT reset `size_at_last_consolidation` here. + // The doubling threshold is meant to amortize consolidation across + // the snapshot's lifetime, not per `sync_inner` invocation. Resetting + // it means that for the common steady-state case (one ts per sync) + // `maybe_consolidate` never fires (delta-vs-current is far below 2x), + // and consolidation only ever happens via the unconditional pass + // below — paying O(N log N) on every commit. Persisting the threshold + // across calls makes `maybe_consolidate` actually trigger when total + // growth doubles, which is what we want. + + let phase = self.metrics.sync_phase_seconds.clone(); + let mut listen_fetch_secs = 0.0; + let mut apply_secs = 0.0; + let mut consolidate_secs = 0.0; while self.upper < target_upper { + let t = std::time::Instant::now(); let listen_events = self.listen.fetch_next().await; + listen_fetch_secs += t.elapsed().as_secs_f64(); for listen_event in listen_events { match listen_event { ListenEvent::Progress(upper) => { @@ -592,8 +636,12 @@ impl> PersistHandle { } }, ); + let t = std::time::Instant::now(); self.apply_updates(updates)?; + apply_secs += t.elapsed().as_secs_f64(); + let t = std::time::Instant::now(); self.maybe_consolidate(); + consolidate_secs += t.elapsed().as_secs_f64(); } } ListenEvent::Updates(batch_updates) => { @@ -606,8 +654,24 @@ impl> PersistHandle { } } assert_eq!(updates, BTreeMap::new(), "all updates should be applied"); - // Always consolidate at the end to ensure the snapshot is clean. - self.consolidate(); + // Do one final amortized consolidate check. Combined with the per-ts + // `maybe_consolidate` above, this is enough to bound memory at 2x the + // last-consolidated snapshot size while keeping per-call cost O(K log K) + // where K is the delta from the last consolidation (not O(N log N) + // every sync). + let t = std::time::Instant::now(); + self.maybe_consolidate(); + consolidate_secs += t.elapsed().as_secs_f64(); + + phase + .with_label_values(&["listen_fetch"]) + .observe(listen_fetch_secs); + phase + .with_label_values(&["apply_updates"]) + .observe(apply_secs); + phase + .with_label_values(&["consolidate"]) + .observe(consolidate_secs); Ok(()) } @@ -1337,12 +1401,12 @@ impl UnopenedPersistCatalogState { }; if read_only { - let (txn_batch, _) = txn.into_parts(); + let txn_batch = txn.into_parts(); // The upper here doesn't matter because we are only applying the updates in memory. let updates = StateUpdate::from_txn_batch_ts(txn_batch, catalog.upper); catalog.apply_updates_and_consolidate(updates)?; } else { - txn.commit_internal(commit_ts).await?; + txn.commit_internal(&mut catalog, commit_ts).await?; } if matches!(catalog.mode, Mode::Writable) { @@ -1574,17 +1638,307 @@ impl OpenableDurableCatalogState for UnopenedPersistCatalogState { } } +/// Shared, indexed materialised view of the durable catalog tables. +/// +/// One `Arc>` per durable-catalog table, maintained +/// incrementally on the trace-apply path. Cloning is O(1) per field +/// (`Arc::clone`); structural sharing means transactions holding old `Arc`s +/// see a frozen snapshot while later mutations through `Arc::make_mut` only +/// fork the affected nodes. +#[derive(Debug, Clone)] +pub struct DurableCatalogData { + pub(crate) databases: Arc>, + pub(crate) schemas: Arc>, + pub(crate) roles: Arc>, + pub(crate) role_auth: Arc>, + pub(crate) items: Arc>, + pub(crate) comments: Arc>, + pub(crate) clusters: Arc>, + pub(crate) network_policies: Arc>, + pub(crate) cluster_replicas: Arc>, + pub(crate) introspection_sources: + Arc>, + pub(crate) id_allocator: Arc>, + pub(crate) configs: Arc>, + pub(crate) settings: Arc>, + pub(crate) source_references: Arc>, + pub(crate) system_object_mappings: Arc>, + pub(crate) system_configurations: + Arc>, + pub(crate) default_privileges: Arc>, + pub(crate) system_privileges: Arc>, + pub(crate) storage_collection_metadata: + Arc>, + pub(crate) unfinalized_shards: Arc>, + pub(crate) txn_wal_shard: Arc>, + pub(crate) indexes: DurableCatalogIndexes, +} + +impl DurableCatalogData { + pub(crate) fn new() -> DurableCatalogData { + DurableCatalogData { + databases: Arc::new(imbl::OrdMap::new()), + schemas: Arc::new(imbl::OrdMap::new()), + roles: Arc::new(imbl::OrdMap::new()), + role_auth: Arc::new(imbl::OrdMap::new()), + items: Arc::new(imbl::OrdMap::new()), + comments: Arc::new(imbl::OrdMap::new()), + clusters: Arc::new(imbl::OrdMap::new()), + network_policies: Arc::new(imbl::OrdMap::new()), + cluster_replicas: Arc::new(imbl::OrdMap::new()), + introspection_sources: Arc::new(imbl::OrdMap::new()), + id_allocator: Arc::new(imbl::OrdMap::new()), + configs: Arc::new(imbl::OrdMap::new()), + settings: Arc::new(imbl::OrdMap::new()), + source_references: Arc::new(imbl::OrdMap::new()), + system_object_mappings: Arc::new(imbl::OrdMap::new()), + system_configurations: Arc::new(imbl::OrdMap::new()), + default_privileges: Arc::new(imbl::OrdMap::new()), + system_privileges: Arc::new(imbl::OrdMap::new()), + storage_collection_metadata: Arc::new(imbl::OrdMap::new()), + unfinalized_shards: Arc::new(imbl::OrdMap::new()), + txn_wal_shard: Arc::new(imbl::OrdMap::new()), + indexes: DurableCatalogIndexes::new(), + } + } +} + +/// Name-keyed indexes into [`DurableCatalogData`]. +/// +/// Each index is an `Arc>` maintained in lock-step with the +/// table it indexes, on the trace-apply path. Lookups are O(log N); clones are +/// O(1). +#[derive(Debug, Clone)] +pub(crate) struct DurableCatalogIndexes { + pub(crate) database_by_name: Arc>, + pub(crate) schema_by_parent_name: Arc, String), SchemaKey>>, + pub(crate) role_by_name: Arc>, + pub(crate) cluster_by_name: Arc>, + pub(crate) replica_by_cluster_and_name: + Arc>, + pub(crate) network_policy_by_name: Arc>, + /// Items at a `(schema_id, name)` are typically 0 or 1, occasionally 2 due + /// to the Type/Func compatibility rule. The value is the small list of + /// items at that namespace. + pub(crate) item_by_namespace: + Arc>>, + pub(crate) oid_owner: Arc>, +} + +impl DurableCatalogIndexes { + fn new() -> DurableCatalogIndexes { + DurableCatalogIndexes { + database_by_name: Arc::new(imbl::OrdMap::new()), + schema_by_parent_name: Arc::new(imbl::OrdMap::new()), + role_by_name: Arc::new(imbl::OrdMap::new()), + cluster_by_name: Arc::new(imbl::OrdMap::new()), + replica_by_cluster_and_name: Arc::new(imbl::OrdMap::new()), + network_policy_by_name: Arc::new(imbl::OrdMap::new()), + item_by_namespace: Arc::new(imbl::OrdMap::new()), + oid_owner: Arc::new(imbl::OrdMap::new()), + } + } + + /// Build indexes by walking the catalog tables once. O(N); used by the + /// snapshot path that pays the proto→Rust conversion upfront. + pub(crate) fn from_tables( + databases: &imbl::OrdMap, + schemas: &imbl::OrdMap, + roles: &imbl::OrdMap, + clusters: &imbl::OrdMap, + cluster_replicas: &imbl::OrdMap, + network_policies: &imbl::OrdMap, + items: &imbl::OrdMap, + introspection_sources: &imbl::OrdMap< + ClusterIntrospectionSourceIndexKey, + ClusterIntrospectionSourceIndexValue, + >, + ) -> DurableCatalogIndexes { + let mut database_by_name = imbl::OrdMap::new(); + let mut schema_by_parent_name = imbl::OrdMap::new(); + let mut role_by_name = imbl::OrdMap::new(); + let mut cluster_by_name = imbl::OrdMap::new(); + let mut replica_by_cluster_and_name = imbl::OrdMap::new(); + let mut network_policy_by_name = imbl::OrdMap::new(); + let mut item_by_namespace: imbl::OrdMap< + (SchemaId, String), + imbl::Vector<(ItemKey, CatalogItemType)>, + > = imbl::OrdMap::new(); + let mut oid_owner = imbl::OrdMap::new(); + + for (k, v) in databases.iter() { + database_by_name.insert(v.name.clone(), *k); + oid_owner.insert(v.oid, OidOwner::Database(*k)); + } + for (k, v) in schemas.iter() { + schema_by_parent_name.insert((v.database_id, v.name.clone()), *k); + oid_owner.insert(v.oid, OidOwner::Schema(*k)); + } + for (k, v) in roles.iter() { + role_by_name.insert(v.name.clone(), k.clone()); + oid_owner.insert(v.oid, OidOwner::Role(k.clone())); + } + for (k, v) in clusters.iter() { + cluster_by_name.insert(v.name.clone(), k.clone()); + } + for (k, v) in cluster_replicas.iter() { + replica_by_cluster_and_name.insert((v.cluster_id, v.name.clone()), k.clone()); + } + for (k, v) in network_policies.iter() { + network_policy_by_name.insert(v.name.clone(), k.clone()); + } + for (k, v) in items.iter() { + let bucket = item_by_namespace + .entry((v.schema_id, v.name.clone())) + .or_default(); + bucket.push_back((k.clone(), v.item_type())); + oid_owner.insert(v.oid, OidOwner::Item(k.clone())); + } + for (k, v) in introspection_sources.iter() { + oid_owner.insert(v.oid, OidOwner::IntrospectionSource(k.clone())); + } + + DurableCatalogIndexes { + database_by_name: Arc::new(database_by_name), + schema_by_parent_name: Arc::new(schema_by_parent_name), + role_by_name: Arc::new(role_by_name), + cluster_by_name: Arc::new(cluster_by_name), + replica_by_cluster_and_name: Arc::new(replica_by_cluster_and_name), + network_policy_by_name: Arc::new(network_policy_by_name), + item_by_namespace: Arc::new(item_by_namespace), + oid_owner: Arc::new(oid_owner), + } + } +} + +/// Identifies which durable table owns a given OID, for the `oid_owner` index. +#[derive(Debug, Clone, PartialEq, Eq)] +pub(crate) enum OidOwner { + Database(DatabaseKey), + Schema(SchemaKey), + Role(RoleKey), + Item(ItemKey), + IntrospectionSource(ClusterIntrospectionSourceIndexKey), +} + +/// Apply a `+1`/`-1` diff against an `Arc>` table. +/// +/// Mirrors the insertion/retraction assertions in `with_snapshot`. Mutates +/// through `Arc::make_mut`, so transactions holding the old `Arc` keep their +/// frozen view. +fn apply_durable(map: &mut Arc>, key: K, value: V, diff: Diff) +where + K: Ord + Clone + Debug, + V: Clone + Debug + PartialEq, +{ + let m = Arc::make_mut(map); + if diff == Diff::ONE { + let prev = m.insert(key, value); + assert_eq!( + prev, None, + "values must be explicitly retracted before inserting a new value" + ); + } else if diff == Diff::MINUS_ONE { + let prev = m.remove(&key); + assert_eq!( + prev, + Some(value), + "retraction does not match existing value" + ); + } else { + panic!("invalid diff: {diff:?}"); + } +} + +/// Apply a `+1`/`-1` diff to a name-keyed `Arc>` index. +fn apply_index(map: &mut Arc>, name: N, key: K, diff: Diff) +where + N: Ord + Clone + Debug, + K: Clone + Debug + PartialEq, +{ + let m = Arc::make_mut(map); + if diff == Diff::ONE { + let prev = m.insert(name, key); + assert_eq!(prev, None, "index entry already present for insert"); + } else if diff == Diff::MINUS_ONE { + let prev = m.remove(&name); + assert_eq!(prev, Some(key), "index retraction does not match"); + } else { + panic!("invalid diff: {diff:?}"); + } +} + +/// Apply a `+1`/`-1` diff to the bucketed `item_by_namespace` index. +fn apply_item_index( + map: &mut Arc>>, + schema_id: SchemaId, + name: String, + key: ItemKey, + item_type: CatalogItemType, + diff: Diff, +) { + let m = Arc::make_mut(map); + let bucket_key = (schema_id, name); + if diff == Diff::ONE { + let mut bucket = m.get(&bucket_key).cloned().unwrap_or_default(); + debug_assert!( + !bucket.iter().any(|(k, _)| k == &key), + "item already present in namespace bucket" + ); + bucket.push_back((key, item_type)); + m.insert(bucket_key, bucket); + } else if diff == Diff::MINUS_ONE { + let mut bucket = m + .get(&bucket_key) + .cloned() + .expect("retraction with no existing namespace bucket"); + let before = bucket.len(); + bucket.retain(|(k, _)| k != &key); + debug_assert_eq!(bucket.len() + 1, before, "expected exactly one match"); + if bucket.is_empty() { + m.remove(&bucket_key); + } else { + m.insert(bucket_key, bucket); + } + } else { + panic!("invalid diff: {diff:?}"); + } +} + +/// Apply a `+1`/`-1` diff to the `oid_owner` index. +fn apply_oid_owner( + map: &mut Arc>, + oid: u32, + owner: OidOwner, + diff: Diff, +) { + let m = Arc::make_mut(map); + if diff == Diff::ONE { + let prev = m.insert(oid, owner); + assert_eq!(prev, None, "oid already owned"); + } else if diff == Diff::MINUS_ONE { + let prev = m.remove(&oid); + assert_eq!(prev, Some(owner), "oid owner mismatch on retraction"); + } else { + panic!("invalid diff: {diff:?}"); + } +} + /// Applies updates for an opened catalog. #[derive(Debug)] struct CatalogStateInner { /// A trace of all catalog updates that can be consumed by some higher layer. updates: VecDeque, + /// Materialised, indexed view of the durable catalog tables. Cheap to + /// clone into a transaction (one `Arc::clone` per field). + data: DurableCatalogData, } impl CatalogStateInner { fn new() -> CatalogStateInner { CatalogStateInner { updates: VecDeque::new(), + data: DurableCatalogData::new(), } } } @@ -1612,6 +1966,221 @@ impl ApplyUpdate for CatalogStateInner { } } + let diff = update.diff; + if diff == Diff::ONE || diff == Diff::MINUS_ONE { + match &update.kind { + StateUpdateKind::AuditLog(_, ()) | StateUpdateKind::FenceToken(_) => {} + StateUpdateKind::Database(key, value) => { + let key: DatabaseKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: DatabaseValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_index( + &mut self.data.indexes.database_by_name, + value.name.clone(), + key, + diff, + ); + apply_oid_owner( + &mut self.data.indexes.oid_owner, + value.oid, + OidOwner::Database(key), + diff, + ); + apply_durable(&mut self.data.databases, key, value, diff); + } + StateUpdateKind::Schema(key, value) => { + let key: SchemaKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: SchemaValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_index( + &mut self.data.indexes.schema_by_parent_name, + (value.database_id, value.name.clone()), + key, + diff, + ); + apply_oid_owner( + &mut self.data.indexes.oid_owner, + value.oid, + OidOwner::Schema(key), + diff, + ); + apply_durable(&mut self.data.schemas, key, value, diff); + } + StateUpdateKind::Role(key, value) => { + let key: RoleKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: RoleValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_index( + &mut self.data.indexes.role_by_name, + value.name.clone(), + key.clone(), + diff, + ); + apply_oid_owner( + &mut self.data.indexes.oid_owner, + value.oid, + OidOwner::Role(key.clone()), + diff, + ); + apply_durable(&mut self.data.roles, key, value, diff); + } + StateUpdateKind::RoleAuth(key, value) => { + let key: RoleAuthKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: RoleAuthValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.role_auth, key, value, diff); + } + StateUpdateKind::Item(key, value) => { + let key: ItemKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: ItemValue = + RustType::from_proto(value.clone()).expect("invalid value"); + let item_type = value.item_type(); + apply_item_index( + &mut self.data.indexes.item_by_namespace, + value.schema_id, + value.name.clone(), + key.clone(), + item_type, + diff, + ); + apply_oid_owner( + &mut self.data.indexes.oid_owner, + value.oid, + OidOwner::Item(key.clone()), + diff, + ); + apply_durable(&mut self.data.items, key, value, diff); + } + StateUpdateKind::Comment(key, value) => { + let key: CommentKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: CommentValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.comments, key, value, diff); + } + StateUpdateKind::Cluster(key, value) => { + let key: ClusterKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: ClusterValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_index( + &mut self.data.indexes.cluster_by_name, + value.name.clone(), + key.clone(), + diff, + ); + apply_durable(&mut self.data.clusters, key, value, diff); + } + StateUpdateKind::NetworkPolicy(key, value) => { + let key: NetworkPolicyKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: NetworkPolicyValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_index( + &mut self.data.indexes.network_policy_by_name, + value.name.clone(), + key.clone(), + diff, + ); + apply_durable(&mut self.data.network_policies, key, value, diff); + } + StateUpdateKind::ClusterReplica(key, value) => { + let key: ClusterReplicaKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: ClusterReplicaValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_index( + &mut self.data.indexes.replica_by_cluster_and_name, + (value.cluster_id, value.name.clone()), + key.clone(), + diff, + ); + apply_durable(&mut self.data.cluster_replicas, key, value, diff); + } + StateUpdateKind::IntrospectionSourceIndex(key, value) => { + let key: ClusterIntrospectionSourceIndexKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: ClusterIntrospectionSourceIndexValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_oid_owner( + &mut self.data.indexes.oid_owner, + value.oid, + OidOwner::IntrospectionSource(key.clone()), + diff, + ); + apply_durable(&mut self.data.introspection_sources, key, value, diff); + } + StateUpdateKind::IdAllocator(key, value) => { + let key: IdAllocKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: IdAllocValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.id_allocator, key, value, diff); + } + StateUpdateKind::Config(key, value) => { + let key: ConfigKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: ConfigValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.configs, key, value, diff); + } + StateUpdateKind::Setting(key, value) => { + let key: SettingKey = RustType::from_proto(key.clone()).expect("invalid key"); + let value: SettingValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.settings, key, value, diff); + } + StateUpdateKind::SourceReferences(key, value) => { + let key: SourceReferencesKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: SourceReferencesValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.source_references, key, value, diff); + } + StateUpdateKind::SystemObjectMapping(key, value) => { + let key: GidMappingKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: GidMappingValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.system_object_mappings, key, value, diff); + } + StateUpdateKind::SystemConfiguration(key, value) => { + let key: ServerConfigurationKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: ServerConfigurationValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.system_configurations, key, value, diff); + } + StateUpdateKind::DefaultPrivilege(key, value) => { + let key: DefaultPrivilegesKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: DefaultPrivilegesValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.default_privileges, key, value, diff); + } + StateUpdateKind::SystemPrivilege(key, value) => { + let key: SystemPrivilegesKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: SystemPrivilegesValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.system_privileges, key, value, diff); + } + StateUpdateKind::StorageCollectionMetadata(key, value) => { + let key: StorageCollectionMetadataKey = + RustType::from_proto(key.clone()).expect("invalid key"); + let value: StorageCollectionMetadataValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.storage_collection_metadata, key, value, diff); + } + StateUpdateKind::UnfinalizedShard(key, ()) => { + let key: UnfinalizedShardKey = + RustType::from_proto(key.clone()).expect("invalid key"); + apply_durable(&mut self.data.unfinalized_shards, key, (), diff); + } + StateUpdateKind::TxnWalShard((), value) => { + let value: TxnWalShardValue = + RustType::from_proto(value.clone()).expect("invalid value"); + apply_durable(&mut self.data.txn_wal_shard, (), value, diff); + } + } + } + match (update.kind, update.diff) { (StateUpdateKind::AuditLog(_, ()), _) => Ok(None), // Nothing to due for fence token retractions but wait for the next insertion. @@ -1775,9 +2344,12 @@ impl DurableCatalogState for PersistCatalogState { #[mz_ore::instrument(level = "debug")] async fn transaction(&mut self) -> Result { self.metrics.transactions_started.inc(); - let snapshot = self.snapshot().await?; + self.sync_to_current_upper().await?; + let data = self.update_applier.data.clone(); let commit_ts = self.upper.clone(); - Transaction::new(self, snapshot, commit_ts) + let bootstrap_complete = self.bootstrap_complete; + let is_savepoint = matches!(self.mode, Mode::Savepoint); + Transaction::new_from_durable_data(bootstrap_complete, is_savepoint, data, commit_ts) } fn transaction_from_snapshot( @@ -1785,7 +2357,19 @@ impl DurableCatalogState for PersistCatalogState { snapshot: Snapshot, ) -> Result { let commit_ts = self.upper.clone(); - Transaction::new(self, snapshot, commit_ts) + let bootstrap_complete = self.bootstrap_complete; + let is_savepoint = matches!(self.mode, Mode::Savepoint); + Transaction::new(bootstrap_complete, is_savepoint, snapshot, commit_ts) + } + + fn transaction_from_durable_data( + &mut self, + data: DurableCatalogData, + ) -> Result { + let commit_ts = self.upper.clone(); + let bootstrap_complete = self.bootstrap_complete; + let is_savepoint = matches!(self.mode, Mode::Savepoint); + Transaction::new_from_durable_data(bootstrap_complete, is_savepoint, data, commit_ts) } #[mz_ore::instrument(level = "debug")] diff --git a/src/catalog/src/durable/transaction.rs b/src/catalog/src/durable/transaction.rs index ca8b8c958b46b..9a127434b0672 100644 --- a/src/catalog/src/durable/transaction.rs +++ b/src/catalog/src/durable/transaction.rs @@ -10,6 +10,7 @@ use std::collections::{BTreeMap, BTreeSet}; use std::fmt::Debug; use std::num::NonZeroU32; +use std::sync::Arc; use std::time::Duration; use anyhow::anyhow; @@ -77,10 +78,12 @@ type Timestamp = u64; /// An operation also logically groups multiple catalog updates together. #[derive(Derivative)] #[derivative(Debug)] -pub struct Transaction<'a> { - #[derivative(Debug = "ignore")] - #[derivative(PartialEq = "ignore")] - durable_catalog: &'a mut dyn DurableCatalogState, +pub struct Transaction { + /// Captured from the source [`DurableCatalogState`] at construction time so + /// the transaction can be moved (e.g. into `TransactionOps::DDL`) without + /// holding a borrow on the storage. + bootstrap_complete: bool, + is_savepoint: bool, databases: TableTransaction, schemas: TableTransaction, items: TableTransaction, @@ -111,12 +114,37 @@ pub struct Transaction<'a> { upper: mz_repr::Timestamp, /// The ID of the current operation of this transaction. op_id: Timestamp, + /// Frozen view of the durable `oid_owner` index at transaction start. + /// Captures which OIDs are taken across the five OID-bearing tables + /// (`databases`, `schemas`, `roles`, `items`, `introspection_sources`) + /// as of the transaction's snapshot. + /// + /// Constructed via `Arc::clone` from the shared + /// [`crate::durable::persist::DurableCatalogIndexes::oid_owner`] — O(1), + /// no walk. In-flight allocations and removals on top of this view are + /// folded in per-call inside [`Transaction::allocate_oids`] via + /// `pending_added` / `pending_removed`. + initial_oid_owner: Arc>, } -impl<'a> Transaction<'a> { +impl Transaction { pub fn new( - durable_catalog: &'a mut dyn DurableCatalogState, - Snapshot { + bootstrap_complete: bool, + is_savepoint: bool, + snapshot: Snapshot, + upper: mz_repr::Timestamp, + ) -> Result { + let data = snapshot_to_durable_data(snapshot)?; + Self::new_from_durable_data(bootstrap_complete, is_savepoint, data, upper) + } + + pub fn new_from_durable_data( + bootstrap_complete: bool, + is_savepoint: bool, + data: crate::durable::persist::DurableCatalogData, + upper: mz_repr::Timestamp, + ) -> Result { + let crate::durable::persist::DurableCatalogData { databases, schemas, roles, @@ -138,19 +166,39 @@ impl<'a> Transaction<'a> { storage_collection_metadata, unfinalized_shards, txn_wal_shard, - }: Snapshot, - upper: mz_repr::Timestamp, - ) -> Result, CatalogError> { - Ok(Transaction { - durable_catalog, - databases: TableTransaction::new_with_uniqueness_fn( - databases, - |a: &DatabaseValue, b| a.name == b.name, - )?, - schemas: TableTransaction::new_with_uniqueness_fn(schemas, |a: &SchemaValue, b| { - a.database_id == b.database_id && a.name == b.name - })?, - items: TableTransaction::new_with_uniqueness_fn(items, |a: &ItemValue, b| { + indexes, + } = data; + + // Capture the durable `oid_owner` view as a frozen `Arc`. This is the + // moral equivalent of the previous `initial_oids: BTreeSet` build + // — but `Arc::clone` is O(1), and the shared index is maintained + // incrementally on the trace-apply path rather than walked here. + let initial_oid_owner = Arc::clone(&indexes.oid_owner); + + let database_by_name = Arc::clone(&indexes.database_by_name); + let databases = TableTransaction::new_with_uniqueness_and_index( + databases, + |a: &DatabaseValue, b| a.name == b.name, + Box::new(move |v: &DatabaseValue| { + database_by_name.get(&v.name).copied().into_iter().collect() + }), + ); + let schema_by_parent_name = Arc::clone(&indexes.schema_by_parent_name); + let schemas = TableTransaction::new_with_uniqueness_and_index( + schemas, + |a: &SchemaValue, b| a.database_id == b.database_id && a.name == b.name, + Box::new(move |v: &SchemaValue| { + schema_by_parent_name + .get(&(v.database_id, v.name.clone())) + .copied() + .into_iter() + .collect() + }), + ); + let item_by_namespace = Arc::clone(&indexes.item_by_namespace); + let items = TableTransaction::new_with_uniqueness_and_index( + items, + |a: &ItemValue, b| { a.schema_id == b.schema_id && a.name == b.name && { // `item_type` is slow, only compute if needed. let a_type = a.item_type(); @@ -159,41 +207,87 @@ impl<'a> Transaction<'a> { || (a_type == CatalogItemType::Type && b_type.conflicts_with_type()) || (b_type == CatalogItemType::Type && a_type.conflicts_with_type()) } - })?, - comments: TableTransaction::new(comments)?, - roles: TableTransaction::new_with_uniqueness_fn(roles, |a: &RoleValue, b| { - a.name == b.name - })?, - role_auth: TableTransaction::new(role_auth)?, - clusters: TableTransaction::new_with_uniqueness_fn(clusters, |a: &ClusterValue, b| { - a.name == b.name - })?, - network_policies: TableTransaction::new_with_uniqueness_fn( - network_policies, - |a: &NetworkPolicyValue, b| a.name == b.name, - )?, - cluster_replicas: TableTransaction::new_with_uniqueness_fn( - cluster_replicas, - |a: &ClusterReplicaValue, b| a.cluster_id == b.cluster_id && a.name == b.name, - )?, - introspection_sources: TableTransaction::new(introspection_sources)?, - id_allocator: TableTransaction::new(id_allocator)?, - configs: TableTransaction::new(configs)?, - settings: TableTransaction::new(settings)?, - source_references: TableTransaction::new(source_references)?, - system_gid_mapping: TableTransaction::new(system_object_mappings)?, - system_configurations: TableTransaction::new(system_configurations)?, - default_privileges: TableTransaction::new(default_privileges)?, - system_privileges: TableTransaction::new(system_privileges)?, - storage_collection_metadata: TableTransaction::new(storage_collection_metadata)?, - unfinalized_shards: TableTransaction::new(unfinalized_shards)?, + }, + Box::new(move |v: &ItemValue| { + item_by_namespace + .get(&(v.schema_id, v.name.clone())) + .into_iter() + .flat_map(|bucket| bucket.iter().map(|(k, _t)| k.clone())) + .collect() + }), + ); + let role_by_name = Arc::clone(&indexes.role_by_name); + let roles = TableTransaction::new_with_uniqueness_and_index( + roles, + |a: &RoleValue, b| a.name == b.name, + Box::new(move |v: &RoleValue| role_by_name.get(&v.name).cloned().into_iter().collect()), + ); + let introspection_sources = TableTransaction::new(introspection_sources); + + let cluster_by_name = Arc::clone(&indexes.cluster_by_name); + let clusters = TableTransaction::new_with_uniqueness_and_index( + clusters, + |a: &ClusterValue, b| a.name == b.name, + Box::new(move |v: &ClusterValue| { + cluster_by_name.get(&v.name).cloned().into_iter().collect() + }), + ); + let network_policy_by_name = Arc::clone(&indexes.network_policy_by_name); + let network_policies = TableTransaction::new_with_uniqueness_and_index( + network_policies, + |a: &NetworkPolicyValue, b| a.name == b.name, + Box::new(move |v: &NetworkPolicyValue| { + network_policy_by_name + .get(&v.name) + .cloned() + .into_iter() + .collect() + }), + ); + let replica_by_cluster_and_name = Arc::clone(&indexes.replica_by_cluster_and_name); + let cluster_replicas = TableTransaction::new_with_uniqueness_and_index( + cluster_replicas, + |a: &ClusterReplicaValue, b| a.cluster_id == b.cluster_id && a.name == b.name, + Box::new(move |v: &ClusterReplicaValue| { + replica_by_cluster_and_name + .get(&(v.cluster_id, v.name.clone())) + .cloned() + .into_iter() + .collect() + }), + ); + + Ok(Transaction { + bootstrap_complete, + is_savepoint, + databases, + schemas, + items, + comments: TableTransaction::new(comments), + roles, + role_auth: TableTransaction::new(role_auth), + clusters, + network_policies, + cluster_replicas, + introspection_sources, + id_allocator: TableTransaction::new(id_allocator), + configs: TableTransaction::new(configs), + settings: TableTransaction::new(settings), + source_references: TableTransaction::new(source_references), + system_gid_mapping: TableTransaction::new(system_object_mappings), + system_configurations: TableTransaction::new(system_configurations), + default_privileges: TableTransaction::new(default_privileges), + system_privileges: TableTransaction::new(system_privileges), + storage_collection_metadata: TableTransaction::new(storage_collection_metadata), + unfinalized_shards: TableTransaction::new(unfinalized_shards), // Uniqueness violations for this value occur at the key rather than // the value (the key is the unit struct `()` so this is a singleton // value). - txn_wal_shard: TableTransaction::new(txn_wal_shard)?, + txn_wal_shard: TableTransaction::new(txn_wal_shard), audit_log_updates: Vec::new(), upper, op_id: 0, + initial_oid_owner, }) } @@ -744,7 +838,7 @@ impl<'a> Transaction<'a> { amount: u64, ) -> Result, CatalogError> { assert!( - key != SYSTEM_ITEM_ALLOC_KEY || !self.durable_catalog.is_bootstrap_complete(), + key != SYSTEM_ITEM_ALLOC_KEY || !self.bootstrap_complete, "system item IDs cannot be allocated outside of bootstrap" ); @@ -776,7 +870,7 @@ impl<'a> Transaction<'a> { amount: u64, ) -> Result, CatalogError> { assert!( - !self.durable_catalog.is_bootstrap_complete(), + !self.bootstrap_complete, "we can only allocate system item IDs during bootstrap" ); Ok(self @@ -941,36 +1035,80 @@ impl<'a> Transaction<'a> { return Err(CatalogError::Catalog(SqlCatalogError::OidExhaustion)); } - // This is potentially slow to do everytime we allocate an OID. A faster approach might be - // to have an ID allocator that is updated everytime an OID is allocated or de-allocated. - // However, benchmarking shows that this doesn't make a noticeable difference and the other - // approach requires making sure that allocator always stays in-sync which can be - // error-prone. If DDL starts slowing down, this is a good place to try and optimize. - let mut allocated_oids = HashSet::with_capacity( - self.databases.len() - + self.schemas.len() - + self.roles.len() - + self.items.len() - + self.introspection_sources.len() - + temporary_oids.len(), + // Compute the set of OIDs currently in use by the five OID-bearing + // tables. The initial state is shared via `self.initial_oid_owner` + // (an `Arc`-cloned snapshot of the durable `oid_owner` index, frozen + // at transaction start). Here we only need to fold in pending changes + // for this transaction. In the common case there are very few pending + // changes per call, so this is effectively O(|pending| · log N) per + // allocation, rather than the previous O(|catalog|) full scan. + // + // For each pending key we compute the *delta* relative to its initial + // value: the initial OID (if any) is removed (it might be re-taken in + // a moment), and the current effective OID (if the key still exists in + // the txn view) is added. The same OID appearing in both adds and + // removes nets to "still taken" via `pending_added`. + let mut pending_added: BTreeSet = BTreeSet::new(); + let mut pending_removed: BTreeSet = BTreeSet::new(); + fn collect_pending( + table: &TableTransaction, + extract_oid: impl Fn(&V) -> u32, + added: &mut BTreeSet, + removed: &mut BTreeSet, + ) where + K: Ord + Eq + Clone + Debug, + V: Ord + Clone + Debug + UniqueName, + { + for k in table.pending.keys() { + if let Some(initial_v) = table.base.get(k) { + removed.insert(extract_oid(initial_v)); + } + if let Some(current_v) = table.get(k) { + added.insert(extract_oid(current_v)); + } + } + } + collect_pending( + &self.databases, + |v| v.oid, + &mut pending_added, + &mut pending_removed, + ); + collect_pending( + &self.schemas, + |v| v.oid, + &mut pending_added, + &mut pending_removed, + ); + collect_pending( + &self.roles, + |v| v.oid, + &mut pending_added, + &mut pending_removed, + ); + collect_pending( + &self.items, + |v| v.oid, + &mut pending_added, + &mut pending_removed, + ); + collect_pending( + &self.introspection_sources, + |v| v.oid, + &mut pending_added, + &mut pending_removed, ); - self.databases.for_values(|_, value| { - allocated_oids.insert(value.oid); - }); - self.schemas.for_values(|_, value| { - allocated_oids.insert(value.oid); - }); - self.roles.for_values(|_, value| { - allocated_oids.insert(value.oid); - }); - self.items.for_values(|_, value| { - allocated_oids.insert(value.oid); - }); - self.introspection_sources.for_values(|_, value| { - allocated_oids.insert(value.oid); - }); - let is_allocated = |oid| allocated_oids.contains(&oid) || temporary_oids.contains(&oid); + let initial_oid_owner = &self.initial_oid_owner; + let is_allocated = |oid: u32| -> bool { + if temporary_oids.contains(&oid) || pending_added.contains(&oid) { + return true; + } + // The OID is "still in the initial set" if it was there originally + // and no pending change removed it. (If pending re-added it under a + // different key, `pending_added` already covered that above.) + initial_oid_owner.contains_key(&oid) && !pending_removed.contains(&oid) + }; let start_oid: u32 = self .id_allocator @@ -1024,6 +1162,56 @@ impl<'a> Transaction<'a> { .map(|oids| oids.into_element()) } + /// Exports the current state of this transaction as a + /// [`crate::durable::persist::DurableCatalogData`]. Cheaper than + /// [`Self::current_snapshot`] because each table is an + /// `Arc` (no proto conversion, structural sharing with the + /// pre-transaction view when no pending changes exist for a table). + pub fn current_durable_data(&self) -> crate::durable::persist::DurableCatalogData { + let databases = self.databases.current_data(); + let schemas = self.schemas.current_data(); + let roles = self.roles.current_data(); + let clusters = self.clusters.current_data(); + let cluster_replicas = self.cluster_replicas.current_data(); + let network_policies = self.network_policies.current_data(); + let items = self.items.current_data(); + let introspection_sources = self.introspection_sources.current_data(); + let indexes = crate::durable::persist::DurableCatalogIndexes::from_tables( + &databases, + &schemas, + &roles, + &clusters, + &cluster_replicas, + &network_policies, + &items, + &introspection_sources, + ); + crate::durable::persist::DurableCatalogData { + databases, + schemas, + roles, + role_auth: self.role_auth.current_data(), + items, + comments: self.comments.current_data(), + clusters, + network_policies, + cluster_replicas, + introspection_sources, + id_allocator: self.id_allocator.current_data(), + configs: self.configs.current_data(), + settings: self.settings.current_data(), + source_references: self.source_references.current_data(), + system_object_mappings: self.system_gid_mapping.current_data(), + system_configurations: self.system_configurations.current_data(), + default_privileges: self.default_privileges.current_data(), + system_privileges: self.system_privileges.current_data(), + storage_collection_metadata: self.storage_collection_metadata.current_data(), + unfinalized_shards: self.unfinalized_shards.current_data(), + txn_wal_shard: self.txn_wal_shard.current_data(), + indexes, + } + } + /// Exports the current state of this transaction as a [`Snapshot`]. /// /// This merges each `TableTransaction`'s initial data with its pending @@ -2327,7 +2515,8 @@ impl<'a> Transaction<'a> { } let Transaction { - durable_catalog: _, + bootstrap_complete: _, + is_savepoint: _, databases, schemas, items, @@ -2353,6 +2542,7 @@ impl<'a> Transaction<'a> { txn_wal_shard: _, upper, op_id: _, + initial_oid_owner: _, } = &self; let updates = std::iter::empty() @@ -2457,7 +2647,7 @@ impl<'a> Transaction<'a> { } pub fn is_savepoint(&self) -> bool { - self.durable_catalog.is_savepoint() + self.is_savepoint } fn commit_op(&mut self) { @@ -2472,14 +2662,14 @@ impl<'a> Transaction<'a> { self.upper } - pub(crate) fn into_parts(self) -> (TransactionBatch, &'a mut dyn DurableCatalogState) { + pub(crate) fn into_parts(self) -> TransactionBatch { let audit_log_updates = self .audit_log_updates .into_iter() .map(|(k, diff, _op)| (k.into_proto(), (), diff)) .collect(); - let txn_batch = TransactionBatch { + TransactionBatch { databases: self.databases.pending(), schemas: self.schemas.pending(), items: self.items.pending(), @@ -2503,8 +2693,7 @@ impl<'a> Transaction<'a> { txn_wal_shard: self.txn_wal_shard.pending(), audit_log_updates, upper: self.upper, - }; - (txn_batch, self.durable_catalog) + } } /// Commits the storage transaction to durable storage. Any error returned outside read-only @@ -2521,9 +2710,10 @@ impl<'a> Transaction<'a> { #[mz_ore::instrument(level = "debug")] pub(crate) async fn commit_internal( self, + storage: &mut dyn DurableCatalogState, commit_ts: mz_repr::Timestamp, - ) -> Result<(&'a mut dyn DurableCatalogState, mz_repr::Timestamp), CatalogError> { - let (mut txn_batch, durable_catalog) = self.into_parts(); + ) -> Result { + let mut txn_batch = self.into_parts(); let TransactionBatch { databases, schemas, @@ -2574,10 +2764,8 @@ impl<'a> Transaction<'a> { differential_dataflow::consolidation::consolidate_updates(txn_wal_shard); differential_dataflow::consolidation::consolidate_updates(audit_log_updates); - let upper = durable_catalog - .commit_transaction(txn_batch, commit_ts) - .await?; - Ok((durable_catalog, upper)) + let upper = storage.commit_transaction(txn_batch, commit_ts).await?; + Ok(upper) } /// Commits the storage transaction to durable storage. Any error returned outside read-only @@ -2597,22 +2785,26 @@ impl<'a> Transaction<'a> { /// after committing and only then apply the updates in-memory. While this removes assumptions /// about the caller in this method, in practice it results in duplicate work on every commit. #[mz_ore::instrument(level = "debug")] - pub async fn commit(self, commit_ts: mz_repr::Timestamp) -> Result<(), CatalogError> { + pub async fn commit( + self, + storage: &mut dyn DurableCatalogState, + commit_ts: mz_repr::Timestamp, + ) -> Result<(), CatalogError> { let op_updates = self.get_op_updates(); assert!( op_updates.is_empty(), "unconsumed transaction updates: {op_updates:?}" ); - let (durable_storage, upper) = self.commit_internal(commit_ts).await?; + let upper = self.commit_internal(storage, commit_ts).await?; // Drain all the updates from the commit since it is assumed that they were already applied. - let updates = durable_storage.sync_updates(upper).await?; + let updates = storage.sync_updates(upper).await?; // Writable and savepoint catalogs should have consumed all updates before committing a // transaction, otherwise the commit was performed with an out of date state. // Read-only catalogs can only commit empty transactions, so they don't need to consume all // updates before committing. soft_assert_no_log!( - durable_storage.is_read_only() || updates.iter().all(|update| update.ts == commit_ts), + storage.is_read_only() || updates.iter().all(|update| update.ts == commit_ts), "unconsumed updates existed before transaction commit: commit_ts={commit_ts:?}, updates:{updates:?}" ); Ok(()) @@ -2624,7 +2816,7 @@ use crate::durable::async_trait; use super::objects::{RoleAuthKey, RoleAuthValue}; #[async_trait] -impl StorageTxn for Transaction<'_> { +impl StorageTxn for Transaction { fn get_collection_metadata(&self) -> BTreeMap { self.storage_collection_metadata .items() @@ -2638,6 +2830,22 @@ impl StorageTxn for Transaction<'_> { .collect() } + fn get_collection_shard(&self, id: GlobalId) -> Option { + self.storage_collection_metadata + .get(&StorageCollectionMetadataKey { id }) + .map(|StorageCollectionMetadataValue { shard }| *shard) + } + + fn collection_metadata_contains_shard(&self, shard: ShardId) -> bool { + let mut found = false; + self.storage_collection_metadata.for_values(|_, v| { + if v.shard == shard { + found = true; + } + }); + found + } + fn insert_collection_metadata( &mut self, metadata: BTreeMap, @@ -2834,6 +3042,98 @@ impl TransactionBatch { } } +/// Convert a proto-typed [`Snapshot`] into a Rust-typed +/// [`crate::durable::persist::DurableCatalogData`]. +/// +/// This pays the O(N) proto→Rust conversion cost. The hot path in +/// [`DurableCatalogState::transaction`] avoids it by handing the already- +/// materialised [`crate::durable::persist::DurableCatalogData`] directly to +/// [`Transaction::new_from_durable_data`]. +fn snapshot_to_durable_data( + snapshot: Snapshot, +) -> Result { + fn convert( + m: BTreeMap, + ) -> Result>, TryFromProtoError> + where + K: RustType + Ord + Clone, + V: RustType + Clone, + { + let mut out = imbl::OrdMap::new(); + for (k, v) in m { + out.insert(K::from_proto(k)?, V::from_proto(v)?); + } + Ok(Arc::new(out)) + } + + let Snapshot { + databases, + schemas, + roles, + role_auth, + items, + comments, + clusters, + network_policies, + cluster_replicas, + introspection_sources, + id_allocator, + configs, + settings, + source_references, + system_object_mappings, + system_configurations, + default_privileges, + system_privileges, + storage_collection_metadata, + unfinalized_shards, + txn_wal_shard, + } = snapshot; + + let databases = convert(databases)?; + let schemas = convert(schemas)?; + let roles = convert(roles)?; + let clusters = convert(clusters)?; + let cluster_replicas = convert(cluster_replicas)?; + let network_policies = convert(network_policies)?; + let items = convert(items)?; + let introspection_sources = convert(introspection_sources)?; + let indexes = crate::durable::persist::DurableCatalogIndexes::from_tables( + &databases, + &schemas, + &roles, + &clusters, + &cluster_replicas, + &network_policies, + &items, + &introspection_sources, + ); + Ok(crate::durable::persist::DurableCatalogData { + databases, + schemas, + roles, + role_auth: convert(role_auth)?, + items, + comments: convert(comments)?, + clusters, + network_policies, + cluster_replicas, + introspection_sources, + id_allocator: convert(id_allocator)?, + configs: convert(configs)?, + settings: convert(settings)?, + source_references: convert(source_references)?, + system_object_mappings: convert(system_object_mappings)?, + system_configurations: convert(system_configurations)?, + default_privileges: convert(default_privileges)?, + system_privileges: convert(system_privileges)?, + storage_collection_metadata: convert(storage_collection_metadata)?, + unfinalized_shards: convert(unfinalized_shards)?, + txn_wal_shard: convert(txn_wal_shard)?, + indexes, + }) +} + #[derive(Debug, Clone, PartialEq, Eq)] struct TransactionUpdate { value: V, @@ -2921,13 +3221,24 @@ mod unique_name { /// /// `K` is the primary key type. Multiple entries with the same key are disallowed. /// `V` is the an arbitrary value type. -#[derive(Debug)] +/// +/// Reads overlay `pending` on top of `base`: a key visible in `pending` +/// (insertion or retraction) shadows its entry in `base`. +#[derive(Derivative)] +#[derivative(Debug)] struct TableTransaction { - initial: BTreeMap, + #[derivative(Debug = "ignore")] + base: Arc>, // The desired updates to keys after commit. // Invariant: Value is sorted by `ts`. pending: BTreeMap>>, + #[derivative(Debug = "ignore")] uniqueness_violation: Option bool>, + /// Returns the set of keys in `base` whose value would collide with `v` + /// under the uniqueness predicate. Implemented via an index, so this is + /// O(log N + collisions) instead of O(N). + #[derivative(Debug = "ignore")] + index_probe: Option Vec + Send + Sync>>, } impl TableTransaction @@ -2935,49 +3246,45 @@ where K: Ord + Eq + Clone + Debug, V: Ord + Clone + Debug + UniqueName, { - /// Create a new TableTransaction with initial data. - /// - /// Internally the catalog serializes data as protobuf. All fields in a proto message are - /// optional, which makes using them in Rust cumbersome. Generic parameters `KP` and `VP` are - /// protobuf types which deserialize to `K` and `V` that a [`TableTransaction`] is generic - /// over. - fn new(initial: BTreeMap) -> Result - where - K: RustType, - V: RustType, - { - let initial = initial - .into_iter() - .map(RustType::from_proto) - .collect::>()?; - - Ok(Self { - initial, + /// Create a new TableTransaction sharing `base` as its initial view. + fn new(base: Arc>) -> Self { + Self { + base, pending: BTreeMap::new(), uniqueness_violation: None, - }) + index_probe: None, + } } /// Like [`Self::new`], but you can also provide `uniqueness_violation`, which is a function /// that determines whether there is a uniqueness violation among two values. - fn new_with_uniqueness_fn( - initial: BTreeMap, + #[allow(dead_code)] + fn new_with_uniqueness_fn( + base: Arc>, uniqueness_violation: fn(a: &V, b: &V) -> bool, - ) -> Result - where - K: RustType, - V: RustType, - { - let initial = initial - .into_iter() - .map(RustType::from_proto) - .collect::>()?; + ) -> Self { + Self { + base, + pending: BTreeMap::new(), + uniqueness_violation: Some(uniqueness_violation), + index_probe: None, + } + } - Ok(Self { - initial, + /// Like [`Self::new_with_uniqueness_fn`], but additionally supplies an + /// index-backed candidate generator for the `base` side of the uniqueness + /// check. + fn new_with_uniqueness_and_index( + base: Arc>, + uniqueness_violation: fn(a: &V, b: &V) -> bool, + index_probe: Box Vec + Send + Sync>, + ) -> Self { + Self { + base, pending: BTreeMap::new(), uniqueness_violation: Some(uniqueness_violation), - }) + index_probe: Some(index_probe), + } } /// Consumes and returns the pending changes and their diffs. `Diff` is @@ -3048,7 +3355,8 @@ where /// Verifies that no items in `self` violate `self.uniqueness_violation` with `keys`. /// /// Runtime is O(n * k), where n is the number of items in `self` and k is the number of - /// items in `keys`. + /// items in `keys`. With an `index_probe`, the `base` side is O((log N + collisions) * k), + /// and only `pending` is walked linearly. fn verify_keys<'a>( &self, keys: impl IntoIterator, @@ -3061,11 +3369,35 @@ where .into_iter() .filter_map(|key| self.get(key).map(|value| (key, value))) .collect(); - // Compare each value in `entries` to each value in `self` and ensure they are unique. - for (ki, vi) in self.items() { + if let Some(index_probe) = &self.index_probe { for (kj, vj) in &entries { - if ki != *kj && uniqueness_violation(vi, vj) { - return Err(DurableCatalogError::UniquenessViolation); + for cand in index_probe(vj) { + if &cand == *kj { + continue; + } + if let Some(vi) = self.get(&cand) { + if uniqueness_violation(vi, vj) { + return Err(DurableCatalogError::UniquenessViolation); + } + } + } + for pk in self.pending.keys() { + if pk == *kj { + continue; + } + if let Some(vi) = self.get(pk) { + if uniqueness_violation(vi, vj) { + return Err(DurableCatalogError::UniquenessViolation); + } + } + } + } + } else { + for (ki, vi) in self.items() { + for (kj, vj) in &entries { + if ki != *kj && uniqueness_violation(vi, vj) { + return Err(DurableCatalogError::UniquenessViolation); + } } } } @@ -3087,7 +3419,7 @@ where f(k, v); } } - for (k, v) in self.initial.iter() { + for (k, v) in self.base.iter() { // Add on initial items that don't have updates. if !seen.contains(k) { f(k, v); @@ -3099,7 +3431,7 @@ where fn get(&self, k: &K) -> Option<&V> { let pending = self.pending.get(k).map(Vec::as_slice).unwrap_or_default(); let mut updates = Vec::with_capacity(pending.len() + 1); - if let Some(initial) = self.initial.get(k) { + if let Some(initial) = self.base.get(k) { updates.push((initial, Diff::ONE)); } updates.extend( @@ -3142,6 +3474,27 @@ where items } + /// Returns the current state of the table as an `Arc`, + /// folding `pending` into `base`. Reuses the shared `base` if no pending + /// updates exist (cheap `Arc::clone`). + fn current_data(&self) -> Arc> { + if self.pending.is_empty() { + return Arc::clone(&self.base); + } + let mut out: imbl::OrdMap = (*self.base).clone(); + for k in self.pending.keys() { + match self.get(k) { + Some(v) => { + out.insert(k.clone(), v.clone()); + } + None => { + out.remove(k); + } + } + } + Arc::new(out) + } + /// Returns the items viewable in the current transaction as references. Returns a map /// of references. fn items(&self) -> BTreeMap<&K, &V> { @@ -3162,6 +3515,7 @@ where } /// Returns the number of items viewable in the current transaction. + #[allow(dead_code)] fn len(&self) -> usize { let mut count = 0; self.for_values(|_, _| { @@ -3188,19 +3542,38 @@ where /// /// Returns an error if the uniqueness check failed or the key already exists. fn insert(&mut self, k: K, v: V, ts: Timestamp) -> Result<(), DurableCatalogError> { - let mut violation = None; - self.for_values(|for_k, for_v| { - if &k == for_k { - violation = Some(DurableCatalogError::DuplicateKey); - } - if let Some(uniqueness_violation) = self.uniqueness_violation { - if uniqueness_violation(for_v, &v) { - violation = Some(DurableCatalogError::UniquenessViolation); + if self.get(&k).is_some() { + return Err(DurableCatalogError::DuplicateKey); + } + if let Some(uniqueness_violation) = self.uniqueness_violation { + if let Some(index_probe) = &self.index_probe { + // Fast path: probe the index for candidates in `base`, then + // walk only `pending`. + for cand in index_probe(&v) { + if let Some(for_v) = self.get(&cand) { + if uniqueness_violation(for_v, &v) { + return Err(DurableCatalogError::UniquenessViolation); + } + } + } + for pk in self.pending.keys() { + if let Some(for_v) = self.get(pk) { + if uniqueness_violation(for_v, &v) { + return Err(DurableCatalogError::UniquenessViolation); + } + } + } + } else { + let mut violation = false; + self.for_values(|_, for_v| { + if uniqueness_violation(for_v, &v) { + violation = true; + } + }); + if violation { + return Err(DurableCatalogError::UniquenessViolation); } } - }); - if let Some(violation) = violation { - return Err(violation); } self.pending.entry(k).or_default().push(TransactionUpdate { value: v, @@ -3478,16 +3851,22 @@ mod tests { use crate::durable::{TestCatalogStateBuilder, test_bootstrap_args}; use crate::memory; + fn base(m: BTreeMap) -> Arc> { + Arc::new(m.into_iter().collect()) + } + #[mz_ore::test] fn test_table_transaction_simple() { fn uniqueness_violation(a: &String, b: &String) -> bool { a == b } let mut table = TableTransaction::new_with_uniqueness_fn( - BTreeMap::from([(1i64.to_le_bytes().to_vec(), "a".to_string())]), + base(BTreeMap::from([( + 1i64.to_le_bytes().to_vec(), + "a".to_string(), + )])), uniqueness_violation, - ) - .unwrap(); + ); // Ideally, we compare for errors here, but it's hard/impossible to implement PartialEq // for DurableCatalogError. @@ -3505,6 +3884,52 @@ mod tests { ); } + /// Exercises the `index_probe` fast path: the probe maps the value's + /// uniqueness key (the name itself, for this `String -> String` table) + /// back to the matching base key. + #[mz_ore::test] + fn test_table_transaction_index_probe() { + fn uniqueness_violation(a: &String, b: &String) -> bool { + a == b + } + let base_map = base(BTreeMap::from([ + ("k1".to_string(), "alice".to_string()), + ("k2".to_string(), "bob".to_string()), + ])); + // Build an index mapping value -> key, mirroring `database_by_name`. + let mut index: imbl::OrdMap = imbl::OrdMap::new(); + for (k, v) in base_map.iter() { + index.insert(v.clone(), k.clone()); + } + let index = Arc::new(index); + let probe_index = Arc::clone(&index); + let mut table = TableTransaction::new_with_uniqueness_and_index( + Arc::clone(&base_map), + uniqueness_violation, + Box::new(move |v: &String| probe_index.get(v).cloned().into_iter().collect()), + ); + + // Non-conflicting insert succeeds. + assert_ok!(table.insert("k3".to_string(), "carol".to_string(), 0)); + // Conflicting insert is rejected via the index probe (would otherwise + // require a full scan to detect). + assert!( + table + .insert("k4".to_string(), "alice".to_string(), 0) + .is_err() + ); + // After retracting "alice", inserting the same value succeeds. + let _ = table.delete_by_key("k1".to_string(), 1); + assert_ok!(table.insert("k4".to_string(), "alice".to_string(), 2)); + // But inserting "carol" again (now a pending insert) is rejected — the + // pending walk must also catch collisions. + assert!( + table + .insert("k5".to_string(), "carol".to_string(), 3) + .is_err() + ); + } + #[mz_ore::test] fn test_table_transaction() { fn uniqueness_violation(a: &String, b: &String) -> bool { @@ -3534,7 +3959,7 @@ mod tests { table.insert(1i64.to_le_bytes().to_vec(), "v1".to_string()); table.insert(2i64.to_le_bytes().to_vec(), "v2".to_string()); let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); assert_eq!(table_txn.items_cloned(), table); assert_eq!(table_txn.delete(|_k, _v| false, 0).len(), 0); assert_eq!(table_txn.delete(|_k, v| v == "v2", 1).len(), 1); @@ -3599,7 +4024,7 @@ mod tests { ); let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); // Deleting then creating an item that has a uniqueness violation should work. assert_eq!( table_txn.delete(|k, _v| k == &1i64.to_le_bytes(), 0).len(), @@ -3651,7 +4076,7 @@ mod tests { ); let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); assert_eq!(table_txn.delete(|_k, _v| true, 0).len(), 3); table_txn .insert(1i64.to_le_bytes().to_vec(), "v1".to_string(), 0) @@ -3664,7 +4089,7 @@ mod tests { ); let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); assert_eq!(table_txn.delete(|_k, _v| true, 0).len(), 1); table_txn .insert(1i64.to_le_bytes().to_vec(), "v2".to_string(), 0) @@ -3677,7 +4102,7 @@ mod tests { // Verify we don't try to delete v3 or v4 during commit. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); assert_eq!(table_txn.delete(|_k, _v| true, 0).len(), 1); table_txn .insert(1i64.to_le_bytes().to_vec(), "v3".to_string(), 0) @@ -3697,7 +4122,7 @@ mod tests { // Test `set`. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); // Uniqueness violation. table_txn .set(2i64.to_le_bytes().to_vec(), Some("v5".to_string()), 0) @@ -3727,7 +4152,7 @@ mod tests { // Duplicate `set`. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); table_txn .set(3i64.to_le_bytes().to_vec(), Some("v6".to_string()), 0) .unwrap(); @@ -3736,7 +4161,7 @@ mod tests { // Test `set_many`. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); // Uniqueness violation. table_txn .set_many( @@ -3789,7 +4214,7 @@ mod tests { // Duplicate `set_many`. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); table_txn .set_many( BTreeMap::from([ @@ -3812,7 +4237,7 @@ mod tests { // Test `update_by_key` let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); // Uniqueness violation. table_txn .update_by_key(1i64.to_le_bytes().to_vec(), "v7".to_string(), 0) @@ -3850,7 +4275,7 @@ mod tests { // Duplicate `update_by_key`. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); assert!( table_txn .update_by_key(1i64.to_le_bytes().to_vec(), "v8".to_string(), 0) @@ -3869,7 +4294,7 @@ mod tests { // Test `update_by_keys` let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); // Uniqueness violation. table_txn .update_by_keys( @@ -3923,7 +4348,7 @@ mod tests { // Duplicate `update_by_keys`. let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); let n = table_txn .update_by_keys( [ @@ -3947,7 +4372,7 @@ mod tests { // Test `delete_by_key` let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); let prev = table_txn.delete_by_key(1i64.to_le_bytes().to_vec(), 0); assert_eq!(prev, Some("v9".to_string())); let prev = table_txn.delete_by_key(5i64.to_le_bytes().to_vec(), 1); @@ -3971,7 +4396,7 @@ mod tests { // Test `delete_by_keys` let mut table_txn = - TableTransaction::new_with_uniqueness_fn(table.clone(), uniqueness_violation).unwrap(); + TableTransaction::new_with_uniqueness_fn(base(table.clone()), uniqueness_violation); let prevs = table_txn.delete_by_keys( [42i64.to_le_bytes().to_vec(), 55i64.to_le_bytes().to_vec()], 0, @@ -4045,7 +4470,9 @@ mod tests { .insert_user_database(db_name, db_owner, db_privileges.clone(), &HashSet::new()) .unwrap(); let commit_ts = txn.upper(); - txn.commit_internal(commit_ts).await.unwrap(); + txn.commit_internal(&mut *savepoint_state, commit_ts) + .await + .unwrap(); let updates = savepoint_state.sync_to_current_updates().await.unwrap(); let update = updates.into_element(); diff --git a/src/catalog/src/memory/objects.rs b/src/catalog/src/memory/objects.rs index 3ea8258b31f6b..ba9914e458c17 100644 --- a/src/catalog/src/memory/objects.rs +++ b/src/catalog/src/memory/objects.rs @@ -89,8 +89,8 @@ pub struct Database { pub id: DatabaseId, pub oid: u32, #[serde(serialize_with = "mz_ore::serde::map_key_to_string")] - pub schemas_by_id: BTreeMap, - pub schemas_by_name: BTreeMap, + pub schemas_by_id: imbl::OrdMap, + pub schemas_by_name: imbl::OrdMap, pub owner_id: RoleId, pub privileges: PrivilegeMap, } @@ -120,8 +120,8 @@ impl From for Database { Database { id, oid, - schemas_by_id: BTreeMap::new(), - schemas_by_name: BTreeMap::new(), + schemas_by_id: imbl::OrdMap::new(), + schemas_by_name: imbl::OrdMap::new(), name, owner_id, privileges: PrivilegeMap::from_mz_acl_items(privileges), @@ -153,9 +153,9 @@ pub struct Schema { pub name: QualifiedSchemaName, pub id: SchemaSpecifier, pub oid: u32, - pub items: BTreeMap, - pub functions: BTreeMap, - pub types: BTreeMap, + pub items: imbl::OrdMap, + pub functions: imbl::OrdMap, + pub types: imbl::OrdMap, pub owner_id: RoleId, pub privileges: PrivilegeMap, } @@ -191,9 +191,9 @@ impl From for Schema { }, id: id.into(), oid, - items: BTreeMap::new(), - functions: BTreeMap::new(), - types: BTreeMap::new(), + items: imbl::OrdMap::new(), + functions: imbl::OrdMap::new(), + types: imbl::OrdMap::new(), owner_id, privileges: PrivilegeMap::from_mz_acl_items(privileges), } @@ -344,14 +344,20 @@ pub struct Cluster { pub name: String, pub id: ClusterId, pub config: ClusterConfig, + // `log_indexes` stays as `BTreeMap` because it has a tight API + // contract with the compute controller (`arranged_logs: + // BTreeMap`) and is bounded in size by the + // number of log variants (~10), so it doesn't benefit from the + // persistent-clone optimization that applies to the rest of these + // collections. #[serde(skip)] pub log_indexes: BTreeMap, /// Objects bound to this cluster. Does not include introspection source /// indexes. - pub bound_objects: BTreeSet, - pub replica_id_by_name_: BTreeMap, + pub bound_objects: imbl::OrdSet, + pub replica_id_by_name_: imbl::OrdMap, #[serde(serialize_with = "mz_ore::serde::map_key_to_string")] - pub replicas_by_id_: BTreeMap, + pub replicas_by_id_: imbl::OrdMap, pub owner_id: RoleId, pub privileges: PrivilegeMap, } @@ -480,10 +486,10 @@ impl From for Cluster { Cluster { name: name.clone(), id, - bound_objects: BTreeSet::new(), + bound_objects: imbl::OrdSet::new(), log_indexes: BTreeMap::new(), - replica_id_by_name_: BTreeMap::new(), - replicas_by_id_: BTreeMap::new(), + replica_id_by_name_: imbl::OrdMap::new(), + replicas_by_id_: imbl::OrdMap::new(), owner_id, privileges: PrivilegeMap::from_mz_acl_items(privileges), config: config.into(), @@ -540,7 +546,7 @@ pub struct ClusterReplicaProcessStatus { #[derive(Debug, Serialize, Clone, PartialEq)] pub struct SourceReferences { pub updated_at: u64, - pub references: Vec, + pub references: imbl::Vector, } #[derive(Debug, Serialize, Clone, PartialEq)] @@ -642,6 +648,14 @@ impl From for mz_sql::plan::SourceReference { #[derive(Clone, Debug, Serialize)] pub struct CatalogEntry { pub item: CatalogItem, + // NOTE: `referenced_by`/`used_by` stay as `Vec` rather than + // `imbl::Vector` because the `mz_sql::catalog::CatalogItem` trait + // returns `&[CatalogItemId]` for them, which `imbl::Vector` cannot + // satisfy (no `Deref`). In practice these vectors are + // small per entry, and the dominant per-entry clone cost during + // `entry_by_id` leaf path-copy is `item: CatalogItem` (which + // carries optimized/physical plans for MV/Index/CT items), not + // these vectors of ids. #[serde(skip)] pub referenced_by: Vec, // TODO(database-issues#7922)––this should have an invariant tied to it that all @@ -3352,7 +3366,7 @@ impl mz_sql::catalog::CatalogDatabase for Database { !self.schemas_by_name.is_empty() } - fn schema_ids(&self) -> &BTreeMap { + fn schema_ids(&self) -> &imbl::OrdMap { &self.schemas_by_name } @@ -3419,7 +3433,7 @@ impl mz_sql::catalog::CatalogRole for Role { self.id } - fn membership(&self) -> &BTreeMap { + fn membership(&self) -> &imbl::OrdMap { &self.membership.map } @@ -3427,7 +3441,7 @@ impl mz_sql::catalog::CatalogRole for Role { &self.attributes } - fn vars(&self) -> &BTreeMap { + fn vars(&self) -> &imbl::OrdMap { &self.vars.map } } @@ -3459,11 +3473,11 @@ impl mz_sql::catalog::CatalogCluster<'_> for Cluster { self.id } - fn bound_objects(&self) -> &BTreeSet { + fn bound_objects(&self) -> &imbl::OrdSet { &self.bound_objects } - fn replica_ids(&self) -> &BTreeMap { + fn replica_ids(&self) -> &imbl::OrdMap { &self.replica_id_by_name_ } diff --git a/src/catalog/tests/debug.rs b/src/catalog/tests/debug.rs index d773b9125cdcc..14a63cf5bb693 100644 --- a/src/catalog/tests/debug.rs +++ b/src/catalog/tests/debug.rs @@ -462,7 +462,7 @@ async fn test_debug_delete_fencing(state_builder: TestCatalogStateBuilder) { let mut txn = state.transaction().await.unwrap(); txn.set_config("joe".to_string(), Some(666)).unwrap(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); let mut debug_state = state_builder .clone() diff --git a/src/catalog/tests/open.rs b/src/catalog/tests/open.rs index f8c739e2505e5..66b07373b44f2 100644 --- a/src/catalog/tests/open.rs +++ b/src/catalog/tests/open.rs @@ -323,7 +323,7 @@ async fn test_open_savepoint(state_builder: TestCatalogStateBuilder) { // Drain txn updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); // Read back writes. let snapshot = state.snapshot().await.unwrap(); @@ -351,7 +351,7 @@ async fn test_open_savepoint(state_builder: TestCatalogStateBuilder) { // Drain txn updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); // Read back updates. let snapshot = state.snapshot().await.unwrap(); @@ -466,7 +466,7 @@ async fn test_open_read_only(state_builder: TestCatalogStateBuilder) { // Drain txn updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); let snapshot = read_only_state.snapshot().await.unwrap(); let role = snapshot.roles.get(&proto::RoleKey { @@ -601,7 +601,7 @@ async fn test_persist_unopened_deploy_generation_fencing() { txn.set_0dt_deployment_max_wait(zdt_deployment_max_wait) .unwrap(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); } let mut openable_state = state_builder.clone().unwrap_build().await; @@ -804,7 +804,7 @@ async fn test_fencing_during_write(state_builder: TestCatalogStateBuilder) { txn.set_config("cmu".to_string(), Some(1900)).unwrap(); // Open catalog, which will bump the epoch. - let mut state = state_builder + let mut new_state = state_builder .clone() .with_deploy_generation(deploy_generation) .unwrap_build() @@ -814,11 +814,11 @@ async fn test_fencing_during_write(state_builder: TestCatalogStateBuilder) { .unwrap() .0; // Drain updates. - let _ = state.sync_to_current_updates().await; + let _ = new_state.sync_to_current_updates().await; // Committing results in an epoch fence error. let commit_ts = txn.upper(); - let err = txn.commit(commit_ts).await.unwrap_err(); + let err = txn.commit(&mut *state, commit_ts).await.unwrap_err(); assert!( matches!( err, @@ -827,11 +827,12 @@ async fn test_fencing_during_write(state_builder: TestCatalogStateBuilder) { "unexpected err: {err:?}" ); + let mut state = new_state; let mut txn = state.transaction().await.unwrap(); txn.set_config("wes".to_string(), Some(1831)).unwrap(); // Open catalog, which will bump the epoch AND deploy generation. - let _state = state_builder + let _new_state = state_builder .clone() .with_deploy_generation(deploy_generation + 1) .unwrap_build() @@ -843,7 +844,7 @@ async fn test_fencing_during_write(state_builder: TestCatalogStateBuilder) { // Committing results in a deploy generation fence error. let commit_ts = txn.upper(); - let err = txn.commit(commit_ts).await.unwrap_err(); + let err = txn.commit(&mut *state, commit_ts).await.unwrap_err(); assert!( matches!( err, diff --git a/src/catalog/tests/read-write.rs b/src/catalog/tests/read-write.rs index 79a1a70371ec1..323cae9ffb6da 100644 --- a/src/catalog/tests/read-write.rs +++ b/src/catalog/tests/read-write.rs @@ -193,7 +193,7 @@ async fn test_audit_logs(state_builder: TestCatalogStateBuilder) { // Drain txn updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); let persisted_audit_logs = state.get_audit_logs().await.unwrap(); for audit_log in &audit_logs { @@ -215,7 +215,7 @@ async fn test_items(state_builder: TestCatalogStateBuilder) { let items = [ Item { id: CatalogItemId::User(100), - oid: 20_000, + oid: 30_000, global_id: GlobalId::User(100), schema_id: SchemaId::User(1), name: "foo".to_string(), @@ -226,7 +226,7 @@ async fn test_items(state_builder: TestCatalogStateBuilder) { }, Item { id: CatalogItemId::User(200), - oid: 20_001, + oid: 30_001, global_id: GlobalId::User(200), schema_id: SchemaId::User(1), name: "bar".to_string(), @@ -267,7 +267,7 @@ async fn test_items(state_builder: TestCatalogStateBuilder) { // Drain txn updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); let snapshot_items: Vec<_> = state .snapshot() @@ -321,7 +321,7 @@ async fn test_schemas(state_builder: TestCatalogStateBuilder) { // Drain txn updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); // Test removing schemas where one doesn't exist. let mut txn = state.transaction().await.unwrap(); @@ -405,7 +405,7 @@ async fn test_non_writer_commits(state_builder: TestCatalogStateBuilder) { // Drain updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *writer_state, commit_ts).await.unwrap(); let roles = writer_state.snapshot().await.unwrap().roles; let role = roles @@ -431,7 +431,7 @@ async fn test_non_writer_commits(state_builder: TestCatalogStateBuilder) { // Drain updates. let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *savepoint_state, commit_ts).await.unwrap(); let snapshot = savepoint_state.snapshot().await.unwrap(); @@ -456,7 +456,7 @@ async fn test_non_writer_commits(state_builder: TestCatalogStateBuilder) { { let txn = reader_state.transaction().await.unwrap(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *reader_state, commit_ts).await.unwrap(); } } @@ -509,7 +509,7 @@ async fn test_persist_ddl_detection_with_batch_allocated_ids() { let id = first_id + i; txn.insert_item( CatalogItemId::User(id), - 20_000 + u32::try_from(i).unwrap(), + 30_000 + u32::try_from(i).unwrap(), GlobalId::User(id), SchemaId::User(1), &format!("item_{i}"), @@ -522,7 +522,7 @@ async fn test_persist_ddl_detection_with_batch_allocated_ids() { } let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *state, commit_ts).await.unwrap(); // Now verify the two approaches to computing the next ID baseline. let txn = state.transaction().await.unwrap(); @@ -609,7 +609,7 @@ async fn test_persist_sync_consolidation_not_quadratic() { .unwrap(); let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *writer, commit_ts).await.unwrap(); } // Record the consolidation counter before the reader syncs. @@ -678,7 +678,7 @@ async fn test_persist_sync_snapshot_stays_bounded_under_churn() { .unwrap(); let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *writer, commit_ts).await.unwrap(); // Open reader, sync to current state. let mut reader = state_builder @@ -705,7 +705,7 @@ async fn test_persist_sync_snapshot_stays_bounded_under_churn() { txn.update_database(db.id, db.clone()).unwrap(); let _ = txn.get_and_commit_op_updates(); let commit_ts = txn.upper(); - txn.commit(commit_ts).await.unwrap(); + txn.commit(&mut *writer, commit_ts).await.unwrap(); } // Reader syncs through all 200 renames. diff --git a/src/persist-client/Cargo.toml b/src/persist-client/Cargo.toml index 0fad73a172d71..349331c2f9da4 100644 --- a/src/persist-client/Cargo.toml +++ b/src/persist-client/Cargo.toml @@ -27,6 +27,12 @@ bench = false name = "benches" harness = false +# Persist-only CAS slope microbenchmark. Used by the envd-ddl-scalability +# investigation under test/envd-ddl-scalability/. +[[example]] +name = "persist_cas_bench" +path = "examples/persist_cas_bench.rs" + [dependencies] anyhow.workspace = true arrayvec.workspace = true diff --git a/src/persist-client/examples/persist_cas_bench.rs b/src/persist-client/examples/persist_cas_bench.rs new file mode 100644 index 0000000000000..39718ab144645 --- /dev/null +++ b/src/persist-client/examples/persist_cas_bench.rs @@ -0,0 +1,346 @@ +// 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. + +//! Persist-only CAS slope microbenchmark. +//! +//! Opens a single persist shard, fills it to a target SeqNo with a +//! catalog-like write pattern (one small `compare_and_append` per "DDL"), +//! then takes a window of timed `compare_and_append` measurements at that +//! state size. Repeats for each rung of a size-ladder. +//! +//! Runs against any [`Consensus`] / [`Blob`] backend pair, so we can compare +//! the slope under real CockroachDB vs. bogo-consensus vs. in-memory and +//! decide whether the per-CAS slope lives above or below the [`Consensus`] +//! trait boundary. + +use std::path::PathBuf; +use std::str::FromStr; +use std::sync::Arc; +use std::time::{Duration, Instant}; + +use anyhow::{Context, Result, anyhow, bail}; +use clap::Parser; +use mz_build_info::DUMMY_BUILD_INFO; +use mz_dyncfg::ConfigSet; +use mz_ore::metrics::MetricsRegistry; +use mz_ore::now::SYSTEM_TIME; +use mz_ore::url::SensitiveUrl; +use mz_persist::bogo::{BogoConsensus, BogoConsensusConfig}; +use mz_persist::file::{FileBlob, FileBlobConfig}; +use mz_persist::location::{Blob, Consensus}; +use mz_persist::mem::{MemBlob, MemBlobConfig, MemConsensus}; +use mz_persist::postgres::{ + PostgresConsensus, PostgresConsensusConfig, USE_POSTGRES_TUNED_QUERIES, +}; +use mz_persist_client::async_runtime::IsolatedRuntime; +use mz_persist_client::cache::StateCache; +use mz_persist_client::cfg::PersistConfig; +use mz_persist_client::metrics::Metrics; +use mz_persist_client::rpc::PubSubClientConnection; +use mz_persist_client::{Diagnostics, PersistClient, ShardId}; +use mz_persist_types::codec_impls::{StringSchema, UnitSchema}; +use mz_postgres_client::PostgresClientKnobs; +use mz_postgres_client::metrics::PostgresClientMetrics; +use timely::progress::Antichain; + +/// One CAS measurement row in the output CSV. +#[derive(Debug)] +struct Sample { + backend: String, + blob: String, + size: u64, + rep: usize, + ms: f64, +} + +#[derive(Parser, Debug)] +#[command(about = "Persist-only CAS slope microbenchmark", long_about = None)] +struct Args { + /// `bogo://host:port`, `postgres://...`, or `mem`. + #[arg(long)] + consensus: String, + + /// `file:///path` or `mem`. + #[arg(long, default_value = "mem")] + blob: String, + + /// Comma-separated pre-fill SeqNo ladder. Each rung gets a fresh shard. + #[arg( + long, + value_delimiter = ',', + default_value = "0,500,1000,2500,5000,10000" + )] + sizes: Vec, + + /// Timed CAS measurements taken at each rung after pre-fill. + #[arg(long, default_value_t = 200)] + measurements: usize, + + /// Override `persist_rollup_threshold`; 0 keeps the persist default (128). + /// Set very high (e.g. 1_000_000) to suppress rollups so state genuinely + /// grows; leave at default to model production rollup cadence. + #[arg(long, default_value_t = 0)] + rollup_threshold: usize, + + /// Output CSV path. Appends if it exists. + #[arg(long)] + out: PathBuf, + + /// Free-form tag recorded in each CSV row (e.g. `crdb-default-rollup`). + #[arg(long, default_value = "")] + tag: String, +} + +#[derive(Debug)] +struct CasKnobs; + +impl PostgresClientKnobs for CasKnobs { + fn connection_pool_max_size(&self) -> usize { + 2 + } + fn connection_pool_max_wait(&self) -> Option { + Some(Duration::from_secs(5)) + } + fn connection_pool_ttl(&self) -> Duration { + Duration::MAX + } + fn connection_pool_ttl_stagger(&self) -> Duration { + Duration::MAX + } + fn connect_timeout(&self) -> Duration { + Duration::from_secs(5) + } + fn tcp_user_timeout(&self) -> Duration { + Duration::ZERO + } + fn keepalives_idle(&self) -> Duration { + Duration::from_secs(10) + } + fn keepalives_interval(&self) -> Duration { + Duration::from_secs(5) + } + fn keepalives_retries(&self) -> u32 { + 5 + } +} + +async fn open_consensus(url: &str) -> Result> { + if url == "mem" { + return Ok(Arc::new(MemConsensus::default())); + } + if url.starts_with("bogo://") { + let u = SensitiveUrl::from_str(url).context("parsing bogo url")?; + let cfg = BogoConsensusConfig::new(u).map_err(|e| anyhow!("bogo config: {e}"))?; + let c = BogoConsensus::open(cfg) + .await + .map_err(|e| anyhow!("bogo open: {e}"))?; + return Ok(Arc::new(c)); + } + if url.starts_with("postgres://") || url.starts_with("postgresql://") { + let u = SensitiveUrl::from_str(url).context("parsing postgres url")?; + let dyncfg = Arc::new(ConfigSet::default().add(&USE_POSTGRES_TUNED_QUERIES)); + let metrics = PostgresClientMetrics::new(&MetricsRegistry::new(), "mz_persist"); + let cfg = PostgresConsensusConfig::new(&u, Box::new(CasKnobs), metrics, dyncfg) + .map_err(|e| anyhow!("pg config: {e}"))?; + let c = PostgresConsensus::open(cfg) + .await + .map_err(|e| anyhow!("pg open: {e}"))?; + return Ok(Arc::new(c)); + } + bail!("unsupported consensus URL: {}", url); +} + +async fn open_blob(url: &str) -> Result> { + if url == "mem" { + return Ok(Arc::new(MemBlob::open(MemBlobConfig::default()))); + } + if let Some(path) = url.strip_prefix("file://") { + std::fs::create_dir_all(path).context("creating blob dir")?; + let cfg = FileBlobConfig::from(path); + let b = FileBlob::open(cfg) + .await + .map_err(|e| anyhow!("file blob open: {e}"))?; + return Ok(Arc::new(b)); + } + bail!("unsupported blob URL: {}", url); +} + +fn make_client( + consensus: Arc, + blob: Arc, + rollup_threshold: usize, +) -> Result { + let cfg = PersistConfig::new_default_configs(&DUMMY_BUILD_INFO, SYSTEM_TIME.clone()); + if rollup_threshold > 0 { + cfg.set_rollup_threshold(rollup_threshold); + } + let metrics = Arc::new(Metrics::new(&cfg, &MetricsRegistry::new())); + let isolated_runtime = Arc::new(IsolatedRuntime::new_for_tests()); + let pubsub_sender = PubSubClientConnection::noop().sender; + let shared_states = Arc::new(StateCache::new( + &cfg, + Arc::clone(&metrics), + Arc::clone(&pubsub_sender), + )); + PersistClient::new( + cfg, + blob, + consensus, + metrics, + isolated_runtime, + shared_states, + pubsub_sender, + ) + .map_err(|e| anyhow!("persist client: {e}")) +} + +/// One iteration of the catalog-like CAS pattern: append a tiny batch that +/// advances `upper` from `ts` to `ts+1`. Returns the new upper. +async fn cas_once( + write: &mut mz_persist_client::write::WriteHandle, + ts: u64, + row: &str, +) -> Result { + let updates = vec![((row.to_owned(), ()), ts, 1i64)]; + let expected = Antichain::from_elem(ts); + let new = Antichain::from_elem(ts + 1); + let res = write + .compare_and_append(updates, expected, new) + .await + .map_err(|e| anyhow!("invalid usage: {e:?}"))?; + res.map_err(|m| anyhow!("upper mismatch: {m:?}"))?; + Ok(ts + 1) +} + +async fn run_rung( + client: &PersistClient, + backend_tag: &str, + blob_tag: &str, + size: u64, + measurements: usize, +) -> Result> { + let shard_id = ShardId::new(); + let diag = Diagnostics { + shard_name: "persist_cas_bench".to_string(), + handle_purpose: "cas-microbench".to_string(), + }; + let (mut write, _read) = client + .open::( + shard_id, + Arc::new(StringSchema), + Arc::new(UnitSchema), + diag, + true, + ) + .await + .map_err(|e| anyhow!("open: {e:?}"))?; + + // Pre-fill: advance the shard's history to `size` SeqNos by issuing + // `size` non-empty compare_and_appends. + let mut ts: u64 = 0; + let row = "catalog-like-small-row".to_string(); + let mut last_print = Instant::now(); + while ts < size { + ts = cas_once(&mut write, ts, &row).await?; + if last_print.elapsed() > Duration::from_secs(5) { + eprintln!(" pre-fill {}/{} ts={}", ts, size, ts); + last_print = Instant::now(); + } + } + + // Measurement window. + let mut samples = Vec::with_capacity(measurements); + for rep in 0..measurements { + let start = Instant::now(); + ts = cas_once(&mut write, ts, &row).await?; + let ms = start.elapsed().as_secs_f64() * 1000.0; + samples.push(Sample { + backend: backend_tag.to_string(), + blob: blob_tag.to_string(), + size, + rep, + ms, + }); + } + + Ok(samples) +} + +fn write_csv(path: &PathBuf, tag: &str, samples: &[Sample]) -> Result<()> { + let already = path.exists(); + let mut f = std::fs::OpenOptions::new() + .create(true) + .append(true) + .open(path) + .with_context(|| format!("opening {}", path.display()))?; + use std::io::Write; + if !already { + writeln!(f, "tag,backend,blob,size,rep,ms")?; + } + for s in samples { + writeln!( + f, + "{},{},{},{},{},{:.4}", + tag, s.backend, s.blob, s.size, s.rep, s.ms + )?; + } + Ok(()) +} + +fn percentile(sorted_ms: &[f64], p: f64) -> f64 { + if sorted_ms.is_empty() { + return f64::NAN; + } + let idx = ((sorted_ms.len() as f64 - 1.0) * p).round() as usize; + sorted_ms[idx] +} + +fn summarize(samples: &[Sample]) { + let mut ms: Vec = samples.iter().map(|s| s.ms).collect(); + ms.sort_by(|a, b| a.partial_cmp(b).unwrap()); + let sum: f64 = ms.iter().sum(); + let mean = sum / (ms.len() as f64); + eprintln!( + " n={} mean={:.2}ms p50={:.2}ms p95={:.2}ms p99={:.2}ms max={:.2}ms", + ms.len(), + mean, + percentile(&ms, 0.50), + percentile(&ms, 0.95), + percentile(&ms, 0.99), + ms.last().copied().unwrap_or(f64::NAN), + ); +} + +#[tokio::main] +async fn main() -> Result<()> { + mz_ore::test::init_logging_default("warn"); + + let args = Args::parse(); + let backend_tag = args.consensus.clone(); + let blob_tag = args.blob.clone(); + eprintln!( + "persist_cas_bench: consensus={} blob={} sizes={:?} measurements={} rollup_threshold={} tag={}", + backend_tag, blob_tag, args.sizes, args.measurements, args.rollup_threshold, args.tag + ); + + for size in &args.sizes { + eprintln!("== size={} ==", size); + // Build a fresh client per rung so the state cache and metrics + // don't carry over between rungs. + let consensus = open_consensus(&args.consensus).await?; + let blob = open_blob(&args.blob).await?; + let client = make_client(consensus, blob, args.rollup_threshold)?; + let samples = run_rung(&client, &backend_tag, &blob_tag, *size, args.measurements).await?; + summarize(&samples); + write_csv(&args.out, &args.tag, &samples)?; + } + + eprintln!("done; wrote {}", args.out.display()); + Ok(()) +} diff --git a/src/persist-client/src/cache.rs b/src/persist-client/src/cache.rs index dfd4a98e1416d..54e15d3d4f9b7 100644 --- a/src/persist-client/src/cache.rs +++ b/src/persist-client/src/cache.rs @@ -203,12 +203,26 @@ impl PersistClientCache { Entry::Vacant(x) => { // Intentionally hold the lock, so we don't double connect under // concurrency. - let consensus = ConsensusConfig::try_from( + let mut consensus = ConsensusConfig::try_from( x.key(), Box::new(self.cfg.clone()), self.metrics.postgres_consensus.clone(), Arc::clone(&self.cfg().configs), )?; + // For bogo-consensus, attach a wire-timer hook that lets us + // split `external_op_latency_by_kind` (around `run_op` in + // `MetricsConsensus`) from the inner gRPC client wall time. + // No-op for other backends. + if let ConsensusConfig::Bogo(ref mut bogo_cfg) = consensus { + let metrics = Arc::clone(&self.metrics); + bogo_cfg.wire_timer = Some(Arc::new(move |rpc, key, elapsed| { + let kind = metrics.shard_kind_for_key(key); + metrics + .consensus_wire_seconds_by_kind + .with_label_values(&[rpc, kind]) + .observe(elapsed.as_secs_f64()); + })); + } let consensus = retry_external(&self.metrics.retries.external.consensus_open, || { consensus.clone().open() @@ -382,7 +396,12 @@ where fn push_diff(&self, diff: VersionedData) { self.write_lock(&self.metrics.locks.applier_write, |state| { let seqno_before = state.seqno; - state.apply_encoded_diffs(&self.cfg, &self.metrics, std::iter::once(&diff)); + state.apply_encoded_diffs( + &self.cfg, + &self.metrics, + "pubsub_push", + std::iter::once(&diff), + ); let seqno_after = state.seqno; assert!(seqno_after >= seqno_before); diff --git a/src/persist-client/src/internal/apply.rs b/src/persist-client/src/internal/apply.rs index a48982ff77eb9..f1d6fcc546f20 100644 --- a/src/persist-client/src/internal/apply.rs +++ b/src/persist-client/src/internal/apply.rs @@ -101,6 +101,7 @@ where diagnostics: Diagnostics, ) -> Result> { let shard_metrics = metrics.shards.shard(&shard_id, &diagnostics.shard_name); + metrics.register_shard_kind(shard_id, &diagnostics.shard_name); let state = shared_states .get::( shard_id, @@ -665,7 +666,12 @@ where let new_seqno = self .state .write_lock(&self.metrics.locks.applier_write, |state| { - state.apply_encoded_diffs(&self.cfg, &self.metrics, &diffs_to_current); + state.apply_encoded_diffs( + &self.cfg, + &self.metrics, + "cas_update", + &diffs_to_current, + ); state.seqno }); diff --git a/src/persist-client/src/internal/encoding.rs b/src/persist-client/src/internal/encoding.rs index b4599bd02e6ce..4261580c7abe4 100644 --- a/src/persist-client/src/internal/encoding.rs +++ b/src/persist-client/src/internal/encoding.rs @@ -765,6 +765,7 @@ impl UntypedState { &mut self, cfg: &PersistConfig, metrics: &Metrics, + source: &'static str, diffs: I, ) { // The apply_encoded_diffs might panic if T is not correct. Making this @@ -773,7 +774,7 @@ impl UntypedState { if T::codec_name() != self.ts_codec { return; } - self.state.apply_encoded_diffs(cfg, metrics, diffs); + self.state.apply_encoded_diffs(cfg, metrics, source, diffs); } pub fn check_codecs( @@ -2250,7 +2251,12 @@ mod tests { seqno: SeqNo(5), data: diff_proto.encode_to_vec().into(), }; - state.apply_encoded_diffs(cache.cfg(), &cache.metrics, std::iter::once(&encoded_diff)); + state.apply_encoded_diffs( + cache.cfg(), + &cache.metrics, + "test", + std::iter::once(&encoded_diff), + ); assert_eq!( state .state diff --git a/src/persist-client/src/internal/metrics.rs b/src/persist-client/src/internal/metrics.rs index 8daba8b775854..0c1f510f86385 100644 --- a/src/persist-client/src/internal/metrics.rs +++ b/src/persist-client/src/internal/metrics.rs @@ -11,7 +11,7 @@ use async_stream::stream; use mz_persist_types::stats::PartStatsMetrics; -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::{Arc, Mutex, Weak}; use std::time::{Duration, Instant}; use tokio::sync::{OnceCell, OwnedSemaphorePermit, Semaphore}; @@ -113,6 +113,39 @@ pub struct Metrics { /// Metrics for Postgres-backed consensus implementation pub postgres_consensus: PostgresClientMetrics, + /// Per-op latency histogram broken down by shard_kind, so we can + /// disentangle catalog / txns / user-shard contributions to total CAS time. + pub external_op_latency_by_kind: HistogramVec, + /// Sub-stage latency histogram for `State::apply_diff` and friends, broken + /// down by `[stage, shard_kind]`. `stage` is one of `total`, `flatten`, + /// `unflatten`, `decode`. Use this to localize where per-CAS state-apply + /// time goes for a given shard kind. The histogram count for `stage="total"` + /// also doubles as a "number of `apply_diff` invocations" counter per + /// shard_kind, which is what reveals the per-DDL state-apply work growth. + pub state_apply_latency_by_kind: HistogramVec, + /// Per-`apply_diff` invocation counter, broken down by `[source, shard_kind]`. + /// `source` is one of `cas_update` (apply.rs `fetch_and_update_state` fast + /// path), `slow_refetch` (`state_versions.rs::fetch_current_state` full + /// replay from rollup), `pubsub_push` (`cache.rs::push_diff` from PubSub + /// broadcast), or `state_iter` (historical walk via + /// `state_versions.rs::StateVersionsIter::next`). Used to attribute the + /// per-DDL `apply_diff` count growth on the catalog/txns shards to the + /// specific code path that is calling state apply. + pub state_apply_calls_by_source_kind: IntCounterVec, + /// Per-Consensus-op inner wire latency, broken down by shard_kind. Recorded + /// inside `BogoConsensus` around the bogo gRPC client call only, so we can + /// split `external_op_latency_by_kind` (post-spawn through `run_op`) from + /// the time spent at the actual wire. `external - wire` ≈ overhead inside + /// `MetricsConsensus::run_op` + the bogo adapter; `wire` itself ≈ tonic + /// client send + server processing + return. Only populated when the + /// active consensus backend is `bogo://`. + pub consensus_wire_seconds_by_kind: HistogramVec, + /// Registry mapping a persist key prefix (the ShardId in stringified form) + /// to a coarse `shard_kind` label. Populated when shards are opened by + /// `Applier::new` via [Metrics::register_shard_kind] and read on the hot + /// path of [MetricsConsensus] / [MetricsBlob] to attribute external ops. + shard_kinds: Arc>>, + #[allow(dead_code)] pub(crate) registry: MetricsRegistry, } @@ -141,6 +174,31 @@ impl Metrics { ); let s3_blob = S3BlobMetrics::new(registry); let columnar = ColumnarMetrics::new(registry); + let external_op_latency_by_kind = registry.register(metric!( + name: "mz_persist_external_op_latency_by_shard_kind", + help: "latency observed by individual performance-critical external ops, broken down by shard_kind", + var_labels: ["op", "shard_kind"], + buckets: histogram_seconds_buckets(0.000_500, 32.0), + )); + let state_apply_latency_by_kind = registry.register(metric!( + name: "mz_persist_state_apply_latency_by_shard_kind", + help: "latency of State::apply_diff sub-stages (total/flatten/unflatten/decode), broken down by shard_kind", + var_labels: ["stage", "shard_kind"], + // Apply work is mostly sub-millisecond per-call; keep the low end fine. + buckets: histogram_seconds_buckets(0.000_050, 32.0), + )); + let state_apply_calls_by_source_kind = registry.register(metric!( + name: "mz_persist_state_apply_calls_by_source_shard_kind", + help: "count of State::apply_diff invocations, broken down by call-site source and shard_kind", + var_labels: ["source", "shard_kind"], + )); + let consensus_wire_seconds_by_kind = registry.register(metric!( + name: "mz_persist_consensus_wire_seconds_by_shard_kind", + help: "wall time of the inner bogo-consensus gRPC call (client-side), broken down by op and shard_kind", + var_labels: ["op", "shard_kind"], + // Same buckets as external_op_latency_by_kind so they line up. + buckets: histogram_seconds_buckets(0.000_500, 32.0), + )); Metrics { blob: vecs.blob_metrics(), consensus: vecs.consensus_metrics(), @@ -169,12 +227,68 @@ impl Metrics { sink: SinkMetrics::new(registry), s3_blob, postgres_consensus: PostgresClientMetrics::new(registry, "mz_persist"), + external_op_latency_by_kind, + state_apply_latency_by_kind, + state_apply_calls_by_source_kind, + consensus_wire_seconds_by_kind, + shard_kinds: Arc::new(Mutex::new(HashMap::new())), _vecs: vecs, _uptime: uptime, registry: registry.clone(), } } + /// Classify a `shard_name` (as provided in `Diagnostics::shard_name` at + /// shard-open time) into a coarse, fixed taxonomy useful for breaking down + /// hot-path metrics. The taxonomy intentionally has a small, closed set of + /// labels so it does not blow up Prometheus cardinality. + pub fn classify_shard_kind(shard_name: &str) -> &'static str { + // Known process-singleton shards by exact name. Names below come from + // the canonical open sites and should match string literals there. + match shard_name { + "catalog" => "catalog", + "txns" => "txns", + "builtin_migration" => "builtin_migration", + "expression_cache" => "expression_cache", + "storage-usage" | "storage_usage" => "storage_usage", + "unknown" => "unknown", + _ => "user_data", + } + } + + /// Register a `shard_id` -> `shard_kind` mapping derived from + /// `shard_name`. Called once per shard open from `Applier::new`; the map + /// is read on the hot path of [MetricsConsensus] / [MetricsBlob] to + /// attribute external-op latency. + pub fn register_shard_kind(&self, shard_id: ShardId, shard_name: &str) { + let kind = Self::classify_shard_kind(shard_name); + let mut map = self.shard_kinds.lock().expect("mutex poisoned"); + map.entry(shard_id.to_string()).or_insert(kind); + } + + /// Look up the `shard_kind` for a persist external-op `key`. The key is a + /// ShardId for consensus ops, or a `/<...>` blob path for blob + /// ops; we extract just the shard-id prefix and consult [Self::shard_kinds]. + /// Returns `"unknown"` if the shard has not been registered yet (e.g. for + /// pre-registration init writes). + pub(crate) fn shard_kind_for_key(&self, key: &str) -> &'static str { + let prefix = match key.split_once('/') { + Some((p, _)) => p, + None => key, + }; + let map = self.shard_kinds.lock().expect("mutex poisoned"); + map.get(prefix).copied().unwrap_or("unknown") + } + + /// Look up the `shard_kind` for a [ShardId]. Returns `"unknown"` if the + /// shard has not been registered yet via [Self::register_shard_kind]. + /// Used by in-persist hot paths (state apply) to attribute work to the + /// shard kind without going through the consensus-key prefix logic. + pub(crate) fn shard_kind_for(&self, shard_id: &ShardId) -> &'static str { + let map = self.shard_kinds.lock().expect("mutex poisoned"); + map.get(&shard_id.to_string()).copied().unwrap_or("unknown") + } + /// Returns the current lifetime write amplification reflected in these /// metrics. /// @@ -2782,12 +2896,18 @@ impl MetricsBlob { impl Blob for MetricsBlob { #[instrument(name = "blob::get", fields(shard=blob_key_shard_id(key)))] async fn get(&self, key: &str) -> Result, ExternalError> { + let kind = self.metrics.shard_kind_for_key(key); + let start = Instant::now(); let res = self .metrics .blob .get .run_op(|| self.blob.get(key), Self::on_err) .await; + self.metrics + .external_op_latency_by_kind + .with_label_values(&["blob_get", kind]) + .observe(start.elapsed().as_secs_f64()); if let Ok(Some(value)) = res.as_ref() { self.metrics .blob @@ -2837,12 +2957,18 @@ impl Blob for MetricsBlob { #[instrument(name = "blob::set", fields(shard=blob_key_shard_id(key),size_bytes=value.len()))] async fn set(&self, key: &str, value: Bytes) -> Result<(), ExternalError> { let bytes = value.len(); + let kind = self.metrics.shard_kind_for_key(key); + let start = Instant::now(); let res = self .metrics .blob .set .run_op(|| self.blob.set(key, value), Self::on_err) .await; + self.metrics + .external_op_latency_by_kind + .with_label_values(&["blob_set", kind]) + .observe(start.elapsed().as_secs_f64()); if res.is_ok() { self.metrics.blob.set.bytes.inc_by(u64::cast_from(bytes)); self.metrics.blob.blob_sizes.observe(f64::cast_lossy(bytes)); @@ -2920,12 +3046,18 @@ impl Consensus for MetricsConsensus { #[instrument(name = "consensus::head", fields(shard=key))] async fn head(&self, key: &str) -> Result, ExternalError> { + let kind = self.metrics.shard_kind_for_key(key); + let start = Instant::now(); let res = self .metrics .consensus .head .run_op(|| self.consensus.head(key), Self::on_err) .await; + self.metrics + .external_op_latency_by_kind + .with_label_values(&["consensus_head", kind]) + .observe(start.elapsed().as_secs_f64()); if let Ok(Some(data)) = res.as_ref() { self.metrics .consensus @@ -2943,12 +3075,18 @@ impl Consensus for MetricsConsensus { new: VersionedData, ) -> Result { let bytes = new.data.len(); + let kind = self.metrics.shard_kind_for_key(key); + let start = Instant::now(); let res = self .metrics .consensus .compare_and_set .run_op(|| self.consensus.compare_and_set(key, new), Self::on_err) .await; + self.metrics + .external_op_latency_by_kind + .with_label_values(&["consensus_cas", kind]) + .observe(start.elapsed().as_secs_f64()); match res.as_ref() { Ok(CaSResult::Committed) => self .metrics @@ -2968,12 +3106,18 @@ impl Consensus for MetricsConsensus { from: SeqNo, limit: usize, ) -> Result, ExternalError> { + let kind = self.metrics.shard_kind_for_key(key); + let start = Instant::now(); let res = self .metrics .consensus .scan .run_op(|| self.consensus.scan(key, from, limit), Self::on_err) .await; + self.metrics + .external_op_latency_by_kind + .with_label_values(&["consensus_scan", kind]) + .observe(start.elapsed().as_secs_f64()); if let Ok(dataz) = res.as_ref() { let bytes: usize = dataz.iter().map(|x| x.data.len()).sum(); self.metrics @@ -2988,10 +3132,16 @@ impl Consensus for MetricsConsensus { #[instrument(name = "consensus::truncate", fields(shard=key))] async fn truncate(&self, key: &str, seqno: SeqNo) -> Result, ExternalError> { let metrics = &self.metrics.consensus; + let kind = self.metrics.shard_kind_for_key(key); + let start = Instant::now(); let deleted = metrics .truncate .run_op(|| self.consensus.truncate(key, seqno), Self::on_err) .await?; + self.metrics + .external_op_latency_by_kind + .with_label_values(&["consensus_truncate", kind]) + .observe(start.elapsed().as_secs_f64()); if let Some(deleted) = deleted { metrics.truncated_count.inc_by(u64::cast_from(deleted)); } diff --git a/src/persist-client/src/internal/state_diff.rs b/src/persist-client/src/internal/state_diff.rs index 1be97db11bc5e..51f4737249b93 100644 --- a/src/persist-client/src/internal/state_diff.rs +++ b/src/persist-client/src/internal/state_diff.rs @@ -11,6 +11,7 @@ use std::cmp::Ordering; use std::collections::BTreeMap; use std::fmt::Debug; use std::sync::Arc; +use std::time::Instant; use bytes::{Bytes, BytesMut}; use differential_dataflow::lattice::Lattice; @@ -320,7 +321,7 @@ impl StateDiff { use crate::internal::state::ProtoStateDiff; let mut roundtrip_state = from_state.clone(from_state.hostname.clone()); - roundtrip_state.apply_diff(metrics, diff.clone())?; + roundtrip_state.apply_diff(metrics, "roundtrip_check", diff.clone())?; if &roundtrip_state != to_state { // The weird spacing in this format string is so they all line up @@ -355,25 +356,32 @@ impl State { &mut self, cfg: &PersistConfig, metrics: &Metrics, + source: &'static str, diffs: I, ) { let mut state_seqno = self.seqno; + let shard_kind = metrics.shard_kind_for(&self.shard_id); let diffs = diffs.into_iter().filter_map(move |x| { if x.seqno != state_seqno.next() { // No-op. return None; } let data = x.data.clone(); + let decode_start = Instant::now(); let diff = metrics .codecs .state_diff // Note: `x.data` is a `Bytes`, so cloning just increments a ref count .decode(|| StateDiff::decode(&cfg.build_version, x.data.clone())); + metrics + .state_apply_latency_by_kind + .with_label_values(&["decode", shard_kind]) + .observe(decode_start.elapsed().as_secs_f64()); assert_eq!(diff.seqno_from, state_seqno); state_seqno = diff.seqno_to; Some((diff, data)) }); - self.apply_diffs(metrics, diffs); + self.apply_diffs(metrics, source, diffs); } } @@ -381,13 +389,14 @@ impl State { pub fn apply_diffs, Bytes)>>( &mut self, metrics: &Metrics, + source: &'static str, diffs: I, ) { for (diff, data) in diffs { // TODO: This could special-case batch apply for diffs where it's // more efficient (in particular, spine batches that hit the slow // path). - match self.apply_diff(metrics, diff) { + match self.apply_diff(metrics, source, diff) { Ok(()) => {} Err(err) => { // Having the full diff in the error message is critical for debugging any @@ -408,9 +417,30 @@ impl State { // Intentionally not even pub(crate) because all callers should use // [Self::apply_diffs]. pub(super) fn apply_diff( + &mut self, + metrics: &Metrics, + source: &'static str, + diff: StateDiff, + ) -> Result<(), String> { + let apply_start = Instant::now(); + let shard_kind = metrics.shard_kind_for(&self.shard_id); + metrics + .state_apply_calls_by_source_kind + .with_label_values(&[source, shard_kind]) + .inc(); + let res = self.apply_diff_inner(metrics, diff, shard_kind); + metrics + .state_apply_latency_by_kind + .with_label_values(&["total", shard_kind]) + .observe(apply_start.elapsed().as_secs_f64()); + res + } + + fn apply_diff_inner( &mut self, metrics: &Metrics, diff: StateDiff, + shard_kind: &'static str, ) -> Result<(), String> { // Deconstruct diff so we get a compile failure if new fields are added. let StateDiff { @@ -491,7 +521,12 @@ impl State { let mut flat = if trace.roundtrip_structure { metrics.state.apply_spine_flattened.inc(); + let flatten_start = Instant::now(); let mut flat = trace.flatten(); + metrics + .state_apply_latency_by_kind + .with_label_values(&["flatten", shard_kind]) + .observe(flatten_start.elapsed().as_secs_f64()); apply_diffs_single("since", diff_since, &mut flat.since)?; apply_diffs_map( "legacy_batches", @@ -529,7 +564,15 @@ impl State { }; if !structure_unchanged { - let flat = flat.get_or_insert_with(|| trace.flatten()); + let flat = flat.get_or_insert_with(|| { + let flatten_start = Instant::now(); + let f = trace.flatten(); + metrics + .state_apply_latency_by_kind + .with_label_values(&["flatten", shard_kind]) + .observe(flatten_start.elapsed().as_secs_f64()); + f + }); apply_diffs_map( "hollow_batches", diff_hollow_batches, @@ -540,7 +583,12 @@ impl State { } if let Some(flat) = flat { + let unflatten_start = Instant::now(); *trace = Trace::unflatten(flat)?; + metrics + .state_apply_latency_by_kind + .with_label_values(&["unflatten", shard_kind]) + .observe(unflatten_start.elapsed().as_secs_f64()); } // There's various sanity checks that this method could run (e.g. since, diff --git a/src/persist-client/src/internal/state_versions.rs b/src/persist-client/src/internal/state_versions.rs index 242e832882638..017a42e2c1d21 100644 --- a/src/persist-client/src/internal/state_versions.rs +++ b/src/persist-client/src/internal/state_versions.rs @@ -463,7 +463,7 @@ impl StateVersions { } }; - state.apply_encoded_diffs(&self.cfg, &self.metrics, &live_diffs); + state.apply_encoded_diffs(&self.cfg, &self.metrics, "slow_refetch", &live_diffs); return state; } } @@ -1137,7 +1137,7 @@ impl StateVersionsIter { let diff_seqno_to = diff.seqno_to; self.state - .apply_diffs(&self.metrics, std::iter::once((diff, data))); + .apply_diffs(&self.metrics, "state_iter", std::iter::once((diff, data))); assert_eq!(self.state.seqno, diff_seqno_to); #[cfg(debug_assertions)] { diff --git a/src/persist/Cargo.toml b/src/persist/Cargo.toml index fef843fe37101..fe2799621b39a 100644 --- a/src/persist/Cargo.toml +++ b/src/persist/Cargo.toml @@ -44,6 +44,7 @@ mz-foundationdb = { path = "../foundationdb", default-features = false, optional itertools.workspace = true md-5.workspace = true mz-aws-util = { path = "../aws-util", features = ["s3"] } +mz-bogo-consensus = { path = "../bogo-consensus", default-features = false } mz-dyncfg = { path = "../dyncfg" } mz-ore = { path = "../ore", default-features = false, features = ["metrics", "async", "bytes", "region", "parquet"] } mz-persist-types = { path = "../persist-types" } @@ -65,6 +66,7 @@ serde_json = { workspace = true, optional = true } timely.workspace = true tokio.workspace = true tokio-postgres.workspace = true +tonic.workspace = true tracing.workspace = true turmoil = { workspace = true, optional = true } url.workspace = true @@ -82,6 +84,7 @@ features = ["asm"] mz-ore = { path = "../ore", default-features = false, features = ["test"] } serde_json.workspace = true tempfile.workspace = true +tokio-stream = { workspace = true, features = ["net"] } [build-dependencies] mz-build-tools = { path = "../build-tools", default-features = false, features = ["protobuf-src"] } diff --git a/src/persist/src/bogo.rs b/src/persist/src/bogo.rs new file mode 100644 index 0000000000000..2da1e0e0e5bec --- /dev/null +++ b/src/persist/src/bogo.rs @@ -0,0 +1,445 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file at the root of this repository. +// +// 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. + +//! A [`Consensus`] implementation backed by the `mz-bogo-consensus` gRPC +//! server. +//! +//! Use `bogo://host:port` as `--persist-consensus-url` to route persist's +//! Consensus traffic to an in-memory bogo-consensus server. This bypasses +//! Postgres/CRDB entirely; it exists solely to take that backend out of the +//! loop during performance measurements. + +use std::sync::Arc; +use std::time::{Duration, Instant}; + +use anyhow::anyhow; +use async_stream::try_stream; +use async_trait::async_trait; +use futures_util::StreamExt; +use mz_bogo_consensus::client::{BogoConsensusClient, CaSResult as ProtoCaSResult}; +use mz_bogo_consensus::proto::VersionedData as ProtoVersionedData; +use mz_ore::url::SensitiveUrl; +use tonic::{Code, Status}; + +use crate::location::{ + CaSResult, Consensus, Determinate, ExternalError, Indeterminate, ResultStream, SCAN_ALL, SeqNo, + VersionedData, +}; + +/// Optional client-side wire-timing hook. Called once per `Consensus` op with +/// `(rpc_name, key, wall_time_at_bogo_client_call)`. The intent is to split +/// the existing `mz_persist_external_op_latency_by_shard_kind` (which spans +/// `run_op` in `MetricsConsensus`) from the time inside the bogo gRPC client +/// itself, so we can attribute slope to wire vs. wrapper overhead. +pub type BogoWireTimer = Arc; + +/// Configuration for opening a [`BogoConsensus`]. +#[derive(Clone)] +pub struct BogoConsensusConfig { + /// The `bogo://` URL of the server. + pub url: SensitiveUrl, + /// Optional hook invoked with the wall time of each inner gRPC call. + pub wire_timer: Option, +} + +impl std::fmt::Debug for BogoConsensusConfig { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("BogoConsensusConfig") + .field("url", &self.url) + .field("wire_timer", &self.wire_timer.as_ref().map(|_| "")) + .finish() + } +} + +impl BogoConsensusConfig { + /// Builds a config from a `bogo://host:port` URL. Returns an error if the + /// URL is missing the host/port or uses the wrong scheme. + pub fn new(url: SensitiveUrl) -> Result { + if url.scheme() != "bogo" { + return Err(ExternalError::from(anyhow!( + "bogo-consensus URL must use the `bogo://` scheme, got `{}://`", + url.scheme() + ))); + } + if url.host_str().is_none() || url.port().is_none() { + return Err(ExternalError::from(anyhow!( + "bogo-consensus URL must include host and port, got `{}`", + url + ))); + } + Ok(Self { + url, + wire_timer: None, + }) + } +} + +/// A [`Consensus`] backed by a remote `mz-bogo-consensus` server. +#[derive(Clone)] +pub struct BogoConsensus { + client: Arc, + wire_timer: Option, +} + +impl std::fmt::Debug for BogoConsensus { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("BogoConsensus") + .field("client", &self.client) + .field("wire_timer", &self.wire_timer.as_ref().map(|_| "")) + .finish() + } +} + +impl BogoConsensus { + /// Connects to the bogo-consensus server identified by `cfg.url`. + pub async fn open(cfg: BogoConsensusConfig) -> Result { + let host = cfg + .url + .host_str() + .ok_or_else(|| ExternalError::from(anyhow!("bogo URL missing host")))?; + let port = cfg + .url + .port() + .ok_or_else(|| ExternalError::from(anyhow!("bogo URL missing port")))?; + // tonic wants an http:// (or https://) endpoint; the `bogo://` scheme + // is just a marker we use for routing. + let endpoint = format!("http://{host}:{port}"); + let client = BogoConsensusClient::connect(endpoint) + .await + .map_err(|e| ExternalError::from(anyhow!("connecting to bogo-consensus: {e:#}")))?; + Ok(Self { + client: Arc::new(client), + wire_timer: cfg.wire_timer, + }) + } + + fn observe_wire(&self, rpc: &'static str, key: &str, start: Instant) { + if let Some(timer) = &self.wire_timer { + timer(rpc, key, start.elapsed()); + } + } +} + +/// Map a `tonic::Status` to `ExternalError`, classifying codes as +/// determinate vs indeterminate. +/// +/// Codes returned by the server's deterministic logic (e.g. invalid seqno, +/// truncate-too-high) are mapped to `Determinate`. Network/transport-class +/// codes are `Indeterminate` — the operation may or may not have committed. +fn status_to_external(rpc: &'static str, status: Status) -> ExternalError { + let msg = anyhow!( + "bogo-consensus {rpc}: {} ({})", + status.message(), + status.code() + ); + match status.code() { + Code::InvalidArgument | Code::FailedPrecondition | Code::OutOfRange => { + ExternalError::Determinate(Determinate::new(msg)) + } + _ => ExternalError::Indeterminate(Indeterminate::new(msg)), + } +} + +fn from_proto(v: ProtoVersionedData) -> VersionedData { + VersionedData { + seqno: SeqNo(v.seqno), + data: v.data, + } +} + +fn to_proto(v: VersionedData) -> ProtoVersionedData { + ProtoVersionedData { + seqno: v.seqno.0, + data: v.data, + } +} + +#[async_trait] +impl Consensus for BogoConsensus { + fn list_keys(&self) -> ResultStream<'_, String> { + let client = Arc::clone(&self.client); + Box::pin(try_stream! { + let mut stream = client + .list_keys() + .await + .map_err(|s| status_to_external("list_keys", s))?; + while let Some(item) = stream.next().await { + let key = item.map_err(|s| status_to_external("list_keys", s))?; + yield key; + } + }) + } + + async fn head(&self, key: &str) -> Result, ExternalError> { + let wire_start = Instant::now(); + let data = self.client.head(key).await; + self.observe_wire("consensus_head", key, wire_start); + let data = data.map_err(|s| status_to_external("head", s))?; + Ok(data.map(from_proto)) + } + + async fn compare_and_set( + &self, + key: &str, + new: VersionedData, + ) -> Result { + if new.seqno.0 > i64::MAX.try_into().expect("i64::MAX fits in u64") { + return Err(ExternalError::from(anyhow!( + "sequence numbers must fit within [0, i64::MAX], received: {:?}", + new.seqno + ))); + } + let wire_start = Instant::now(); + let result = self.client.compare_and_set(key, to_proto(new)).await; + self.observe_wire("consensus_cas", key, wire_start); + let result = result.map_err(|s| status_to_external("compare_and_set", s))?; + Ok(match result { + ProtoCaSResult::Committed => CaSResult::Committed, + ProtoCaSResult::ExpectationMismatch => CaSResult::ExpectationMismatch, + ProtoCaSResult::Unspecified => { + return Err(ExternalError::Indeterminate(Indeterminate::new(anyhow!( + "bogo-consensus returned unspecified CaSResult" + )))); + } + }) + } + + async fn scan( + &self, + key: &str, + from: SeqNo, + limit: usize, + ) -> Result, ExternalError> { + let scan_all_u64 = u64::try_from(SCAN_ALL).unwrap_or(u64::MAX); + let limit_u64 = u64::try_from(limit).unwrap_or(scan_all_u64); + let wire_start = Instant::now(); + let data = self.client.scan(key, from.0, limit_u64).await; + self.observe_wire("consensus_scan", key, wire_start); + let data = data.map_err(|s| status_to_external("scan", s))?; + Ok(data.into_iter().map(from_proto).collect()) + } + + async fn truncate(&self, key: &str, seqno: SeqNo) -> Result, ExternalError> { + let wire_start = Instant::now(); + let deleted = self.client.truncate(key, seqno.0).await; + self.observe_wire("consensus_truncate", key, wire_start); + let deleted = deleted.map_err(|s| status_to_external("truncate", s))?; + Ok(deleted.map(|d| usize::try_from(d).unwrap_or(usize::MAX))) + } +} + +#[cfg(test)] +mod tests { + use std::net::{Ipv4Addr, SocketAddr}; + use std::str::FromStr; + use std::sync::Arc; + use std::time::Instant; + + use bytes::Bytes; + use futures_util::stream::{FuturesUnordered, StreamExt}; + use mz_bogo_consensus::metrics::BogoMetrics; + use mz_bogo_consensus::proto::bogo_consensus_server::BogoConsensusServer as BogoGrpcServer; + use mz_bogo_consensus::server::BogoConsensusServer; + use mz_ore::metrics::MetricsRegistry; + use mz_ore::url::SensitiveUrl; + use tokio::net::TcpListener; + use tokio_stream::wrappers::TcpListenerStream; + use tonic::transport::Server; + + use super::*; + use crate::location::tests::consensus_impl_test; + use crate::mem::MemConsensus; + + async fn spawn_server() -> SocketAddr { + let listener = TcpListener::bind(SocketAddr::from((Ipv4Addr::LOCALHOST, 0))) + .await + .expect("bind ephemeral port"); + let addr = listener.local_addr().expect("local_addr"); + let incoming = TcpListenerStream::new(listener); + let metrics = Arc::new(BogoMetrics::new(&MetricsRegistry::new())); + let svc = BogoGrpcServer::new(BogoConsensusServer::new(metrics)) + .max_decoding_message_size(256 * 1024 * 1024) + .max_encoding_message_size(256 * 1024 * 1024); + tokio::spawn(async move { + Server::builder() + .tcp_nodelay(true) + // Bump HTTP/2 flow control windows well above tonic's 65 KiB + // default. With the default, after the connection window fills + // up the sender stalls waiting for WINDOW_UPDATE round-trips, + // dominating per-call latency on local benchmarks. + .initial_stream_window_size(8 * 1024 * 1024) + .initial_connection_window_size(16 * 1024 * 1024) + .add_service(svc) + .serve_with_incoming(incoming) + .await + .expect("server"); + }); + addr + } + + #[mz_ore::test(tokio::test)] + #[cfg_attr(coverage, ignore)] + async fn bogo_consensus_conformance() -> Result<(), ExternalError> { + let addr = spawn_server().await; + consensus_impl_test(|| { + let url = SensitiveUrl::from_str(&format!("bogo://{}", addr)).expect("valid url"); + async move { + let cfg = BogoConsensusConfig::new(url)?; + BogoConsensus::open(cfg).await + } + }) + .await + } + + // Microbench: compare bogo (gRPC) vs MemConsensus (in-process) head-to-head. + // + // Run with: + // cargo test --release -p mz-persist bogo_consensus_microbench -- --ignored --nocapture + // + // The workload mirrors persist's per-shard access: many CAS appends, then + // scans returning growing history. We run sequentially first to capture + // per-op latency, then with concurrency to capture mutex/serialisation + // behaviour. + #[mz_ore::test(tokio::test(flavor = "multi_thread", worker_threads = 4))] + #[ignore] + async fn bogo_consensus_microbench() -> Result<(), ExternalError> { + let bogo_addr = spawn_server().await; + let bogo_url = SensitiveUrl::from_str(&format!("bogo://{}", bogo_addr)).expect("valid url"); + let bogo: Arc = + Arc::new(BogoConsensus::open(BogoConsensusConfig::new(bogo_url)?).await?); + let mem: Arc = Arc::new(MemConsensus::default()); + + for (name, c) in &[("mem", &mem), ("bogo", &bogo)] { + println!("\n=== {} ===", name); + // Fresh-server-state benches first to isolate per-op overhead from + // accumulated-state effects. iters=2000 to amortise warmup. + bench_serial_head(c, 2000).await; + bench_serial_cas_size(c, 2000, 0).await; + bench_serial_cas_size(c, 2000, 64).await; + bench_serial_cas_size(c, 2000, 1024).await; + bench_serial_cas_size(c, 2000, 16 * 1024).await; + bench_serial_head(c, 2000).await; // post-CAS + bench_serial_scan(c).await; + bench_concurrent_cas(c, 16).await; + bench_concurrent_cas(c, 64).await; + } + Ok(()) + } + + async fn bench_serial_cas_size(consensus: &Arc, iters: u64, size: usize) { + let key = format!("serial-cas-{}-{}", size, uuid::Uuid::new_v4()); + let data = Bytes::from(vec![0u8; size]); + let start = Instant::now(); + for i in 0..iters { + let r = consensus + .compare_and_set( + &key, + VersionedData { + seqno: SeqNo(i), + data: data.clone(), + }, + ) + .await + .expect("ok"); + assert_eq!(r, CaSResult::Committed); + } + let elapsed = start.elapsed(); + let per_op_us = elapsed.as_micros() as f64 / iters as f64; + println!( + " serial_cas (size={:6}): {} ops in {:.2?} = {:.1} μs/op", + size, iters, elapsed, per_op_us + ); + } + + async fn bench_serial_head(consensus: &Arc, iters: u64) { + let key = format!("head-{}", uuid::Uuid::new_v4()); + let start = Instant::now(); + for _ in 0..iters { + let _ = consensus.head(&key).await.expect("ok"); + } + let elapsed = start.elapsed(); + let per_op_us = elapsed.as_micros() as f64 / iters as f64; + println!( + " serial_head: {} ops in {:.2?} = {:.1} μs/op", + iters, elapsed, per_op_us + ); + } + + async fn bench_serial_scan(consensus: &Arc) { + // Pre-populate a key with 1000 versions, each 4 KiB. Then time scan-all. + let key = format!("scan-{}", uuid::Uuid::new_v4()); + let data = Bytes::from(vec![0u8; 4 * 1024]); + let history = 1000usize; + for i in 0..history { + consensus + .compare_and_set( + &key, + VersionedData { + seqno: SeqNo(u64::try_from(i).unwrap()), + data: data.clone(), + }, + ) + .await + .expect("ok"); + } + let iters = 100; + let start = Instant::now(); + for _ in 0..iters { + let res = consensus + .scan(&key, SeqNo::minimum(), SCAN_ALL) + .await + .expect("ok"); + assert_eq!(res.len(), history); + } + let elapsed = start.elapsed(); + let per_op_us = elapsed.as_micros() as f64 / iters as f64; + println!( + " serial_scan ({} versions × 4 KiB): {} ops in {:.2?} = {:.1} μs/op", + history, iters, elapsed, per_op_us + ); + } + + async fn bench_concurrent_cas(consensus: &Arc, concurrency: usize) { + // Many shards, each one client doing serial CAS. Spawn `concurrency` + // tasks that each hammer their own shard. Mimics persist's typical + // "many tables, all writing concurrently" pattern. + let data = Bytes::from(vec![0u8; 1024]); + let iters_per_shard: u64 = 100; + let mut handles = FuturesUnordered::new(); + let start = Instant::now(); + for shard_idx in 0..concurrency { + let c = Arc::clone(consensus); + let data = data.clone(); + let key = format!("concurrent-cas-{}-{}", uuid::Uuid::new_v4(), shard_idx); + handles.push(mz_ore::task::spawn(|| "bench_concurrent_cas", async move { + for i in 0..iters_per_shard { + let r = c + .compare_and_set( + &key, + VersionedData { + seqno: SeqNo(i), + data: data.clone(), + }, + ) + .await + .expect("ok"); + assert_eq!(r, CaSResult::Committed); + } + })); + } + while let Some(_h) = handles.next().await {} + let elapsed = start.elapsed(); + let total_ops = concurrency as u64 * iters_per_shard; + let throughput = total_ops as f64 / elapsed.as_secs_f64(); + println!( + " concurrent_cas (concurrency={}, {} ops/shard): total {} ops in {:.2?} = {:.0} ops/s", + concurrency, iters_per_shard, total_ops, elapsed, throughput, + ); + } +} diff --git a/src/persist/src/cfg.rs b/src/persist/src/cfg.rs index 4ee3173bd5785..47231b8fe6522 100644 --- a/src/persist/src/cfg.rs +++ b/src/persist/src/cfg.rs @@ -22,6 +22,7 @@ use mz_postgres_client::PostgresClientKnobs; use mz_postgres_client::metrics::PostgresClientMetrics; use crate::azure::{AzureBlob, AzureBlobConfig}; +use crate::bogo::{BogoConsensus, BogoConsensusConfig}; use crate::file::{FileBlob, FileBlobConfig}; #[cfg(feature = "foundationdb")] use crate::foundationdb::{FdbConsensus, FdbConsensusConfig}; @@ -223,6 +224,9 @@ pub enum ConsensusConfig { Postgres(PostgresConsensusConfig), /// Config for [MemConsensus], only available in testing. Mem, + /// Config for [BogoConsensus], an in-memory gRPC backend used for + /// performance testing. + Bogo(BogoConsensusConfig), #[cfg(feature = "turmoil")] /// Config for [crate::turmoil::TurmoilConsensus]. Turmoil(crate::turmoil::ConsensusConfig), @@ -240,6 +244,7 @@ impl ConsensusConfig { Ok(Arc::new(PostgresConsensus::open(config).await?)) } ConsensusConfig::Mem => Ok(Arc::new(MemConsensus::default())), + ConsensusConfig::Bogo(config) => Ok(Arc::new(BogoConsensus::open(config).await?)), #[cfg(feature = "turmoil")] ConsensusConfig::Turmoil(config) => { Ok(Arc::new(crate::turmoil::TurmoilConsensus::open(config))) @@ -268,6 +273,14 @@ impl ConsensusConfig { } Ok(ConsensusConfig::Mem) } + "bogo" => { + if !cfg!(debug_assertions) { + warn!("persist unexpectedly using bogo consensus in a release binary"); + } + Ok(ConsensusConfig::Bogo(BogoConsensusConfig::new( + url.clone(), + )?)) + } #[cfg(feature = "turmoil")] "turmoil" => { let cfg = crate::turmoil::ConsensusConfig::new(url); diff --git a/src/persist/src/lib.rs b/src/persist/src/lib.rs index 0d2177becb091..11ddb619478fc 100644 --- a/src/persist/src/lib.rs +++ b/src/persist/src/lib.rs @@ -18,6 +18,7 @@ )] pub mod azure; +pub mod bogo; pub mod cfg; pub mod error; pub mod file; diff --git a/src/sql/src/catalog.rs b/src/sql/src/catalog.rs index acab754e3690c..1529b926be745 100644 --- a/src/sql/src/catalog.rs +++ b/src/sql/src/catalog.rs @@ -447,7 +447,7 @@ pub trait CatalogDatabase { /// Returns the schemas of the database as a map from schema name to /// schema ID. - fn schema_ids(&self) -> &BTreeMap; + fn schema_ids(&self) -> &imbl::OrdMap; /// Returns the schemas of the database. fn schemas(&self) -> Vec<&dyn CatalogSchema>; @@ -691,7 +691,7 @@ impl From for RoleAttributesRaw { #[derive(Default, Clone, Debug, PartialEq, Eq, PartialOrd, Ord, Serialize)] pub struct RoleVars { /// Map of variable names to their value. - pub map: BTreeMap, + pub map: imbl::OrdMap, } /// A role in a [`SessionCatalog`]. @@ -706,13 +706,13 @@ pub trait CatalogRole { /// membership. /// /// Key is the role that some role is a member of, value is the grantor role ID. - fn membership(&self) -> &BTreeMap; + fn membership(&self) -> &imbl::OrdMap; /// Returns the attributes associated with this role. fn attributes(&self) -> &RoleAttributes; /// Returns all variables that this role has a default value stored for. - fn vars(&self) -> &BTreeMap; + fn vars(&self) -> &imbl::OrdMap; } /// A network policy in a [`SessionCatalog`]. @@ -739,11 +739,11 @@ pub trait CatalogCluster<'a> { fn id(&self) -> ClusterId; /// Returns the objects that are bound to this cluster. - fn bound_objects(&self) -> &BTreeSet; + fn bound_objects(&self) -> &imbl::OrdSet; /// Returns the replicas of the cluster as a map from replica name to /// replica ID. - fn replica_ids(&self) -> &BTreeMap; + fn replica_ids(&self) -> &imbl::OrdMap; /// Returns the replicas of the cluster. fn replicas(&self) -> Vec<&dyn CatalogClusterReplica<'_>>; @@ -1794,14 +1794,14 @@ pub struct RoleMembership { // from different grantors. This isn't a problem now since we don't implement ADMIN OPTION, but // we should figure this out before implementing ADMIN OPTION. It will likely require a messy // migration. - pub map: BTreeMap, + pub map: imbl::OrdMap, } impl RoleMembership { /// Creates a new [`RoleMembership`]. pub fn new() -> RoleMembership { RoleMembership { - map: BTreeMap::new(), + map: imbl::OrdMap::new(), } } } diff --git a/src/storage-client/Cargo.toml b/src/storage-client/Cargo.toml index 7492c38ce3ec2..f28daf18869ed 100644 --- a/src/storage-client/Cargo.toml +++ b/src/storage-client/Cargo.toml @@ -15,6 +15,7 @@ async-trait.workspace = true chrono.workspace = true differential-dataflow.workspace = true futures.workspace = true +imbl.workspace = true itertools.workspace = true mz-ccsr = { path = "../ccsr" } mz-cluster-client = { path = "../cluster-client" } diff --git a/src/storage-client/src/controller.rs b/src/storage-client/src/controller.rs index e946ab5977807..4e7bd6a3919ca 100644 --- a/src/storage-client/src/controller.rs +++ b/src/storage-client/src/controller.rs @@ -211,8 +211,8 @@ pub enum Response { #[derive(Debug, Clone, Serialize, Default)] pub struct StorageMetadata { #[serde(serialize_with = "mz_ore::serde::map_key_to_string")] - pub collection_metadata: BTreeMap, - pub unfinalized_shards: BTreeSet, + pub collection_metadata: imbl::OrdMap, + pub unfinalized_shards: imbl::OrdSet, } impl StorageMetadata { @@ -238,6 +238,12 @@ pub trait StorageTxn { /// The value of this map should be treated as opaque. fn get_collection_metadata(&self) -> BTreeMap; + /// Point lookup of the shard backing a single collection. O(log N). + fn get_collection_shard(&self, id: GlobalId) -> Option; + + /// Returns whether any collection metadata entry maps to `shard`. + fn collection_metadata_contains_shard(&self, shard: ShardId) -> bool; + /// Add new storage metadata for a collection. /// /// Subsequent calls to [`StorageTxn::get_collection_metadata`] must include diff --git a/src/storage-client/src/metrics.rs b/src/storage-client/src/metrics.rs index af996ec1234d9..bf25c517a4211 100644 --- a/src/storage-client/src/metrics.rs +++ b/src/storage-client/src/metrics.rs @@ -20,9 +20,11 @@ use mz_ore::metrics::{ CounterVec, DeleteOnDropCounter, DeleteOnDropGauge, IntCounterVec, MetricsRegistry, UIntGaugeVec, }; +use mz_ore::stats::histogram_seconds_buckets; use mz_repr::GlobalId; use mz_service::transport; use mz_storage_types::instances::StorageInstanceId; +use prometheus::HistogramVec; use prometheus::core::{AtomicF64, AtomicU64}; use crate::client::{StorageCommand, StorageResponse}; @@ -47,6 +49,9 @@ pub struct StorageControllerMetrics { replica_connects_total: IntCounterVec, replica_connect_wait_time_seconds_total: CounterVec, + /// Per-phase timing for `StorageController::create_collections_for_bootstrap`. + pub create_collections_phase_seconds: HistogramVec, + /// Metrics shared with the compute controller. shared: ControllerMetrics, } @@ -99,6 +104,13 @@ impl StorageControllerMetrics { help: "The total time the storage controller spent waiting for replica (re-)connection.", var_labels: ["instance_id", "replica_id"], )), + create_collections_phase_seconds: metrics_registry.register(metric!( + name: "mz_storage_controller_create_collections_phase_seconds", + help: "The time spent in each phase of a single \ + StorageController::create_collections_for_bootstrap call.", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.0001, 32.0), + )), shared, } diff --git a/src/storage-client/src/storage_collections.rs b/src/storage-client/src/storage_collections.rs index fadb0a318e74b..4be1f528b5a00 100644 --- a/src/storage-client/src/storage_collections.rs +++ b/src/storage-client/src/storage_collections.rs @@ -58,7 +58,7 @@ use timely::progress::frontier::MutableAntichain; use timely::progress::{Antichain, ChangeBatch}; use tokio::sync::{mpsc, oneshot}; use tokio::time::MissedTickBehavior; -use tracing::{debug, info, trace, warn}; +use tracing::{Instrument, debug, info, info_span, trace, warn}; use crate::client::TimestamplessUpdateBuilder; use crate::controller::{ @@ -382,6 +382,26 @@ pub struct StorageCollectionsImpl { /// Collections maintained by this [StorageCollections]. collections: Arc>>, + /// IDs of collections whose `CollectionState::primary` is `Some(_)`. + /// + /// This is a small "side index" maintained in lockstep with writes to + /// `CollectionState::primary` so that hot-path DDL code (specifically + /// `prepare_state`'s `dropped_shard_lookup` step) can decide whether a + /// dropped collection had a primary without taking the global + /// `collections` mutex. The big mutex is contended by the + /// `BackgroundTask`'s O(N) periodic sweep over txns-backed collections + /// (see `update_write_frontiers`), so any lock acquire from the DDL hot + /// path can queue behind hundreds of chunked acquires from the sweep. + /// By moving this single read off the contended path we kill the + /// `prepare_state` slope that survived phase 15: at N=25k tables it was + /// the dominant contributor to the per-DDL latency growth. + /// + /// Invariant: an id appears here iff it has a corresponding entry in + /// `collections` whose `primary.is_some()`. Maintained at every site + /// that writes `CollectionState::primary` and at the single + /// `collections.remove` site. + primaried_ids: Arc>>, + /// A shared TxnsCache running in a task and communicated with over a channel. txns_read: TxnsRead, @@ -398,6 +418,26 @@ pub struct StorageCollectionsImpl { /// cannot hydrate in read-only mode. initial_txn_upper: Antichain, + /// Latest known upper of the txns shard. The [`BackgroundTask`] updates + /// this whenever the txns shard upper advances and is the authoritative + /// source of the write frontier for every txns-backed (table) + /// collection. Readers that need the current write frontier of a + /// txns-backed collection should consult this field rather than the + /// per-collection `write_frontier` on [`CollectionState`]. + /// + /// Background: previously, the `BackgroundTask`'s txns-upper branch + /// would fan the freshly observed upper out to N per-collection + /// `write_frontier` fields on every tick. With many user tables, that + /// O(N) work under the global `collections` mutex blocked every other + /// taker (notably `prepare_state` on the DDL hot path), driving DDL + /// latency superlinearly with N. By storing the shared upper here once + /// and resolving reads through this field, we eliminate the fanout from + /// the hot path entirely. The implied capability (since) downgrade for + /// txns-backed collections still needs to happen so persist can compact; + /// it is driven by a separate timer-based sweep in `BackgroundTask::run` + /// (see `TXNS_SINCE_DOWNGRADE_INTERVAL`). + txns_upper: Arc>>, + /// The persist location where all storage collections are being written to persist_location: PersistLocation, @@ -410,6 +450,12 @@ pub struct StorageCollectionsImpl { /// For sending updates about read holds to our internal task. holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch)>, + /// Per-phase histograms for `create_collections_for_bootstrap`. + create_collections_phase_seconds: prometheus::HistogramVec, + + /// Per-phase histograms for `prepare_state`. + prepare_state_phase_seconds: prometheus::HistogramVec, + /// Handles to tasks we own, making sure they're dropped when we are. _background_task: Arc>, _finalize_shards_task: Arc>, @@ -491,6 +537,7 @@ impl StorageCollectionsImpl { let txns_read = TxnsRead::start::(txns_client.clone(), txns_id).await; let collections = Arc::new(std::sync::Mutex::new(BTreeMap::default())); + let primaried_ids = Arc::new(std::sync::Mutex::new(BTreeSet::default())); let finalizable_shards = Arc::new(ShardIdSet::new(metrics.finalization_outstanding.clone())); let finalized_shards = @@ -501,6 +548,9 @@ impl StorageCollectionsImpl { ))); let initial_txn_upper = txns_write.fetch_recent_upper().await.to_owned(); + // Initialize the shared txns upper to the boot-time value; the + // BackgroundTask will keep it updated as the txns shard advances. + let txns_upper = Arc::new(std::sync::RwLock::new(initial_txn_upper.clone())); let (cmd_tx, cmd_rx) = mpsc::unbounded_channel(); let (holds_tx, holds_rx) = mpsc::unbounded_channel(); @@ -510,11 +560,15 @@ impl StorageCollectionsImpl { cmds_rx: cmd_rx, holds_rx, collections: Arc::clone(&collections), + primaried_ids: Arc::clone(&primaried_ids), finalizable_shards: Arc::clone(&finalizable_shards), shard_by_id: BTreeMap::new(), since_handles: BTreeMap::new(), txns_handle: Some(txns_write), txns_shards: Default::default(), + txns_upper: Arc::clone(&txns_upper), + txns_upper_advances: metrics.txns_upper_advances.clone(), + txns_since_sweeps: metrics.txns_since_sweeps.clone(), }; let background_task = @@ -522,6 +576,9 @@ impl StorageCollectionsImpl { background_task.run().await }); + let create_collections_phase_seconds = metrics.create_collections_phase_seconds.clone(); + let prepare_state_phase_seconds = metrics.prepare_state_phase_seconds.clone(); + let finalize_shards_task = mz_ore::task::spawn( || "storage_collections::finalize_shards_task", finalize_shards_task(FinalizeShardsTaskConfig { @@ -540,20 +597,56 @@ impl StorageCollectionsImpl { finalizable_shards, finalized_shards, collections, + primaried_ids, txns_read, envd_epoch, read_only, config, initial_txn_upper, + txns_upper, persist_location, persist: persist_clients, cmd_tx, holds_tx, + create_collections_phase_seconds, + prepare_state_phase_seconds, _background_task: Arc::new(background_task.abort_on_drop()), _finalize_shards_task: Arc::new(finalize_shards_task.abort_on_drop()), } } + /// Returns the effective write frontier of `collection`: the shared + /// txns upper if the collection is txns-backed, otherwise the + /// per-collection `write_frontier`. + /// + /// The per-collection `write_frontier` is no longer kept current by + /// the [`BackgroundTask`] for txns-backed collections (the per-tick + /// fanout used to drive that field was the slope owner of + /// `prepare_state`). Callers that need the up-to-date write frontier + /// of a txns-backed collection must consult `self.txns_upper`. The + /// periodic sweep in `BackgroundTask::run` does still rewrite each + /// txns-backed collection's `write_frontier` once per second so that + /// downstream bookkeeping (implied capability, read holds) stays in + /// step, but between those sweeps the shared field is the source of + /// truth. + fn effective_write_frontier(&self, collection: &CollectionState) -> Antichain { + if collection.collection_metadata.txns_shard.is_some() { + let shared = self.txns_upper.read().expect("lock poisoned"); + // The shared upper monotonically advances; pick whichever is + // later. We compare instead of unconditionally taking the + // shared one because `collection.write_frontier` may have been + // advanced past `shared` by the periodic sweep / a direct + // upper update path. + if PartialOrder::less_than(&collection.write_frontier, &*shared) { + shared.clone() + } else { + collection.write_frontier.clone() + } + } else { + collection.write_frontier.clone() + } + } + /// Opens a [WriteHandle] and a [SinceHandleWrapper], for holding back the since. /// /// `since` is an optional since that the read handle will be forwarded to @@ -580,19 +673,24 @@ impl StorageCollectionsImpl { } else { // We're managing the data for this shard in read-write mode, which would fence out other // processes in read-only mode; it's safe to upgrade the metadata version. - persist_client - .upgrade_version::( - shard, - Diagnostics { - shard_name: id.to_string(), - handle_purpose: format!("controller data for {}", id), - }, - ) - .await - .expect("invalid persist usage"); + async { + persist_client + .upgrade_version::( + shard, + Diagnostics { + shard_name: id.to_string(), + handle_purpose: format!("controller data for {}", id), + }, + ) + .await + .expect("invalid persist usage"); + } + .instrument(info_span!("odh::upgrade_version")) + .await; let since_handle = self .open_critical_handle(id, shard, since, persist_client) + .instrument(info_span!("odh::open_critical_handle")) .await; SinceHandleWrapper::Critical(since_handle) @@ -600,6 +698,7 @@ impl StorageCollectionsImpl { let mut write_handle = self .open_write_handle(id, shard, relation_desc, persist_client) + .instrument(info_span!("odh::open_write_handle")) .await; // N.B. @@ -612,7 +711,11 @@ impl StorageCollectionsImpl { // // Note that this returns the upper, but also sets it on the handle to // be fetched later. - write_handle.fetch_recent_upper().await; + async { + write_handle.fetch_recent_upper().await; + } + .instrument(info_span!("odh::fetch_recent_upper")) + .await; (write_handle, since_handle) } @@ -809,6 +912,7 @@ impl StorageCollectionsImpl { /// /// This is necessary to ensure that the dependency's since does not advance /// beyond its dependents'. + #[instrument(level = "debug", fields(id = ?id))] fn install_collection_dependency_read_holds_inner( &self, self_collections: &mut BTreeMap, @@ -939,6 +1043,7 @@ impl StorageCollectionsImpl { StorageCollectionsImpl::update_read_capabilities_inner( &self.cmd_tx, self_collections, + &self.primaried_ids, &mut storage_read_updates, ); @@ -1137,6 +1242,9 @@ impl StorageCollectionsImpl { trace!("set_read_policies: {:?}", policies); let mut read_capability_changes = BTreeMap::default(); + // Snapshot the shared txns upper once so all txns-backed + // collections in this batch see a consistent value. + let txns_upper = self.txns_upper.read().expect("lock poisoned").clone(); for (id, policy) in policies.into_iter() { let collection = match collections.get_mut(&id) { @@ -1146,7 +1254,17 @@ impl StorageCollectionsImpl { } }; - let mut new_read_capability = policy.frontier(collection.write_frontier.borrow()); + // For txns-backed collections, the per-collection + // `write_frontier` may lag the actual upper between periodic + // sweeps. Use the shared txns upper if it is more advanced. + let effective_write_frontier = if collection.collection_metadata.txns_shard.is_some() + && PartialOrder::less_than(&collection.write_frontier, &txns_upper) + { + txns_upper.clone() + } else { + collection.write_frontier.clone() + }; + let mut new_read_capability = policy.frontier(effective_write_frontier.borrow()); if PartialOrder::less_equal(&collection.implied_capability, &new_read_capability) { let mut update = ChangeBatch::new(); @@ -1171,6 +1289,7 @@ impl StorageCollectionsImpl { StorageCollectionsImpl::update_read_capabilities_inner( &self.cmd_tx, collections, + &self.primaried_ids, &mut read_capability_changes, ); } @@ -1179,9 +1298,11 @@ impl StorageCollectionsImpl { // This is not an associated function so that we can share it with the task // that updates the persist handles and also has a reference to the shared // collections state. + #[instrument(level = "debug")] fn update_read_capabilities_inner( cmd_tx: &mpsc::UnboundedSender, collections: &mut BTreeMap, + primaried_ids: &std::sync::Mutex>, updates: &mut BTreeMap>, ) { // Location to record consequences that we need to act on. @@ -1265,15 +1386,20 @@ impl StorageCollectionsImpl { // Translate our net compute actions into downgrades of persist sinces. // The actual downgrades are performed by a Tokio task asynchronously. let mut persist_compaction_commands = Vec::with_capacity(collections_net.len()); + let mut removed_primaried = Vec::new(); for (key, (mut changes, frontier)) in collections_net { if !changes.is_empty() { // If the collection has a "primary" collection, let that primary drive compaction. let collection = collections.get(&key).expect("must still exist"); - let should_emit_persist_compaction = collection.primary.is_none(); + let was_primaried = collection.primary.is_some(); + let should_emit_persist_compaction = !was_primaried; if frontier.is_empty() { info!(id = %key, "removing collection state because the since advanced to []!"); collections.remove(&key).expect("must still exist"); + if was_primaried { + removed_primaried.push(key); + } } if should_emit_persist_compaction { @@ -1282,6 +1408,17 @@ impl StorageCollectionsImpl { } } + // Keep the `primaried_ids` side index in lockstep with the + // `collections` removals above. Doing this in a single lock acquire + // after the removal loop keeps it off the per-iteration critical + // path. (Typical loop body: 0 entries; in rare cases , ids_to_register: BTreeMap, ) -> Result<(), StorageError> { - txn.insert_collection_metadata( - ids_to_add - .into_iter() - .map(|id| (id, ShardId::new())) - .collect(), - )?; - txn.insert_collection_metadata(ids_to_register)?; + let phase = self.prepare_state_phase_seconds.clone(); + + { + let _t = phase.with_label_values(&["insert_add"]).start_timer(); + txn.insert_collection_metadata( + ids_to_add + .into_iter() + .map(|id| (id, ShardId::new())) + .collect(), + )?; + } + { + let _t = phase.with_label_values(&["insert_register"]).start_timer(); + txn.insert_collection_metadata(ids_to_register)?; + } // Delete the metadata for any dropped collections. - let dropped_mappings = txn.delete_collection_metadata(ids_to_drop); + let dropped_mappings = { + let _t = phase.with_label_values(&["delete"]).start_timer(); + txn.delete_collection_metadata(ids_to_drop) + }; // Only finalize the shards of dropped collections that don't have a primary. // Otherwise the shard might still be in use by the primary. - let mut dropped_shards = BTreeSet::new(); - { - let collections = self.collections.lock().expect("poisoned"); + // + // We deliberately consult `self.primaried_ids` (a small side index + // of ids whose `primary.is_some()`) instead of the big `collections` + // BTreeMap. The `collections` mutex is contended by the + // `BackgroundTask`'s periodic O(N) sweep (`update_write_frontiers`) + // over txns-backed collections; that sweep takes the lock in chunks + // so any DDL acquire here queues behind in-flight chunks. The lock + // wait drove the `prepare_state` slope by +0.2 ms / +1k tables at + // N=10k → 25k. By using a separate, rarely-written side index we + // remove this contention entirely. + let dropped_shards = { + let _t = phase + .with_label_values(&["dropped_shard_lookup"]) + .start_timer(); + let mut dropped_shards = BTreeSet::new(); + let primaried_ids = self.primaried_ids.lock().expect("lock poisoned"); for (id, shard) in dropped_mappings { - let coll = collections.get(&id).expect("must exist"); - if coll.primary.is_none() { + if !primaried_ids.contains(&id) { dropped_shards.insert(shard); } } + dropped_shards + }; + { + let _t = phase + .with_label_values(&["insert_unfinalized"]) + .start_timer(); + txn.insert_unfinalized_shards(dropped_shards)?; } - txn.insert_unfinalized_shards(dropped_shards)?; // Reconcile any shards we've successfully finalized with the shard // finalization collection. - let finalized_shards = self.finalized_shards.lock().iter().copied().collect(); - txn.mark_shards_as_finalized(finalized_shards); + { + let _t = phase.with_label_values(&["mark_finalized"]).start_timer(); + let finalized_shards = self.finalized_shards.lock().iter().copied().collect(); + txn.mark_shards_as_finalized(finalized_shards); + } Ok(()) } @@ -1690,6 +1859,11 @@ impl StorageCollections for StorageCollectionsImpl { mut collections: Vec<(GlobalId, CollectionDescription)>, migrated_storage_collections: &BTreeSet, ) -> Result<(), StorageError> { + let phase_metric = self.create_collections_phase_seconds.clone(); + let _validate_timer = phase_metric + .with_label_values(&["validate_and_enrich"]) + .start_timer(); + let is_in_txns = |id, metadata: &CollectionMetadata| { metadata.txns_shard.is_some() && !(self.read_only && migrated_storage_collections.contains(&id)) @@ -1733,102 +1907,122 @@ impl StorageCollections for StorageCollectionsImpl { }) .collect_vec(); + drop(_validate_timer); + let _open_client_timer = phase_metric + .with_label_values(&["open_persist_client"]) + .start_timer(); // So that we can open `SinceHandle`s for each collections concurrently. - let persist_client = self - .persist - .open(self.persist_location.clone()) - .await - .unwrap(); + let persist_client = async { + self.persist + .open(self.persist_location.clone()) + .await + .unwrap() + } + .instrument(info_span!("ccfb::open_persist_client")) + .await; let persist_client = &persist_client; + drop(_open_client_timer); + let _open_handles_timer = phase_metric + .with_label_values(&["open_data_handles_concurrent"]) + .start_timer(); // Reborrow the `&mut self` as immutable, as all the concurrent work to // be processed in this stream cannot all have exclusive access. use futures::stream::{StreamExt, TryStreamExt}; let this = &*self; - let mut to_register: Vec<_> = futures::stream::iter(enriched_with_metadata) - .map(|data: Result<_, StorageError>| { - async move { - let (id, description, metadata) = data?; - - // should be replaced with real introspection - // (https://github.com/MaterializeInc/database-issues/issues/4078) - // but for now, it's helpful to have this mapping written down - // somewhere - debug!("mapping GlobalId={} to shard ({})", id, metadata.data_shard); - - // If this collection has a primary, the primary is responsible for downgrading - // the critical since and it would be an error if we did so here while opening - // the since handle. - let since = if description.primary.is_some() { - None - } else { - description.since.as_ref() - }; - - let (write, mut since_handle) = this - .open_data_handles( - &id, - metadata.data_shard, - since, - metadata.relation_desc.clone(), - persist_client, - ) - .await; + let mut to_register: Vec<_> = async { + futures::stream::iter(enriched_with_metadata) + .map(|data: Result<_, StorageError>| { + async move { + let (id, description, metadata) = data?; + + // should be replaced with real introspection + // (https://github.com/MaterializeInc/database-issues/issues/4078) + // but for now, it's helpful to have this mapping written down + // somewhere + debug!("mapping GlobalId={} to shard ({})", id, metadata.data_shard); + + // If this collection has a primary, the primary is responsible for downgrading + // the critical since and it would be an error if we did so here while opening + // the since handle. + let since = if description.primary.is_some() { + None + } else { + description.since.as_ref() + }; + + let (write, mut since_handle) = this + .open_data_handles( + &id, + metadata.data_shard, + since, + metadata.relation_desc.clone(), + persist_client, + ) + .await; - // Present tables as springing into existence at the register_ts - // by advancing the since. Otherwise, we could end up in a - // situation where a table with a long compaction window appears - // to exist before the environment (and this the table) existed. - // - // We could potentially also do the same thing for other - // sources, in particular storage's internal sources and perhaps - // others, but leave them for now. - match description.data_source { - DataSource::Introspection(_) - | DataSource::IngestionExport { .. } - | DataSource::Webhook - | DataSource::Ingestion(_) - | DataSource::Progress - | DataSource::Other => {} - DataSource::Sink { .. } => {} - DataSource::Table => { - let register_ts = register_ts.expect( + // Present tables as springing into existence at the register_ts + // by advancing the since. Otherwise, we could end up in a + // situation where a table with a long compaction window appears + // to exist before the environment (and this the table) existed. + // + // We could potentially also do the same thing for other + // sources, in particular storage's internal sources and perhaps + // others, but leave them for now. + match description.data_source { + DataSource::Introspection(_) + | DataSource::IngestionExport { .. } + | DataSource::Webhook + | DataSource::Ingestion(_) + | DataSource::Progress + | DataSource::Other => {} + DataSource::Sink { .. } => {} + DataSource::Table => { + let register_ts = register_ts.expect( "caller should have provided a register_ts when creating a table", ); - if since_handle.since().elements() == &[Timestamp::MIN] - && !migrated_storage_collections.contains(&id) - { - debug!("advancing {} to initial since of {:?}", id, register_ts); - let token = since_handle.opaque(); - let _ = since_handle - .compare_and_downgrade_since( - &token, - (&token, &Antichain::from_elem(register_ts)), - ) - .await; + if since_handle.since().elements() == &[Timestamp::MIN] + && !migrated_storage_collections.contains(&id) + { + debug!( + "advancing {} to initial since of {:?}", + id, register_ts + ); + let token = since_handle.opaque(); + let _ = since_handle + .compare_and_downgrade_since( + &token, + (&token, &Antichain::from_elem(register_ts)), + ) + .await; + } } } - } - Ok::<_, StorageError>((id, description, write, since_handle, metadata)) - } - }) - // Poll each future for each collection concurrently, maximum of 50 at a time. - .buffer_unordered(50) - // HERE BE DRAGONS: - // - // There are at least 2 subtleties in using `FuturesUnordered` - // (which `buffer_unordered` uses underneath: - // - One is captured here - // - // - And the other is deadlocking if processing an OUTPUT of a - // `FuturesUnordered` stream attempts to obtain an async mutex that - // is also obtained in the futures being polled. - // - // Both of these could potentially be issues in all usages of - // `buffer_unordered` in this method, so we stick the standard - // advice: only use `try_collect` or `collect`! - .try_collect() - .await?; + Ok::<_, StorageError>((id, description, write, since_handle, metadata)) + } + }) + // Poll each future for each collection concurrently, maximum of 50 at a time. + .buffer_unordered(50) + // HERE BE DRAGONS: + // + // There are at least 2 subtleties in using `FuturesUnordered` + // (which `buffer_unordered` uses underneath: + // - One is captured here + // + // - And the other is deadlocking if processing an OUTPUT of a + // `FuturesUnordered` stream attempts to obtain an async mutex that + // is also obtained in the futures being polled. + // + // Both of these could potentially be issues in all usages of + // `buffer_unordered` in this method, so we stick the standard + // advice: only use `try_collect` or `collect`! + .try_collect() + .await + } + .instrument(info_span!("ccfb::open_data_handles_concurrent")) + .await?; + drop(_open_handles_timer); + let _sort_timer = phase_metric.with_label_values(&["sort"]).start_timer(); // Reorder in dependency order. #[derive(Ord, PartialOrd, Eq, PartialEq)] @@ -1845,10 +2039,15 @@ impl StorageCollections for StorageCollectionsImpl { DataSource::Sink { .. } => DependencyOrder::Sink(*id), _ => DependencyOrder::Collection(*id), }); + drop(_sort_timer); + let _install_timer = phase_metric + .with_label_values(&["install_collection_states"]) + .start_timer(); // We hold this lock for a very short amount of time, just doing some // hashmap inserts and unbounded channel sends. let mut self_collections = self.collections.lock().expect("lock poisoned"); + let _install_span = info_span!("ccfb::install_collection_states").entered(); for (id, description, write_handle, since_handle, metadata) in to_register { let write_frontier = write_handle.upper(); @@ -1971,6 +2170,14 @@ impl StorageCollections for StorageCollectionsImpl { metadata.clone(), ); + // Keep the `primaried_ids` side index in lockstep with writes to + // `CollectionState::primary`. `description.primary` is almost + // always `None` in normal usage, but if a fresh collection is + // installed with a primary already set, record it. + if description.primary.is_some() { + self.primaried_ids.lock().expect("lock poisoned").insert(id); + } + // Install the collection state in the appropriate spot. match &description.data_source { DataSource::Introspection(_) => { @@ -2019,9 +2226,16 @@ impl StorageCollections for StorageCollectionsImpl { self.install_collection_dependency_read_holds_inner(&mut *self_collections, id)?; } + drop(_install_span); drop(self_collections); + drop(_install_timer); + let _finalize_timer = phase_metric + .with_label_values(&["synchronize_finalized_shards"]) + .start_timer(); + let _span = info_span!("ccfb::synchronize_finalized_shards").entered(); self.synchronize_finalized_shards(storage_metadata); + drop(_span); Ok(()) } @@ -2128,11 +2342,24 @@ impl StorageCollections for StorageCollectionsImpl { existing.primary = Some(new_collection); existing.storage_dependencies.push(new_collection); + // Mirror the primary-set into the `primaried_ids` side index so + // that `prepare_state`'s `dropped_shard_lookup` can answer the + // primary question without taking the contended `collections` + // mutex. + self.primaried_ids + .lock() + .expect("lock poisoned") + .insert(existing_collection); + // Copy over the frontiers from the previous version. // The new table starts with two holds - the implied capability, and the hold from // the previous version - both at the previous version's read frontier. let implied_capability = existing.read_capabilities.frontier().to_owned(); - let write_frontier = existing.write_frontier.clone(); + // Use the effective write frontier: for a txns-backed table the + // per-collection `write_frontier` may be stale (see + // `effective_write_frontier`). The new version's starting upper + // must not regress past the actual current txns upper. + let write_frontier = self.effective_write_frontier(existing); // Determine the relevant read capabilities on the new collection. // @@ -2167,6 +2394,7 @@ impl StorageCollections for StorageCollectionsImpl { StorageCollectionsImpl::update_read_capabilities_inner( &self.cmd_tx, &mut *self_collections, + &self.primaried_ids, &mut updates, ); }; @@ -2256,6 +2484,7 @@ impl StorageCollections for StorageCollectionsImpl { } } + #[instrument(level = "debug", fields(n = desired_holds.len()))] fn acquire_read_holds( &self, desired_holds: Vec, @@ -2295,6 +2524,7 @@ impl StorageCollections for StorageCollectionsImpl { StorageCollectionsImpl::update_read_capabilities_inner( &self.cmd_tx, &mut collections, + &self.primaried_ids, &mut updates, ); @@ -2327,13 +2557,17 @@ impl StorageCollections for StorageCollectionsImpl { finalizable_shards, finalized_shards, collections, + primaried_ids, txns_read: _, config, initial_txn_upper, + txns_upper, persist_location, persist: _, cmd_tx: _, holds_tx: _, + create_collections_phase_seconds: _, + prepare_state_phase_seconds: _, _background_task: _, _finalize_shards_task: _, } = self; @@ -2354,16 +2588,26 @@ impl StorageCollections for StorageCollectionsImpl { .iter() .map(|(id, c)| (id.to_string(), format!("{c:?}"))) .collect(); + let primaried_ids: Vec<_> = primaried_ids + .lock() + .expect("lock poisoned") + .iter() + .map(ToString::to_string) + .collect(); let config = format!("{:?}", config.lock().expect("poisoned")); + let txns_upper = format!("{:?}", txns_upper.read().expect("lock poisoned")); + Ok(serde_json::json!({ "envd_epoch": envd_epoch, "read_only": read_only, "finalizable_shards": finalizable_shards, "finalized_shards": finalized_shards, "collections": collections, + "primaried_ids": primaried_ids, "config": config, "initial_txn_upper": initial_txn_upper, + "txns_upper": txns_upper, "persist_location": format!("{persist_location:?}"), })) } @@ -2577,12 +2821,24 @@ struct BackgroundTask { holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch)>, finalizable_shards: Arc, collections: Arc>>, + /// Shared with [`StorageCollectionsImpl::primaried_ids`]; see the doc + /// comment on the parent field for rationale. + primaried_ids: Arc>>, // So we know what shard ID corresponds to what global ID, which we need // when re-enqueing futures for determining the next upper update. shard_by_id: BTreeMap, since_handles: BTreeMap, txns_handle: Option>, txns_shards: BTreeSet, + /// Shared with [`StorageCollectionsImpl::txns_upper`]; the latest observed + /// upper of the txns shard. See the doc comment on the parent field for + /// rationale. + txns_upper: Arc>>, + /// Counter incremented on each observed advance of the txns shard upper. + txns_upper_advances: prometheus::Counter, + /// Counter incremented on each periodic since-downgrade sweep over + /// txns-backed collections. + txns_since_sweeps: prometheus::Counter, } #[derive(Debug)] @@ -2652,19 +2908,62 @@ impl BackgroundTask { None => async { std::future::pending().await }.boxed(), }; + // Periodic sweep that downgrades the implied capability of every + // txns-backed collection to track the shared txns upper. The + // txns-upper branch updates the shared field on every advance, but + // does *not* fan out per-collection: doing so on every tick is O(N) + // under the global `collections` mutex and starves the DDL hot path + // (see the doc comment on `StorageCollectionsImpl::txns_upper`). We + // still need to drive persist's since forward for these collections, + // so we do the bookkeeping here at a fixed cadence. The cost per + // sweep is unchanged from the per-tick fanout, but it no longer + // contends with `prepare_state` once per txns-upper-tick. + const TXNS_SINCE_DOWNGRADE_INTERVAL: Duration = Duration::from_secs(1); + let mut txns_since_downgrade_interval = + tokio::time::interval(TXNS_SINCE_DOWNGRADE_INTERVAL); + // Avoid the initial immediate tick — the txns_shards set starts + // empty at boot, so the first sweep would be a no-op. + txns_since_downgrade_interval + .set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + loop { tokio::select! { (id, handle, upper) = &mut txns_upper_future => { trace!("new upper from txns shard: {:?}", upper); - let mut uppers = Vec::new(); - for id in self.txns_shards.iter() { - uppers.push((*id, &upper)); - } - self.update_write_frontiers(&uppers).await; + // Publish the new upper to the shared field. Readers + // that want the current write_frontier of a + // txns-backed collection consult this field rather than + // the per-collection `write_frontier`. We deliberately + // do NOT iterate `self.txns_shards` here: that was the + // O(N) lock-hold on `self.collections` that drove the + // prepare_state slope. The implied-capability / + // since-downgrade work that previously happened in this + // branch is now handled by the periodic + // `txns_since_downgrade_interval` arm below. + *self.txns_upper.write().expect("lock poisoned") = upper.clone(); + self.txns_upper_advances.inc(); let fut = gen_upper_future(id, handle, upper); txns_upper_future = fut.boxed(); } + _ = txns_since_downgrade_interval.tick() => { + // Apply the shared txns upper to the implied capability + // of every txns-backed collection, exactly as the + // pre-Phase-15 txns-upper branch did. We coalesce many + // upper-ticks into one sweep so this O(N) work happens + // at a known low frequency instead of on the DDL hot + // path. + let upper = self.txns_upper.read().expect("lock poisoned").clone(); + if !upper.is_empty() && !self.txns_shards.is_empty() { + let uppers: Vec<_> = self + .txns_shards + .iter() + .map(|id| (*id, &upper)) + .collect(); + self.update_write_frontiers(&uppers).await; + } + self.txns_since_sweeps.inc(); + } Some((id, handle, upper)) = upper_futures.next() => { if id.is_user() { trace!("new upper for collection {id}: {:?}", upper); @@ -2800,6 +3099,7 @@ impl BackgroundTask { StorageCollectionsImpl::update_read_capabilities_inner( &self.cmds_tx, &mut collections, + &self.primaried_ids, &mut batched_changes, ); } @@ -2813,52 +3113,69 @@ impl BackgroundTask { async fn update_write_frontiers(&self, updates: &[(GlobalId, &Antichain)]) { let mut read_capability_changes = BTreeMap::default(); - let mut self_collections = self.collections.lock().expect("lock poisoned"); - - for (id, new_upper) in updates.iter() { - let collection = if let Some(c) = self_collections.get_mut(id) { - c - } else { - trace!( - "Reference to absent collection {id}, due to concurrent removal of that collection" - ); - continue; - }; + // The txns-upper branch in `run()` calls this with one entry per + // txns-backed collection (all sharing the same `upper`), so `updates` + // grows linearly with the number of user tables. We can't hold + // `self.collections` for a whole O(N) iteration: it's the same mutex + // that `prepare_state` (and every DDL path that reads/writes + // CollectionState) needs, so a single sweep starves all DDL behind it. + // + // Process the updates in chunks, releasing the lock between chunks so + // that competing acquirers get a chance to interleave. The chunk size + // is a tradeoff: smaller = lower lock-hold ceiling for prepare_state, + // larger = less lock acquire/release overhead. 256 keeps per-chunk + // worst-case under ~1 ms even at higher per-iteration cost. + const CHUNK_SIZE: usize = 256; + for chunk in updates.chunks(CHUNK_SIZE) { + let mut self_collections = self.collections.lock().expect("lock poisoned"); + for (id, new_upper) in chunk.iter() { + let collection = if let Some(c) = self_collections.get_mut(id) { + c + } else { + trace!( + "Reference to absent collection {id}, due to concurrent removal of that collection" + ); + continue; + }; - if PartialOrder::less_than(&collection.write_frontier, *new_upper) { - collection.write_frontier.clone_from(new_upper); - } + if PartialOrder::less_than(&collection.write_frontier, *new_upper) { + collection.write_frontier.clone_from(new_upper); + } - let mut new_read_capability = collection - .read_policy - .frontier(collection.write_frontier.borrow()); + let mut new_read_capability = collection + .read_policy + .frontier(collection.write_frontier.borrow()); - if id.is_user() { - trace!( - %id, - implied_capability = ?collection.implied_capability, - policy = ?collection.read_policy, - write_frontier = ?collection.write_frontier, - ?new_read_capability, - "update_write_frontiers"); - } + if id.is_user() { + trace!( + %id, + implied_capability = ?collection.implied_capability, + policy = ?collection.read_policy, + write_frontier = ?collection.write_frontier, + ?new_read_capability, + "update_write_frontiers"); + } - if PartialOrder::less_equal(&collection.implied_capability, &new_read_capability) { - let mut update = ChangeBatch::new(); - update.extend(new_read_capability.iter().map(|time| (*time, 1))); - std::mem::swap(&mut collection.implied_capability, &mut new_read_capability); - update.extend(new_read_capability.iter().map(|time| (*time, -1))); + if PartialOrder::less_equal(&collection.implied_capability, &new_read_capability) { + let mut update = ChangeBatch::new(); + update.extend(new_read_capability.iter().map(|time| (*time, 1))); + std::mem::swap(&mut collection.implied_capability, &mut new_read_capability); + update.extend(new_read_capability.iter().map(|time| (*time, -1))); - if !update.is_empty() { - read_capability_changes.insert(*id, update); + if !update.is_empty() { + read_capability_changes.insert(*id, update); + } } } + // Lock released at end of scope. } if !read_capability_changes.is_empty() { + let mut self_collections = self.collections.lock().expect("lock poisoned"); StorageCollectionsImpl::update_read_capabilities_inner( &self.cmds_tx, &mut self_collections, + &self.primaried_ids, &mut read_capability_changes, ); } @@ -3351,10 +3668,22 @@ mod tests { UIntGauge::new("finalizable_shards", "dummy gauge for tests").unwrap(), )), collections: Arc::new(Mutex::new(BTreeMap::new())), + primaried_ids: Arc::new(Mutex::new(BTreeSet::new())), shard_by_id: BTreeMap::new(), since_handles: BTreeMap::new(), txns_handle: None, txns_shards: BTreeSet::new(), + txns_upper: Arc::new(std::sync::RwLock::new(Antichain::new())), + txns_upper_advances: prometheus::Counter::new( + "txns_upper_advances", + "dummy counter for tests", + ) + .unwrap(), + txns_since_sweeps: prometheus::Counter::new( + "txns_since_sweeps", + "dummy counter for tests", + ) + .unwrap(), }; (cmds_tx, task) diff --git a/src/storage-client/src/storage_collections/metrics.rs b/src/storage-client/src/storage_collections/metrics.rs index a0bc9486706cd..a603d8d3e51fb 100644 --- a/src/storage-client/src/storage_collections/metrics.rs +++ b/src/storage-client/src/storage_collections/metrics.rs @@ -14,8 +14,9 @@ use std::sync::{Mutex, MutexGuard}; use mz_ore::cast::CastFrom; use mz_ore::metric; use mz_ore::metrics::{MetricsRegistry, UIntGauge}; +use mz_ore::stats::histogram_seconds_buckets; use mz_persist_types::ShardId; -use prometheus::Counter; +use prometheus::{Counter, HistogramVec}; #[derive(Debug)] pub struct StorageCollectionsMetrics { @@ -24,6 +25,17 @@ pub struct StorageCollectionsMetrics { pub finalization_started: Counter, pub finalization_succeeded: Counter, pub finalization_failed: Counter, + pub create_collections_phase_seconds: HistogramVec, + pub prepare_state_phase_seconds: HistogramVec, + /// Number of observed advances of the txns shard upper. Incremented once + /// each time the `BackgroundTask` learns of a new txns upper. Useful for + /// telling apart "txns shard genuinely commits at rate X" from + /// "BackgroundTask is doing O(N) work at rate X for some other reason". + pub txns_upper_advances: Counter, + /// Number of times the periodic since-downgrade sweep for txns-backed + /// collections has run. Compare with `txns_upper_advances` to see how + /// much fanout work is being coalesced by the sweep. + pub txns_since_sweeps: Counter, } impl StorageCollectionsMetrics { @@ -49,6 +61,30 @@ impl StorageCollectionsMetrics { name: "mz_shard_finalization_op_failed", help: "count of shard finalization operations that failed", )), + create_collections_phase_seconds: registry.register(metric!( + name: "mz_storage_collections_create_collections_phase_seconds", + help: "The time spent in each phase of a single \ + StorageCollections::create_collections_for_bootstrap call.", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.0001, 32.0), + )), + prepare_state_phase_seconds: registry.register(metric!( + name: "mz_storage_collections_prepare_state_phase_seconds", + help: "The time spent in each phase of a single \ + StorageCollections::prepare_state call.", + var_labels: ["phase"], + buckets: histogram_seconds_buckets(0.000_01, 32.0), + )), + txns_upper_advances: registry.register(metric!( + name: "mz_storage_collections_txns_upper_advances_total", + help: "Count of observed advances of the txns shard upper, as \ + observed by the StorageCollections BackgroundTask.", + )), + txns_since_sweeps: registry.register(metric!( + name: "mz_storage_collections_txns_since_sweeps_total", + help: "Count of periodic since-downgrade sweeps over txns-backed \ + collections performed by the StorageCollections BackgroundTask.", + )), } } } diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index 60fd4da7aff32..2ca783a007134 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -746,9 +746,14 @@ impl StorageController for Controller { mut collections: Vec<(GlobalId, CollectionDescription)>, migrated_storage_collections: &BTreeSet, ) -> Result<(), StorageError> { + let phase_metric = self.metrics.create_collections_phase_seconds.clone(); + self.migrated_storage_collections .extend(migrated_storage_collections.iter().cloned()); + let _sc_timer = phase_metric + .with_label_values(&["storage_collections_call"]) + .start_timer(); self.storage_collections .create_collections_for_bootstrap( storage_metadata, @@ -757,11 +762,15 @@ impl StorageController for Controller { migrated_storage_collections, ) .await?; + drop(_sc_timer); // At this point we're connected to all the collection shards in persist. Our warming task // is no longer useful, so abort it if it's still running. drop(self.persist_warm_task.take()); + let _validate_timer = phase_metric + .with_label_values(&["validate_and_enrich"]) + .start_timer(); // Validate first, to avoid corrupting state. // 1. create a dropped identifier, or // 2. create an existing identifier with a new description. @@ -798,6 +807,10 @@ impl StorageController for Controller { }) .collect_vec(); + drop(_validate_timer); + let _open_client_timer = phase_metric + .with_label_values(&["open_persist_client"]) + .start_timer(); // So that we can open persist handles for each collections concurrently. let persist_client = self .persist @@ -805,6 +818,10 @@ impl StorageController for Controller { .await .unwrap(); let persist_client = &persist_client; + drop(_open_client_timer); + let _open_handles_timer = phase_metric + .with_label_values(&["open_data_handles_concurrent"]) + .start_timer(); // Reborrow the `&mut self` as immutable, as all the concurrent work to be processed in // this stream cannot all have exclusive access. @@ -847,6 +864,10 @@ impl StorageController for Controller { // `collect`! .try_collect() .await?; + drop(_open_handles_timer); + let _register_loop_timer = phase_metric + .with_label_values(&["register_loop"]) + .start_timer(); // The set of collections that we should render at the end of this // function. @@ -1121,6 +1142,10 @@ impl StorageController for Controller { self.collections.insert(id, collection_state); } + drop(_register_loop_timer); + let _stats_timer = phase_metric + .with_label_values(&["init_source_statistics"]) + .start_timer(); { let mut source_statistics = self.source_statistics.lock().expect("poisoned"); @@ -1134,6 +1159,10 @@ impl StorageController for Controller { // there is a replica that is reporting them. No need to initialize // here. } + drop(_stats_timer); + let _table_register_timer = phase_metric + .with_label_values(&["table_register"]) + .start_timer(); // Register the tables all in one batch. if !table_registers.is_empty() { @@ -1164,8 +1193,16 @@ impl StorageController for Controller { .expect("table worker unexpectedly shut down"); } } + drop(_table_register_timer); + let _append_mapping_timer = phase_metric + .with_label_values(&["append_shard_mappings"]) + .start_timer(); self.append_shard_mappings(new_collections.into_iter(), Diff::ONE); + drop(_append_mapping_timer); + let _execute_timer = phase_metric + .with_label_values(&["run_to_execute"]) + .start_timer(); // TODO(guswynn): perform the io in this final section concurrently. for id in to_execute { @@ -3236,6 +3273,7 @@ where /// a managed collection. /// - If diff is any value other than `1` or `-1`. #[instrument(level = "debug")] + #[instrument(level = "debug")] fn append_shard_mappings(&self, global_ids: I, diff: Diff) where I: Iterator, diff --git a/src/storage-controller/src/persist_handles.rs b/src/storage-controller/src/persist_handles.rs index 36757ff2f348a..2d901f247f9ff 100644 --- a/src/storage-controller/src/persist_handles.rs +++ b/src/storage-controller/src/persist_handles.rs @@ -245,19 +245,17 @@ impl PersistTableWriteWorker { updates: Vec<(GlobalId, Vec)>, ) -> tokio::sync::oneshot::Receiver> { let (tx, rx) = tokio::sync::oneshot::channel(); - if updates.is_empty() { - tx.send(Ok(())) - .expect("rx has not been dropped at this point"); - rx - } else { - self.send(PersistTableWriteCmd::Append { - write_ts, - advance_to, - updates, - tx, - }); - rx - } + // Always send the append command to the txn-wal layer, even for empty + // updates. The txn-wal commit advances the logical upper of ALL + // registered data shards, which is needed for periodic group commits + // that have no actual data writes. + self.send(PersistTableWriteCmd::Append { + write_ts, + advance_to, + updates, + tx, + }); + rx } /// Drops the handles associated with `ids` from this worker. diff --git a/test/cluster-spec-sheet/README.md b/test/cluster-spec-sheet/README.md index c5f5fa6b8de25..7dbfd37c4f47e 100644 --- a/test/cluster-spec-sheet/README.md +++ b/test/cluster-spec-sheet/README.md @@ -41,24 +41,81 @@ bin/mzcompose --find cluster-spec-sheet run default --target=docker ## Scenarios -There are two kinds of scenarios: +There are four kinds of scenarios: - cluster scaling: These measure run times and arrangement sizes. -- envd scaling: These measure QPS. +- envd qps scalability: These measure QPS while varying envd's CPU allocation. +- envd objects scalability: These measure adapter/envd latency (DDL, simple peeks) as the number of catalog objects grows. +- cluster object limits: These find the maximum number of idle indexes / materialized views a cluster can hold while remaining fresh. -Currently, the envd scaling scenarios can't be run in Production, because changing envd's CPU cores using `mz` is not allowed there. Therefore, these scenarios need to be run with `--target=cloud-staging`. +The envd qps scalability and cluster object limits scenarios can't be run in Production: the former because changing envd's CPU cores using `mz` is not allowed there, the latter because we don't want to burn production resources on long object-limit searches. Both need to be run with `--target=cloud-staging` (or `--target=docker`). You can invoke only one kind of scenarios by using the group name from `SCENARIO_GROUPS`. For example: ``` -bin/mzcompose --find cluster-spec-sheet run default environmentd --target=cloud-staging +bin/mzcompose --find cluster-spec-sheet run default envd_qps_scalability --target=cloud-staging ``` or ``` bin/mzcompose --find cluster-spec-sheet run default cluster ``` +or +``` +bin/mzcompose --find cluster-spec-sheet run default envd_objects_scalability +``` +or +``` +bin/mzcompose --find cluster-spec-sheet run default cluster_object_limits --target=cloud-staging +``` You can also specify a specific scenario by name. For testing just the scaffolding of the cluster spec sheet itself, you can make the run much faster by using the various scaling options, e.g.: ``` ---scale-tpch=0.01 --scale-tpch-queries=0.01 --scale-auction=1 --max-scale=4 +--scale-tpch=0.01 --scale-tpch-queries=0.01 --scale-auction=1 --max-scale=4 --envd-objects-scalability-sizes=1,10,100 --cluster-object-limits-max=500 ``` + +### envd objects scalability scenarios + +The `envd_objects_scalability_tables` and `envd_objects_scalability_mvs` +scenarios fix the measurement cluster size and vary the number of pre-existing +catalog objects, measuring `CREATE TABLE` and `SELECT` latency at each size +point. By default they walk the full size list (`1, 10, 100, 1000, 3000, 5000, +10000, 20000, 30000`); the MV scenario shards across pad clusters at +10000 materialized views per cluster (so 30000 MVs spans 3 single-replica +clusters). Override the size list with `--envd-objects-scalability-sizes`. +These runs are long; expect hours for the full size range, especially the MV +scenario. + +### cluster object limits scenarios + +The `cluster_object_limits_indexes` and `cluster_object_limits_mvs` scenarios +find, per cluster size, how many idle materializations (indexes or +materialized views) one cluster can keep fresh. Each materialization is +derived from a one-row base table that is never updated, so the only work the +cluster has to do is keep advancing each materialization's `write_frontier` +in step with the table's frontier. Once the cluster can't keep up, freshness +collapses; we record the largest N at which freshness was still good. + +Procedure (per cluster size in `REPLICA_SCALES`, capped by `--max-scale`): +1. (Re)create the test cluster `c` at that size, plus a one-row base table. +2. Walk an N-list, adding objects incrementally and querying + `mz_internal.mz_materialization_lag` for max `local_lag` across the test + objects on `c`. +3. Declare healthy if all N objects are reporting and max lag is below + `CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS` (default 2 s). Stop walking N on + the first unhealthy point (the unhealthy point is still recorded so the + cliff is visible). +4. Bisect the (last_healthy, first_unhealthy) interval + `--cluster-object-limits-bisect-steps` times (default 4), adding or + dropping objects in place each step. Each bisection probe halves the + uncertainty around the cliff. Set to 0 to disable. + +The default N-list ramps geometrically up to 1000, then steps in +1000 +increments up to `--cluster-object-limits-max` (default 30000). Override +explicitly with `--cluster-object-limits-sizes=...`. These runs are long; +expect tens of minutes per cluster size on small replicas, longer on bigger +ones. + +Results go to `*.cluster_object_limits.csv` (one row per (cluster_size, N) +sample, with an extra `healthy` column). The analysis produces two plots per +scenario: max-healthy-N per cluster size, and freshness lag vs N with one +series per cluster size. diff --git a/test/cluster-spec-sheet/mzcompose.py b/test/cluster-spec-sheet/mzcompose.py index fe6a27c37ed4f..65d67f8e1fd3d 100644 --- a/test/cluster-spec-sheet/mzcompose.py +++ b/test/cluster-spec-sheet/mzcompose.py @@ -28,6 +28,7 @@ import pandas as pd import psycopg from psycopg import InterfaceError, OperationalError +from psycopg import sql as psycopg_sql from materialize import MZ_ROOT, buildkite from materialize.mz_version import MzVersion @@ -38,6 +39,7 @@ ) from materialize.mzcompose.service import Service as MzComposeService from materialize.mzcompose.services.materialized import Materialized +from materialize.mzcompose.services.metadata_store import metadata_store_services from materialize.mzcompose.services.mz import Mz from materialize.test_analytics.config.test_analytics_db_config import ( create_test_analytics_config, @@ -65,6 +67,14 @@ MATERIALIZED_ADDITIONAL_SYSTEM_PARAMETER_DEFAULTS = { "memory_limiter_interval": "0s", "max_credit_consumption_rate": "1024", + # Lifted for the envd_objects_scalability scenarios, which can build up to 100k + # tables or materialized views in a single schema, plus 10 pad clusters + # (10000 MVs/cluster). Other scenarios are unaffected by these higher + # ceilings. Cloud targets need the same limits configured server-side. + "max_tables": "200000", + "max_materialized_views": "200000", + "max_objects_per_schema": "200000", + "max_clusters": "50", } @@ -75,6 +85,7 @@ def staging_version() -> str: SERVICES = [ # Overridden below Mz(app_password=""), + *metadata_store_services(), Materialized( propagate_crashes=True, additional_system_parameter_defaults=MATERIALIZED_ADDITIONAL_SYSTEM_PARAMETER_DEFAULTS, @@ -97,6 +108,10 @@ def staging_version() -> str: SCENARIO_SOURCE_INGESTION_STRONG = "source_ingestion_strong" SCENARIO_QPS_ENVD_STRONG_SCALING = "qps_envd_strong_scaling" SCENARIO_COPY_FROM_STDIN_ENVD_STRONG_SCALING = "copy_from_stdin_envd_strong_scaling" +SCENARIO_ENVD_OBJECTS_SCALABILITY_TABLES = "envd_objects_scalability_tables" +SCENARIO_ENVD_OBJECTS_SCALABILITY_MVS = "envd_objects_scalability_mvs" +SCENARIO_CLUSTER_OBJECT_LIMITS_INDEXES = "cluster_object_limits_indexes" +SCENARIO_CLUSTER_OBJECT_LIMITS_MVS = "cluster_object_limits_mvs" SCENARIOS_CLUSTERD = [ SCENARIO_AUCTION_STRONG, @@ -118,22 +133,93 @@ def staging_version() -> str: SCENARIOS_SOURCE_INGESTION = [ SCENARIO_SOURCE_INGESTION_STRONG, ] -SCENARIOS_ENVIRONMENTD = [ +SCENARIOS_ENVD_QPS_SCALABILITY = [ SCENARIO_QPS_ENVD_STRONG_SCALING, SCENARIO_COPY_FROM_STDIN_ENVD_STRONG_SCALING, ] -ALL_SCENARIOS = SCENARIOS_CLUSTERD + SCENARIOS_ENVIRONMENTD +SCENARIOS_ENVD_OBJECTS_SCALABILITY = [ + SCENARIO_ENVD_OBJECTS_SCALABILITY_TABLES, + SCENARIO_ENVD_OBJECTS_SCALABILITY_MVS, +] +SCENARIOS_CLUSTER_OBJECT_LIMITS = [ + SCENARIO_CLUSTER_OBJECT_LIMITS_INDEXES, + SCENARIO_CLUSTER_OBJECT_LIMITS_MVS, +] +ALL_SCENARIOS = ( + SCENARIOS_CLUSTERD + + SCENARIOS_ENVD_QPS_SCALABILITY + + SCENARIOS_ENVD_OBJECTS_SCALABILITY + + SCENARIOS_CLUSTER_OBJECT_LIMITS +) SCENARIO_GROUPS = { "cluster": SCENARIOS_CLUSTERD, "cluster_compute": SCENARIOS_CLUSTERD_COMPUTE, "source_ingestion": SCENARIOS_SOURCE_INGESTION, - "environmentd": SCENARIOS_ENVIRONMENTD, + "envd_qps_scalability": SCENARIOS_ENVD_QPS_SCALABILITY, + "envd_objects_scalability": SCENARIOS_ENVD_OBJECTS_SCALABILITY, + "cluster_object_limits": SCENARIOS_CLUSTER_OBJECT_LIMITS, "all": ALL_SCENARIOS, } REPLICA_SCALES = [1, 2, 4, 8, 16, 32] +ENVD_OBJECTS_SCALABILITY_SIZES = [ + 1, + 10, + 100, + 1_000, + 3_000, + 5_000, + 10_000, + 20_000, + 30_000, +] +ENVD_OBJECTS_SCALABILITY_MVS_PER_CLUSTER = 10_000 + +# Default N-walk for the cluster_object_limits scenarios: geometric up to 1k, +# then +1k increments. The cap is configurable via --cluster-object-limits-max +# (default mirrored in `default_cluster_object_limits_sizes()`). +CLUSTER_OBJECT_LIMITS_DEFAULT_MAX = 30_000 +CLUSTER_OBJECT_LIMITS_GEOMETRIC_HEAD = [100, 200, 500, 1_000] +CLUSTER_OBJECT_LIMITS_LINEAR_STEP = 1_000 + +# Freshness probe knobs. Healthy = max local lag below threshold. +CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS = 2_000 +# Ceiling applied to recorded lag values when a materialization has +# completely stalled (e.g. its write_frontier never advanced past the +# minimum timestamp, yielding ~unix-epoch-ms readings). Without a cap, those +# values dwarf the healthy-band data and make plots unreadable. The +# `healthy` column preserves the underlying truth, and the analysis labels +# the plot to make the cap explicit. Set to 10x the threshold. +CLUSTER_OBJECT_LIMITS_LAG_CAP_MS = 10 * CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS +# How long to wait, after a batch of objects has been created, for the +# materializations to hydrate / for lag to fall below threshold before we +# declare "unhealthy". +CLUSTER_OBJECT_LIMITS_HYDRATION_TIMEOUT_S = 60 +# Steady-state sampling window after hydration: take this many samples spaced +# CLUSTER_OBJECT_LIMITS_SAMPLE_INTERVAL_S apart and use the max as the +# representative lag. +CLUSTER_OBJECT_LIMITS_SAMPLES = 5 +CLUSTER_OBJECT_LIMITS_SAMPLE_INTERVAL_S = 2 +# After the coarse N-walk locates the first unhealthy N for a given cluster +# size, bisect the (last_healthy, first_unhealthy) interval this many times to +# narrow the cliff. With a coarse +1k step, 4 bisection steps narrow the cliff +# to ±~60. Set to 0 to disable. +CLUSTER_OBJECT_LIMITS_BISECT_STEPS = 4 + + +def default_cluster_object_limits_sizes(max_n: int) -> list[int]: + """Geometric ramp up to 1k, then +1k increments up to ``max_n``.""" + sizes = [n for n in CLUSTER_OBJECT_LIMITS_GEOMETRIC_HEAD if n <= max_n] + next_n = ( + CLUSTER_OBJECT_LIMITS_GEOMETRIC_HEAD[-1] if sizes else 0 + ) + CLUSTER_OBJECT_LIMITS_LINEAR_STEP + while next_n <= max_n: + sizes.append(next_n) + next_n += CLUSTER_OBJECT_LIMITS_LINEAR_STEP + return sizes + class ConnectionHandler: def __init__(self, new_connection: Callable[[], psycopg.Connection]) -> None: @@ -2186,6 +2272,446 @@ def run(self, runner: ScenarioRunner) -> None: ) +class EnvdObjectsScalabilityScenario(Scenario): + """ + Base class for envd_objects_scalability scenarios. These vary the number of catalog + objects and measure adapter/envd latency for a DDL and a simple peek. + + The scale point (number of catalog objects, N) is driven externally by + `run_scenario_envd_objects_scalability`. The same `runner` is reused across all N, + with `runner.scale` updated per step so that result rows carry N. + """ + + VERSION: str = "1.0.0" + REPETITIONS: int = 10 + + PAD_SCHEMA: str = "pad_schema" + + def __init__(self, replica_size: str | None) -> None: + super().__init__(scale=0, replica_size=replica_size) + self._current_n = 0 + + def name(self) -> str: + raise NotImplementedError + + def materialize_views(self) -> list[str]: + return [] + + # The framework-level `setup()` / `drop()` are unused here; everything + # specific to envd_objects_scalability happens in `init()` / `add_objects()` / + # `teardown()`, which are driven by `run_scenario_envd_objects_scalability`. + def setup(self) -> list[str]: + return [] + + def drop(self) -> list[str]: + return [] + + def init(self, runner: ScenarioRunner) -> None: + # Wipe any leftover pad schema from a previous run. + runner.run_query(f"DROP SCHEMA IF EXISTS {self.PAD_SCHEMA} CASCADE") + runner.run_query(f"CREATE SCHEMA {self.PAD_SCHEMA}") + + def add_objects(self, runner: ScenarioRunner, target_n: int) -> None: + raise NotImplementedError + + def teardown(self, runner: ScenarioRunner) -> None: + # Best-effort cleanup; if it fails, the next run's init() will retry. + try: + runner.run_query(f"DROP SCHEMA IF EXISTS {self.PAD_SCHEMA} CASCADE") + except Exception as e: + print(f"WARNING: failed to drop {self.PAD_SCHEMA}: {e}") + + def run(self, runner: ScenarioRunner) -> None: + # Measure DDL (CREATE TABLE) latency. Each repetition creates a fresh + # `m_tmp` table and drops it afterwards; only the CREATE is timed. + runner.measure( + "adapter_ddl_latency", + "create_table", + setup=["DROP TABLE IF EXISTS m_tmp CASCADE"], + query=["CREATE TABLE m_tmp (a int)"], + after=["DROP TABLE m_tmp"], + repetitions=self.REPETITIONS, + ) + + # Measure simple peek latency against a 1-row table on the fixed + # measurement cluster `c`. + runner.measure( + "adapter_peek_latency", + "select_one_row", + setup=[], + query=["SELECT * FROM t"], + repetitions=self.REPETITIONS, + ) + + +def _bulk_run(runner: ScenarioRunner, statements: list[str], log_label: str) -> None: + """Execute a list of DDL statements with quiet progress logging. + + Statements run one-by-one: this work is dominated by adapter/controller + serialisation, so per-statement round trips are not the bottleneck. + + Callers must pass idempotent statements (``CREATE … IF NOT EXISTS`` / + ``DROP … IF EXISTS``): the underlying ``ConnectionHandler.retryable`` + reconnects and retries on transient errors (e.g. TLS EOF against staging), + and the server may have already committed the original statement before + losing the response. Non-idempotent retries then fail with + "already exists" / "not found" and abort the run. + """ + total = len(statements) + if total == 0: + return + print(f" {log_label}: {total} statements") + log_every = max(1, total // 10) + + def execute_one(stmt: str) -> None: + with runner.connection as cur: + cur.execute(stmt.encode()) + + start = time.time() + for i, stmt in enumerate(statements, 1): + runner.connection.retryable(lambda s=stmt: execute_one(s)) + if i % log_every == 0 or i == total: + elapsed = time.time() - start + rate = i / elapsed if elapsed > 0 else 0 + print(f" {log_label}: {i}/{total} ({rate:.1f}/s)") + + +class EnvdObjectsScalabilityTablesScenario(EnvdObjectsScalabilityScenario): + """N empty tables in the catalog. No controller load; pure catalog/adapter.""" + + def name(self) -> str: + return SCENARIO_ENVD_OBJECTS_SCALABILITY_TABLES + + def add_objects(self, runner: ScenarioRunner, target_n: int) -> None: + if target_n <= self._current_n: + return + statements = [ + f"CREATE TABLE IF NOT EXISTS {self.PAD_SCHEMA}.pad_t_{i} (a int, b text)" + for i in range(self._current_n + 1, target_n + 1) + ] + _bulk_run( + runner, + statements, + f"create tables {self._current_n + 1}..{target_n}", + ) + self._current_n = target_n + + +class EnvdObjectsScalabilityMvsScenario(EnvdObjectsScalabilityScenario): + """N materialized views in the catalog, sharded across pad clusters. + + To bound per-cluster dataflow count, MVs are spread across multiple + single-replica pad clusters (`pad_c_0`, `pad_c_1`, ...), at most + `ENVD_OBJECTS_SCALABILITY_MVS_PER_CLUSTER` MVs per cluster. + + Each MV is a trivial transformation of a single 1-row base table, with + the predicate parameterised on the index `i` so MVs are structurally + distinct (and not collapsible via plan caching). + """ + + MVS_PER_CLUSTER: int = ENVD_OBJECTS_SCALABILITY_MVS_PER_CLUSTER + PAD_BASE: str = "pad_base" + + def __init__(self, replica_size: str | None, pad_replica_size: str) -> None: + super().__init__(replica_size) + self._pad_replica_size = pad_replica_size + self._pad_clusters_created = 0 + + def name(self) -> str: + return SCENARIO_ENVD_OBJECTS_SCALABILITY_MVS + + def init(self, runner: ScenarioRunner) -> None: + # Drop any leftover pad clusters from a previous failed run. + # `%%` escapes the LIKE wildcard from psycopg's `%` placeholder syntax. + existing = runner.run_query( + "SELECT name FROM mz_clusters WHERE name LIKE 'pad_c_%%'", + fetch=True, + ) + for row in existing or []: + runner.run_query(f"DROP CLUSTER IF EXISTS {row[0]} CASCADE") + + super().init(runner) + runner.run_query( + f"CREATE TABLE {self.PAD_SCHEMA}.{self.PAD_BASE} (id int, val text)" + ) + runner.run_query( + f"INSERT INTO {self.PAD_SCHEMA}.{self.PAD_BASE} VALUES (1, 'x')" + ) + + def _ensure_pad_cluster(self, runner: ScenarioRunner, cluster_idx: int) -> None: + while self._pad_clusters_created <= cluster_idx: + k = self._pad_clusters_created + print(f" creating pad cluster pad_c_{k} (size {self._pad_replica_size})") + runner.run_query(f"DROP CLUSTER IF EXISTS pad_c_{k} CASCADE") + runner.run_query( + f"CREATE CLUSTER pad_c_{k} SIZE '{self._pad_replica_size}'" + ) + self._pad_clusters_created += 1 + + def add_objects(self, runner: ScenarioRunner, target_n: int) -> None: + if target_n <= self._current_n: + return + + next_i = self._current_n + 1 + while next_i <= target_n: + cluster_idx = (next_i - 1) // self.MVS_PER_CLUSTER + self._ensure_pad_cluster(runner, cluster_idx) + cluster_end = min(target_n, (cluster_idx + 1) * self.MVS_PER_CLUSTER) + statements = [ + f"CREATE MATERIALIZED VIEW IF NOT EXISTS {self.PAD_SCHEMA}.pad_mv_{i} " + f"IN CLUSTER pad_c_{cluster_idx} " + f"AS SELECT id, val FROM {self.PAD_SCHEMA}.{self.PAD_BASE} " + f"WHERE id < {i}" + for i in range(next_i, cluster_end + 1) + ] + _bulk_run( + runner, + statements, + f"create MVs {next_i}..{cluster_end} on pad_c_{cluster_idx}", + ) + next_i = cluster_end + 1 + + self._current_n = target_n + + def teardown(self, runner: ScenarioRunner) -> None: + # Drop the schema first so MVs (catalog entries) are removed; then + # drop the now-quiescent pad clusters. + super().teardown(runner) + for k in range(self._pad_clusters_created): + try: + runner.run_query(f"DROP CLUSTER IF EXISTS pad_c_{k} CASCADE") + except Exception as e: + print(f"WARNING: failed to drop pad_c_{k}: {e}") + self._pad_clusters_created = 0 + + +class ClusterObjectLimitsScenario(Scenario): + """ + Base class for cluster_object_limits scenarios. These find the maximum + number of idle materializations (indexes or materialized views) one can + place on a single cluster, while still preserving freshness. + + Structure: + - A single one-row base table `base_t` (never updated) lives in + ``PAD_SCHEMA``. Frontiers advance over wall-clock time, so even with + no writes the cluster must keep ticking every materialization's + write_frontier — which is what saturates an undersized cluster. + - N structurally-distinct materializations on the measurement cluster + ``c``: a view-plus-default-index for the index scenario, or an MV for + the MV scenario, all derived from ``base_t``. + - Per cluster-size iteration we drop+recreate ``c`` and ``PAD_SCHEMA`` + and then walk an N-list, adding the delta of objects at each step and + probing freshness via ``mz_internal.mz_materialization_lag``. + + Driven by ``run_scenario_cluster_object_limits``. + """ + + VERSION: str = "1.0.0" + + PAD_SCHEMA: str = "pad_schema" + BASE_TABLE: str = "base_t" + # Cluster used to probe `mz_materialization_lag`. Built-in catalog server + # cluster, so the probe does not add load to `c`. + PROBE_CLUSTER: str = "mz_catalog_server" + + def __init__(self, replica_size: str | None = None) -> None: + super().__init__(scale=0, replica_size=replica_size) + self._current_n = 0 + + def name(self) -> str: + raise NotImplementedError + + # The framework-level `setup()` / `drop()` are unused here; lifecycle is + # driven by `run_scenario_cluster_object_limits`. + def setup(self) -> list[str]: + return [] + + def drop(self) -> list[str]: + return [] + + def materialize_views(self) -> list[str]: + return [] + + def reset_for_cluster_size(self, runner: ScenarioRunner) -> None: + """Drop+recreate the pad schema and the one-row base table. + + Called once per cluster-size iteration (after recreating cluster ``c``) + so each cluster-size run starts from a clean slate. The cluster drop + cascades to remove on-cluster objects (indexes/MVs); dropping the + schema removes the off-cluster views. + """ + runner.run_query(f"DROP SCHEMA IF EXISTS {self.PAD_SCHEMA} CASCADE") + runner.run_query(f"CREATE SCHEMA {self.PAD_SCHEMA}") + runner.run_query( + f"CREATE TABLE {self.PAD_SCHEMA}.{self.BASE_TABLE} (id int, val text)" + ) + runner.run_query( + f"INSERT INTO {self.PAD_SCHEMA}.{self.BASE_TABLE} VALUES (1, 'x')" + ) + self._current_n = 0 + + def add_objects(self, runner: ScenarioRunner, target_n: int) -> None: + raise NotImplementedError + + def remove_objects(self, runner: ScenarioRunner, target_n: int) -> None: + """Drop the test objects above ``target_n`` from cluster ``c``. + + Inverse of ``add_objects``. Needed by the bisection refinement in + ``run_scenario_cluster_object_limits``: after walking forward past the + cliff, we step back down to probe intermediate N without rebuilding + the catalog from scratch. + """ + raise NotImplementedError + + def lag_filter(self) -> str: + """SQL fragment that filters `mz_materialization_lag` to our test + objects on cluster `c`. Subclass-specific because the cluster_id is + carried on `mz_indexes` vs `mz_materialized_views`.""" + raise NotImplementedError + + def probe_lag_ms(self, runner: ScenarioRunner) -> tuple[int, int, float]: + """Return (total, reporting, max_local_lag_ms) for our test objects. + + - `total` is the number of test objects on cluster `c` we expect to + see (sanity check against `_current_n`). + - `reporting` is the subset that has a non-NULL `local_lag`. + - `max_local_lag_ms` is the max over reporting objects; 0 if none yet. + + NULL lag means the object has not yet hydrated / reported. We treat + unreported objects as "not healthy" via the caller. + """ + rows = runner.run_query( + f""" + SELECT + count(*), + count(local_lag), + COALESCE( + max(EXTRACT(EPOCH FROM local_lag) * 1000), + 0 + )::float8 + FROM mz_internal.mz_materialization_lag l + {self.lag_filter()} + """, + fetch=True, + ) + assert rows is not None and len(rows) == 1 + total, reporting, max_lag_ms = rows[0] + return int(total), int(reporting), float(max_lag_ms) + + def teardown(self, runner: ScenarioRunner) -> None: + try: + runner.run_query(f"DROP SCHEMA IF EXISTS {self.PAD_SCHEMA} CASCADE") + except Exception as e: + print(f"WARNING: failed to drop {self.PAD_SCHEMA}: {e}") + try: + runner.run_query("DROP CLUSTER IF EXISTS c CASCADE") + except Exception as e: + print(f"WARNING: failed to drop cluster c: {e}") + + def run(self, runner: ScenarioRunner) -> None: + # The freshness probe is driven externally by + # `run_scenario_cluster_object_limits`; nothing else to measure here. + return + + +class ClusterObjectLimitsIndexesScenario(ClusterObjectLimitsScenario): + """N indexed views on cluster `c`, each derived from a one-row base table.""" + + def name(self) -> str: + return SCENARIO_CLUSTER_OBJECT_LIMITS_INDEXES + + def add_objects(self, runner: ScenarioRunner, target_n: int) -> None: + if target_n <= self._current_n: + return + # Each (view, default index) pair is one independent index dataflow on + # `c`. WHERE id < {i} makes the views structurally distinct (avoids + # plan caching collapse). + statements: list[str] = [] + for i in range(self._current_n + 1, target_n + 1): + statements.append( + f"CREATE VIEW IF NOT EXISTS {self.PAD_SCHEMA}.v_{i} AS " + f"SELECT id, val FROM {self.PAD_SCHEMA}.{self.BASE_TABLE} " + f"WHERE id < {i}" + ) + statements.append( + f"CREATE DEFAULT INDEX IF NOT EXISTS " + f"IN CLUSTER c ON {self.PAD_SCHEMA}.v_{i}" + ) + _bulk_run( + runner, + statements, + f"create indexed views {self._current_n + 1}..{target_n}", + ) + self._current_n = target_n + + def remove_objects(self, runner: ScenarioRunner, target_n: int) -> None: + if target_n >= self._current_n: + return + # DROP VIEW CASCADE removes the dependent default index too. + statements = [ + f"DROP VIEW IF EXISTS {self.PAD_SCHEMA}.v_{i} CASCADE" + for i in range(target_n + 1, self._current_n + 1) + ] + _bulk_run( + runner, + statements, + f"drop indexed views {target_n + 1}..{self._current_n}", + ) + self._current_n = target_n + + def lag_filter(self) -> str: + return """ + JOIN mz_catalog.mz_indexes idx ON l.object_id = idx.id + JOIN mz_catalog.mz_clusters c ON idx.cluster_id = c.id + WHERE c.name = 'c' + """ + + +class ClusterObjectLimitsMvsScenario(ClusterObjectLimitsScenario): + """N materialized views on cluster `c`, each derived from a one-row base table.""" + + def name(self) -> str: + return SCENARIO_CLUSTER_OBJECT_LIMITS_MVS + + def add_objects(self, runner: ScenarioRunner, target_n: int) -> None: + if target_n <= self._current_n: + return + statements = [ + f"CREATE MATERIALIZED VIEW IF NOT EXISTS {self.PAD_SCHEMA}.mv_{i} " + f"IN CLUSTER c " + f"AS SELECT id, val FROM {self.PAD_SCHEMA}.{self.BASE_TABLE} " + f"WHERE id < {i}" + for i in range(self._current_n + 1, target_n + 1) + ] + _bulk_run( + runner, + statements, + f"create MVs {self._current_n + 1}..{target_n}", + ) + self._current_n = target_n + + def remove_objects(self, runner: ScenarioRunner, target_n: int) -> None: + if target_n >= self._current_n: + return + statements = [ + f"DROP MATERIALIZED VIEW IF EXISTS {self.PAD_SCHEMA}.mv_{i}" + for i in range(target_n + 1, self._current_n + 1) + ] + _bulk_run( + runner, + statements, + f"drop MVs {target_n + 1}..{self._current_n}", + ) + self._current_n = target_n + + def lag_filter(self) -> str: + return """ + JOIN mz_catalog.mz_materialized_views mv ON l.object_id = mv.id + JOIN mz_catalog.mz_clusters c ON mv.cluster_id = c.id + WHERE c.name = 'c' + """ + + # TODO: We should factor out the below # `disable_region`, `cloud_disable_enable_and_wait`, `reconfigure_envd_cpus`, `wait_for_envd` # functions into a separate module. (Similar `disable_region` functions also occur in other tests.) @@ -2346,6 +2872,59 @@ def wait_for_envd(target: "BenchTarget", timeout_secs: int = 300) -> None: ) +# System parameters we expect to be lifted server-side on cloud targets (via +# LaunchDarkly defaults or equivalent). Local Docker sets these directly via +# `MATERIALIZED_ADDITIONAL_SYSTEM_PARAMETER_DEFAULTS`. We log them at workflow +# start so we can verify the environment we're actually talking to matches what +# we configured. +SYSTEM_PARAMETERS_TO_LOG: list[str] = [ + "max_tables", + "max_materialized_views", + "max_objects_per_schema", + "max_clusters", + "max_credit_consumption_rate", + "memory_limiter_interval", +] + + +def log_environment_info(target: "BenchTarget") -> None: + """Print the environment id and key system parameters. + + Best-effort: any error is logged and swallowed so a transient probe + failure does not abort the workflow. + """ + print("--- Environment info") + try: + conn = target.new_connection() + except Exception as e: + print(f" WARNING: could not open connection to log environment info: {e}") + return + try: + with conn.cursor() as cur: + try: + cur.execute("SELECT mz_environment_id()") + row = cur.fetchone() + env_id = row[0] if row else "" + print(f" mz_environment_id() = {env_id}") + except Exception as e: + print(f" WARNING: failed to read mz_environment_id(): {e}") + for param in SYSTEM_PARAMETERS_TO_LOG: + try: + cur.execute( + psycopg_sql.SQL("SHOW {}").format(psycopg_sql.Identifier(param)) + ) + row = cur.fetchone() + val = row[0] if row else "" + print(f" {param} = {val}") + except Exception as e: + print(f" WARNING: failed to SHOW {param}: {e}") + finally: + try: + conn.close() + except Exception: + pass + + def workflow_default(composition: Composition, parser: WorkflowArgumentParser) -> None: """ Run the bench workflow by default @@ -2388,11 +2967,53 @@ def workflow_default(composition: Composition, parser: WorkflowArgumentParser) - parser.add_argument( "--scale-auction", type=int, default=3, help="Auction scale factor." ) + parser.add_argument( + "--envd-objects-scalability-sizes", + type=lambda s: [int(x) for x in s.split(",") if x.strip()], + default=ENVD_OBJECTS_SCALABILITY_SIZES, + help=( + "Comma-separated list of catalog object counts to test for " + "envd_objects_scalability scenarios. Defaults to " + f"{','.join(str(n) for n in ENVD_OBJECTS_SCALABILITY_SIZES)}." + ), + ) + parser.add_argument( + "--cluster-object-limits-max", + type=int, + default=CLUSTER_OBJECT_LIMITS_DEFAULT_MAX, + help=( + "Upper bound on N for cluster_object_limits scenarios when no " + "explicit --cluster-object-limits-sizes is given. Defaults to " + f"{CLUSTER_OBJECT_LIMITS_DEFAULT_MAX}." + ), + ) + parser.add_argument( + "--cluster-object-limits-sizes", + type=lambda s: [int(x) for x in s.split(",") if x.strip()], + default=None, + help=( + "Comma-separated list of N (object-count) steps for the " + "cluster_object_limits scenarios. If omitted, defaults to a " + "geometric ramp up to 1000 then +1000 increments up to " + "--cluster-object-limits-max." + ), + ) + parser.add_argument( + "--cluster-object-limits-bisect-steps", + type=int, + default=CLUSTER_OBJECT_LIMITS_BISECT_STEPS, + help=( + "After the coarse N-walk locates the cliff for a given cluster " + "size, bisect the (last_healthy, first_unhealthy) interval this " + "many times to narrow it. Set to 0 to disable refinement. " + f"Default: {CLUSTER_OBJECT_LIMITS_BISECT_STEPS}." + ), + ) parser.add_argument( "scenarios", nargs="*", choices=ALL_SCENARIOS + list(SCENARIO_GROUPS.keys()), - help="Scenarios to run, supports individual scenario names as well as 'all', 'cluster', 'environmentd'.", + help="Scenarios to run, supports individual scenario names as well as 'all', 'cluster', 'envd_qps_scalability', 'envd_objects_scalability', 'cluster_object_limits'.", ) args = parser.parse_args() @@ -2409,6 +3030,24 @@ def workflow_default(composition: Composition, parser: WorkflowArgumentParser) - raise ValueError(f"Unknown scenarios: {unknown}") print(f"--- Running scenarios: {', '.join(scenarios)}") + # cluster_object_limits scenarios push the cluster to its idle-object + # limit; we only run them on staging or local Docker to avoid burning + # production resources or hitting production-only catalog limits. + cluster_object_limits_requested = scenarios & set(SCENARIOS_CLUSTER_OBJECT_LIMITS) + if cluster_object_limits_requested and args.target == "cloud-production": + raise UIError( + "cluster_object_limits scenarios cannot run against " + "--target=cloud-production; use --target=cloud-staging or " + "--target=docker. Requested scenarios: " + f"{', '.join(sorted(cluster_object_limits_requested))}." + ) + + cluster_object_limits_sizes = ( + args.cluster_object_limits_sizes + if args.cluster_object_limits_sizes is not None + else default_cluster_object_limits_sizes(args.cluster_object_limits_max) + ) + if args.target == "cloud-production": target: BenchTarget = CloudTarget( composition, PRODUCTION_USERNAME, PRODUCTION_APP_PASSWORD or "" @@ -2447,13 +3086,22 @@ def workflow_default(composition: Composition, parser: WorkflowArgumentParser) - target.initialize() - # Derive two result files (cluster and envd-focused) from the provided --record path + log_environment_info(target) + + # Derive result files (cluster, envd-focused, envd_objects_scalability, + # cluster_object_limits) from the provided --record path. base_name = os.path.splitext(args.record)[0] cluster_path = f"{base_name}.cluster.csv" envd_path = f"{base_name}.envd.csv" + envd_objects_scalability_path = f"{base_name}.envd_objects_scalability.csv" + cluster_object_limits_path = f"{base_name}.cluster_object_limits.csv" cluster_file = open(cluster_path, "w", newline="") envd_file = open(envd_path, "w", newline="") + envd_objects_scalability_file = open( + envd_objects_scalability_path, "w", newline="" + ) + cluster_object_limits_file = open(cluster_object_limits_path, "w", newline="") # Traditional scenarios: cluster-focused schema cluster_writer = csv.DictWriter( @@ -2492,6 +3140,49 @@ def workflow_default(composition: Composition, parser: WorkflowArgumentParser) - ) envd_writer.writeheader() + # Envd-scalability scenarios reuse the cluster-focused schema: + # `scale` carries the catalog object count (N), `cluster_size` is the + # fixed measurement cluster, `time_ms` is the latency. + envd_objects_scalability_writer = csv.DictWriter( + envd_objects_scalability_file, + fieldnames=[ + "scenario", + "scenario_version", + "scale", + "mode", + "category", + "test_name", + "cluster_size", + "repetition", + "size_bytes", + "time_ms", + ], + extrasaction="ignore", + ) + envd_objects_scalability_writer.writeheader() + + # cluster_object_limits scenarios: per-(cluster_size, N) freshness + # sample. `scale` carries N, `time_ms` carries the max local lag, + # `healthy` is an extra boolean column (1/0). + cluster_object_limits_writer = csv.DictWriter( + cluster_object_limits_file, + fieldnames=[ + "scenario", + "scenario_version", + "scale", + "mode", + "category", + "test_name", + "cluster_size", + "repetition", + "size_bytes", + "time_ms", + "healthy", + ], + extrasaction="ignore", + ) + cluster_object_limits_writer.writeheader() + def process(scenario: str) -> None: with composition.test_case(scenario): conn = ConnectionHandler(target.new_connection) @@ -2603,6 +3294,53 @@ def process(scenario: str) -> None: target=target, max_scale=max_scale, ) + if scenario == SCENARIO_ENVD_OBJECTS_SCALABILITY_TABLES: + print("--- SCENARIO: Running envd scalability (tables)") + run_scenario_envd_objects_scalability( + scenario=EnvdObjectsScalabilityTablesScenario( + target.replica_size_for_scale(1) + ), + results_writer=envd_objects_scalability_writer, + connection=conn, + target=target, + sizes=args.envd_objects_scalability_sizes, + ) + if scenario == SCENARIO_ENVD_OBJECTS_SCALABILITY_MVS: + print("--- SCENARIO: Running envd scalability (materialized views)") + run_scenario_envd_objects_scalability( + scenario=EnvdObjectsScalabilityMvsScenario( + target.replica_size_for_scale(1), + pad_replica_size=target.replica_size_for_scale(2), + ), + results_writer=envd_objects_scalability_writer, + connection=conn, + target=target, + sizes=args.envd_objects_scalability_sizes, + ) + if scenario == SCENARIO_CLUSTER_OBJECT_LIMITS_INDEXES: + print("--- SCENARIO: Running cluster object limits (indexes)") + run_scenario_cluster_object_limits( + scenario=ClusterObjectLimitsIndexesScenario(), + results_writer=cluster_object_limits_writer, + connection=conn, + target=target, + max_scale=max_scale, + sizes=cluster_object_limits_sizes, + bisect_steps=args.cluster_object_limits_bisect_steps, + ) + if scenario == SCENARIO_CLUSTER_OBJECT_LIMITS_MVS: + print( + "--- SCENARIO: Running cluster object limits (materialized views)" + ) + run_scenario_cluster_object_limits( + scenario=ClusterObjectLimitsMvsScenario(), + results_writer=cluster_object_limits_writer, + connection=conn, + target=target, + max_scale=max_scale, + sizes=cluster_object_limits_sizes, + bisect_steps=args.cluster_object_limits_bisect_steps, + ) test_failed = True try: @@ -2612,6 +3350,8 @@ def process(scenario: str) -> None: finally: cluster_file.close() envd_file.close() + envd_objects_scalability_file.close() + cluster_object_limits_file.close() # Clean up if args.cleanup: target.cleanup() @@ -2622,18 +3362,38 @@ def process(scenario: str) -> None: upload_environmentd_results_to_test_analytics( composition, envd_path, not test_failed ) + # The envd_objects_scalability rows share the cluster_spec_sheet_result schema + # (mode='envd_objects_scalability' distinguishes them). + upload_cluster_results_to_test_analytics( + composition, envd_objects_scalability_path, not test_failed + ) + # The cluster_object_limits rows reuse the cluster_spec_sheet_result + # schema as well. The extra `healthy` column is dropped on upload + # (extrasaction="ignore" on the CSV writer); to recover it, consult + # the artifact CSV directly. + upload_cluster_results_to_test_analytics( + composition, cluster_object_limits_path, not test_failed + ) assert not test_failed if buildkite.is_in_buildkite(): - # Upload both CSVs as artifacts + # Upload all CSVs as artifacts buildkite.upload_artifact(cluster_path, cwd=MZ_ROOT, quiet=True) buildkite.upload_artifact(envd_path, cwd=MZ_ROOT, quiet=True) + buildkite.upload_artifact( + envd_objects_scalability_path, cwd=MZ_ROOT, quiet=True + ) + buildkite.upload_artifact( + cluster_object_limits_path, cwd=MZ_ROOT, quiet=True + ) if args.analyze: - # Analyze both files separately (each has its own schema) + # Analyze each file separately (each has its own schema/x-axis) analyze_cluster_results_file(cluster_path) analyze_envd_results_file(envd_path) + analyze_envd_objects_scalability_results_file(envd_objects_scalability_path) + analyze_cluster_object_limits_results_file(cluster_object_limits_path) class BenchTarget: @@ -2938,6 +3698,276 @@ def recreate_cluster() -> None: ) +def run_scenario_envd_objects_scalability( + scenario: EnvdObjectsScalabilityScenario, + results_writer: csv.DictWriter, + connection: ConnectionHandler, + target: BenchTarget, + sizes: list[int], +) -> None: + """ + Run an envd-objects-scalability scenario, where the cluster size is fixed and the + number of catalog objects (`scale`) varies across the points in `sizes`. + + Builds the catalog incrementally so transitioning from N=k to the next + size point only adds (next - k) objects. + """ + measurement_size = target.replica_size_for_scale(1) + + runner = ScenarioRunner( + scenario.name(), + scenario.VERSION, + 0, # filled in per N below + "envd_objects_scalability", + connection, + results_writer, + replica_size=measurement_size, + target=target, + ) + + # (Re)create the fixed-size measurement cluster and a tiny one-row table + # used by the simple-peek measurement. + runner.run_query("DROP CLUSTER IF EXISTS c CASCADE") + runner.run_query(f"CREATE CLUSTER c SIZE '{measurement_size}'") + runner.run_query("SET cluster = 'c'") + runner.run_query("DROP TABLE IF EXISTS t CASCADE") + runner.run_query("CREATE TABLE t (a int)") + runner.run_query("INSERT INTO t VALUES (1)") + + scenario.init(runner) + + try: + for n in sizes: + print( + f"--- envd_objects_scalability {scenario.name()}: building catalog up to N={n}" + ) + scenario.add_objects(runner, n) + runner.scale = n + print(f"--- envd_objects_scalability {scenario.name()}: measuring at N={n}") + scenario.run(runner) + finally: + scenario.teardown(runner) + + +def run_scenario_cluster_object_limits( + scenario: ClusterObjectLimitsScenario, + results_writer: csv.DictWriter, + connection: ConnectionHandler, + target: BenchTarget, + max_scale: int, + sizes: list[int], + lag_threshold_ms: int = CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS, + hydration_timeout_s: int = CLUSTER_OBJECT_LIMITS_HYDRATION_TIMEOUT_S, + samples: int = CLUSTER_OBJECT_LIMITS_SAMPLES, + sample_interval_s: float = CLUSTER_OBJECT_LIMITS_SAMPLE_INTERVAL_S, + bisect_steps: int = CLUSTER_OBJECT_LIMITS_BISECT_STEPS, +) -> None: + """ + For each cluster size in REPLICA_SCALES (capped by `max_scale`), walk the + N-list `sizes`, adding objects incrementally and probing freshness via + `mz_internal.mz_materialization_lag`. Stops walking N for a given cluster + size on the first unhealthy data point (the unhealthy sample is still + recorded so the cliff is visible). Reports max-N-healthy implicitly via + the rows in the CSV. + + If `bisect_steps > 0` and a cliff was located, bisect the + (last_healthy, first_unhealthy) interval that many times, probing each + midpoint to narrow the cliff. Each refinement step adds or drops objects + in place rather than rebuilding the catalog. + """ + + def probe_once(runner: ScenarioRunner) -> tuple[int, int, float]: + """Run the probe on the catalog-server cluster so it doesn't load `c`.""" + # Run the probe via the connection's autocommit cursor on the probe + # cluster. Wrapping in a transaction-local SET would also work but + # autocommit means we'd leak the SET; restore at the end. + runner.run_query(f"SET cluster = '{scenario.PROBE_CLUSTER}'") + try: + return scenario.probe_lag_ms(runner) + finally: + runner.run_query("SET cluster = 'c'") + + def hydrate_and_sample( + runner: ScenarioRunner, target_n: int + ) -> tuple[float, bool, int, int]: + """ + Wait for all N materializations to be reporting & under threshold, + then take `samples` samples and return their max as the + representative lag. + + Returns (max_lag_ms, healthy, reporting, total). `healthy=False` if we + time out waiting for hydration / steady-state, or if any sample + exceeds the threshold. + """ + deadline = time.time() + hydration_timeout_s + last_reporting = 0 + last_total = 0 + last_max_lag = float("inf") + while time.time() < deadline: + total, reporting, max_lag_ms = probe_once(runner) + last_total, last_reporting, last_max_lag = total, reporting, max_lag_ms + # All objects must be reporting AND under threshold to count as + # hydrated. NULL lag => the materialization has not produced its + # first frontier report yet. + if reporting == target_n and max_lag_ms < lag_threshold_ms: + break + time.sleep(min(1.0, sample_interval_s)) + else: + # Did not converge within the hydration window. + return last_max_lag, False, last_reporting, last_total + + # Steady-state sampling: take a few samples and use the max. + max_lag_over_samples = last_max_lag + worst_reporting = last_reporting + for _ in range(samples): + time.sleep(sample_interval_s) + total, reporting, max_lag_ms = probe_once(runner) + last_total = total + worst_reporting = min(worst_reporting, reporting) + max_lag_over_samples = max(max_lag_over_samples, max_lag_ms) + healthy = ( + worst_reporting == target_n and max_lag_over_samples < lag_threshold_ms + ) + return max_lag_over_samples, healthy, worst_reporting, last_total + + # Snapshot of cluster sizes to iterate over. + cluster_scales = [s for s in REPLICA_SCALES if s <= max_scale] + + # Outer loop: cluster size. + for replica_scale in cluster_scales: + replica_size = target.replica_size_for_scale(replica_scale) + print( + f"--- cluster_object_limits {scenario.name()}: cluster size '{replica_size}'" + ) + + runner = ScenarioRunner( + scenario.name(), + scenario.VERSION, + 0, # filled in per N below + "cluster_object_limits", + connection, + results_writer, + replica_size=replica_size, + target=target, + ) + + # (Re)create the test cluster. Skip the iteration if this cluster + # size isn't available on the target — e.g. it isn't in + # `mz_cluster_replica_sizes` for this region, or allocating it would + # exceed `max_credit_consumption_rate`. Without this guard, on + # staging the larger sizes either fail with a noisy traceback or + # show up as a confusing "unhealthy at the smallest N" data point + # instead of a clear "size unavailable" line. We narrow the catch to + # server-side SQL errors and let connection-level OperationalError + # propagate (the in-`run_query` retry loop has already given up). + runner.run_query("DROP CLUSTER IF EXISTS c CASCADE") + try: + runner.run_query(f"CREATE CLUSTER c SIZE '{replica_size}'") + except psycopg.errors.DatabaseError as e: + if isinstance(e, OperationalError): + raise + print( + f"^^^ +++ cluster_object_limits {scenario.name()}: cluster " + f"size '{replica_size}' unavailable on this target " + f"({type(e).__name__}: {str(e).strip()}); skipping." + ) + continue + runner.run_query("SET cluster = 'c'") + + # Reset the base table and pad schema for this cluster-size iteration. + scenario.reset_for_cluster_size(runner) + + def probe_and_record(n: int, refinement: bool = False) -> bool: + runner.scale = n + max_lag_ms, healthy, reporting, total = hydrate_and_sample(runner, n) + # A fully stalled materialization can report `local_lag` as + # `now() - `, i.e. the current unix time in + # ms (~1.78e12). Cap the recorded value so the cliff doesn't + # crush every other data point on the plot. The `healthy` + # column carries the underlying truth, and the analysis + # labels the plot to make the cap explicit. + capped_lag_ms = min(max_lag_ms, CLUSTER_OBJECT_LIMITS_LAG_CAP_MS) + print( + f" {'(bisect) ' if refinement else ''}" + f"N={n}: max_local_lag_ms={max_lag_ms:.1f} " + f"reporting={reporting}/{total} healthy={healthy}" + ) + runner.results_writer.writerow( + { + "scenario": scenario.name(), + "scenario_version": scenario.VERSION, + "scale": n, + "mode": "cluster_object_limits", + "category": "freshness", + "test_name": "max_local_lag_ms", + "cluster_size": replica_size, + "envd_cpus": None, + "repetition": 0, + "size_bytes": None, + "time_ms": int(capped_lag_ms), + "qps": None, + "healthy": 1 if healthy else 0, + } + ) + return healthy + + try: + # Coarse pass: walk the N list, stop at the first unhealthy point. + last_healthy_n = 0 + first_unhealthy_n: int | None = None + for n in sizes: + print( + f"--- cluster_object_limits {scenario.name()}: " + f"size '{replica_size}', building up to N={n}" + ) + scenario.add_objects(runner, n) + print( + f"--- cluster_object_limits {scenario.name()}: " + f"size '{replica_size}', probing freshness at N={n}" + ) + if probe_and_record(n): + last_healthy_n = n + else: + first_unhealthy_n = n + # Found the cliff for this cluster size; skip larger Ns. + print( + f" N={n}: unhealthy, stopping N-walk for size " + f"'{replica_size}'" + ) + break + + # Refinement pass: bisect (last_healthy, first_unhealthy] to + # narrow the cliff. Each step adds or drops objects in place. + if first_unhealthy_n is not None and bisect_steps > 0: + lo, hi = last_healthy_n, first_unhealthy_n + for step in range(bisect_steps): + mid = (lo + hi) // 2 + if mid <= lo: + # Interval already minimal (hi - lo <= 1). + break + print( + f"--- cluster_object_limits {scenario.name()}: " + f"size '{replica_size}', bisect step " + f"{step + 1}/{bisect_steps} at N={mid} " + f"(cliff in ({lo}, {hi}])" + ) + # Exactly one of these does work; the other is a no-op. + scenario.add_objects(runner, mid) + scenario.remove_objects(runner, mid) + if probe_and_record(mid, refinement=True): + lo = mid + else: + hi = mid + print( + f" bisect done for size '{replica_size}': " + f"cliff in ({lo}, {hi}]" + ) + finally: + # Drop the cluster (cascades to indexes/MVs) and the schema (views) + # before moving to the next cluster size, to bound catalog growth. + scenario.teardown(runner) + + def run_scenario_weak( scenario: Scenario, results_writer: csv.DictWriter, @@ -3033,13 +4063,56 @@ def workflow_plot_envd( analyze_envd_results_file(str(file)) +def workflow_plot_envd_objects_scalability( + composition: Composition, parser: WorkflowArgumentParser +) -> None: + """Analyze envd-objects-scalability results (latency vs catalog object count).""" + + parser.add_argument( + "files", + nargs="*", + default="results_*.envd_objects_scalability.csv", + type=str, + help="Glob pattern of envd_objects_scalability result files to plot.", + ) + + args = parser.parse_args() + + for file in itertools.chain(*map(glob.iglob, args.files)): + analyze_envd_objects_scalability_results_file(str(file)) + + +def workflow_plot_cluster_object_limits( + composition: Composition, parser: WorkflowArgumentParser +) -> None: + """Analyze cluster_object_limits results (freshness lag vs N per cluster size).""" + + parser.add_argument( + "files", + nargs="*", + default="results_*.cluster_object_limits.csv", + type=str, + help="Glob pattern of cluster_object_limits result files to plot.", + ) + + args = parser.parse_args() + + for file in itertools.chain(*map(glob.iglob, args.files)): + analyze_cluster_object_limits_results_file(str(file)) + + def workflow_plot(composition: Composition, parser: WorkflowArgumentParser) -> None: """Analyze the results of the workflow.""" parser.add_argument( "files", nargs="*", - default=["results_*.cluster.csv", "results_*.envd.csv"], + default=[ + "results_*.cluster.csv", + "results_*.envd.csv", + "results_*.envd_objects_scalability.csv", + "results_*.cluster_object_limits.csv", + ], type=str, help="Glob pattern of result files to plot.", ) @@ -3049,13 +4122,20 @@ def workflow_plot(composition: Composition, parser: WorkflowArgumentParser) -> N for file in itertools.chain(*map(glob.iglob, args.files)): file_str = str(file) base_name = os.path.basename(file_str) - if base_name.endswith(".cluster.csv"): + # Order matters: `.cluster_object_limits.csv` and `.envd_objects_scalability.csv` + # both happen to end in something that would also match `.cluster` or + # `.envd` if matched naively, so check the longer suffixes first. + if base_name.endswith(".cluster_object_limits.csv"): + analyze_cluster_object_limits_results_file(file_str) + elif base_name.endswith(".envd_objects_scalability.csv"): + analyze_envd_objects_scalability_results_file(file_str) + elif base_name.endswith(".cluster.csv"): analyze_cluster_results_file(file_str) elif base_name.endswith(".envd.csv"): analyze_envd_results_file(file_str) else: raise UIError( - f"Error: Filename '{file_str}' doesn't indicate whether it's a cluster or an envd results file (no .cluster/.envd suffix). Please use the explicit `plot-envd` or `plot-cluster` targets for such files." + f"Error: Filename '{file_str}' doesn't indicate whether it's a cluster, envd, envd_objects_scalability, or cluster_object_limits results file. Please use the explicit `plot-envd`, `plot-cluster`, `plot-envd-objects-scalability`, or `plot-cluster-object-limits` targets for such files." ) @@ -3176,6 +4256,138 @@ def analyze_envd_results_file(file: str) -> None: ) +def analyze_envd_objects_scalability_results_file(file: str) -> None: + """Plot adapter/envd latency vs catalog object count (N). + + Each (scenario, category) pair yields one plot, with `scale` (=N) as the + x-axis (categorical bar chart, since the values span 1..100k). The + accompanying normalized plot is produced by the shared `plot()` helper. + """ + print(f"--- Analyzing envd_objects_scalability results file {file} ...") + df = pd.read_csv(file) + if df.empty: + print(f"^^^ +++ File {file} is empty, skipping") + return + + base_name = os.path.basename(file).split(".")[0] + plot_dir = os.path.join("test", "cluster-spec-sheet", "plots", base_name) + os.makedirs(plot_dir, exist_ok=True) + + for (benchmark, category, mode), sub in df.groupby( + ["scenario", "category", "mode"] + ): + title = f"{str(benchmark).replace('_',' ')} - {str(category).replace('_',' ')} ({mode})" + slug = f"{benchmark}_{category}_{mode}".replace(" ", "_") + sub_t = sub[sub["time_ms"].notna()] + if sub_t.empty: + print(f"Warning: No time data for {title} in {file}") + continue + plot( + plot_dir, + sub_t, + "time_ms", + f"{title} (time)", + f"{slug}_time_ms", + "Time [ms]", + "Normalized time", + x="scale", + ) + + +def analyze_cluster_object_limits_results_file(file: str) -> None: + """Plot cluster_object_limits results. + + For each scenario produce two plots: + - Max-healthy N vs cluster size (one bar per cluster size). + - Max local lag (ms) vs N, with one series per cluster size — shows the + cliff where freshness collapses. + """ + print(f"--- Analyzing cluster_object_limits results file {file} ...") + df = pd.read_csv(file) + if df.empty: + print(f"^^^ +++ File {file} is empty, skipping") + return + + base_name = os.path.basename(file).split(".")[0] + plot_dir = os.path.join("test", "cluster-spec-sheet", "plots", base_name) + os.makedirs(plot_dir, exist_ok=True) + + for benchmark, sub in df.groupby("scenario"): + title_base = str(benchmark).replace("_", " ") + slug_base = str(benchmark) + + # Max-healthy N per cluster size. + healthy = sub[sub["healthy"] == 1] + if healthy.empty: + print( + f"Warning: No healthy data points for {benchmark} in {file}; " + "skipping max-N plot" + ) + else: + max_n = healthy.groupby("cluster_size")["scale"].max().sort_index() + df_max_n = max_n.to_frame(name="max_healthy_N") + ax = df_max_n.plot( + kind="bar", + figsize=(10, 6), + ylabel="Max healthy N", + title=f"{title_base} — max healthy N per cluster size", + grid=True, + legend=False, + ) + ax.set_xlabel("cluster_size") + save_plot( + plot_dir, + df_max_n, + f"{title_base} max healthy N", + f"{slug_base}_max_healthy_n", + ) + + # Lag vs N, one series per cluster size (shows the cliff). + sub_t = sub[sub["time_ms"].notna()] + if sub_t.empty: + print(f"Warning: No lag data for {benchmark} in {file}") + continue + pivot = sub_t.pivot_table( + index="scale", + columns="cluster_size", + values="time_ms", + aggfunc="max", + ).sort_index() + filtered = pivot.dropna(axis=1, how="all") + if filtered.empty: + continue + ax = filtered.plot( + kind="line", + marker="o", + figsize=(12, 6), + ylabel=( + f"Max local lag [ms]" + f" (capped at {CLUSTER_OBJECT_LIMITS_LAG_CAP_MS} ms)" + ), + title=( + f"{title_base} — freshness lag vs N " + f"(values >{CLUSTER_OBJECT_LIMITS_LAG_CAP_MS} ms capped; " + f"healthy threshold {CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS} ms)" + ), + grid=True, + ) + ax.set_xlabel("N (number of materializations on cluster c)") + ax.axhline( + CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS, + color="red", + linestyle="--", + linewidth=1, + label=f"healthy threshold ({CLUSTER_OBJECT_LIMITS_LAG_THRESHOLD_MS} ms)", + ) + ax.legend(loc="upper left", bbox_to_anchor=(1.0, 1.0)) + save_plot( + plot_dir, + filtered, + f"{title_base} lag vs N", + f"{slug_base}_lag_vs_n", + ) + + def save_plot(plot_dir: str, data_frame: pd.DataFrame, title: str, slug: str): all_files = [] diff --git a/test/envd-ddl-scalability/NOTES.md b/test/envd-ddl-scalability/NOTES.md new file mode 100644 index 0000000000000..5bd150b737ffa --- /dev/null +++ b/test/envd-ddl-scalability/NOTES.md @@ -0,0 +1,4593 @@ +# envd DDL scalability audit — working notes + +Living document. Append to it as the investigation progresses. Keep this +short and load-bearing — anything worth keeping for the long term graduates +to README.md or a code comment. + +## Mission + +Audit DDL and catalog-transaction code paths in `environmentd` for code +that scales with the number of catalog objects (O(n) or worse), starting +from empirical scaling measurements, narrowing via tracing, and ending +with concrete design proposals for the worst offenders. + +Anchor signal (`test/cluster-spec-sheet`, `envd_scalability_mvs`, branch +`envd-ddl-scalability`): with N MVs in the catalog, `CREATE TABLE` p50 +grows from 13 ms at N=1 to 88 ms at N=5000 (~6.8×). Peeks stay flat +(~5 ms), so the regression is on the write/catalog path, not the read +path. + +## Working agreement + +- Drive autonomously to a good conclusion; ask the user when blocked or + before doing anything irreversible. +- Commit + push as we go — this machine may go down, so unsaved work is + lost work. Branch: `envd-ddl-scalability`. +- This file is the durable context. Re-read it on session restart. + +## Tooling we built + +- `test/envd-ddl-scalability/audit.py` — tight standalone harness that + connects to a running envd, pads the catalog with N objects of one + type (`tables`, `views`, `mvs`, `indexes`), and times CREATE / DROP / + ALTER / RENAME of various object types at each scale point. Captures + trace IDs from `emit_trace_id_notice` for Tempo lookup. +- `test/envd-ddl-scalability/README.md` — how to run; profiling notes. + +## Decisions log + +- 2026-05-15 — Use a standalone Python script (not mzcompose) so the + iteration loop is tight: one envd startup, many harness runs. Profiling + goes through the canonical `bin/environmentd --optimized --monitoring` + flow. +- 2026-05-15 — Cover all four padding axes (tables / views / mvs / + indexes) and CREATE/DROP/ALTER/RENAME up front; we don't yet know + which axis exposes the worst loops. +- 2026-05-15 — Pad MVs/indexes are sharded across `audit_pad_c_` + clusters with 400 dataflows per cluster, so small replica sizes can + host pad load without dataflow pressure dominating the catalog signal. + +## Findings log + +### 2026-05-15 — source-level survey of the DDL hot path (before traces) + +Anchor doc: `doc/developer/generated/flows.md` "Catalog mutation (DDL)" +section. The path for a single DDL statement: + +1. `mz_sql::plan::statement::ddl::plan_*` — plans the statement. +2. `mz_adapter::coord::sequencer::inner` — calls `catalog_transact`. +3. `mz_adapter::coord::ddl::catalog_transact_inner` + (`src/adapter/src/coord/ddl.rs:311`). +4. `mz_adapter::catalog::transact::transact` + (`src/adapter/src/catalog/transact.rs:420`) → + `transact_inner` (`:636`). +5. Durable commit via `Transaction::commit` → + `mz_catalog::durable::persist::commit_transaction`. +6. In-memory `CatalogState::apply_updates` + (`src/adapter/src/catalog/apply.rs:102`). +7. `Coordinator::apply_catalog_implications` + (`src/adapter/src/coord/catalog_implications.rs:84`). + +**Suspect: doubled in-memory apply.** `transact_inner` applies updates +twice per call: once to a Cow-cloned `preliminary_state` (per op, in +case successive ops in a batch read modified state), and once to a +separate `state` Cow at the end (`transact.rs:696-797`). The comment at +`transact.rs:670` explicitly notes "We won't win any DDL throughput +benchmarks" and acknowledges the doubled work. + +**Not the suspect: state cloning.** `CatalogState` (`state.rs:113`) uses +`imbl::OrdMap` for all its catalog maps; `imbl` clones are O(1) +structural (persistent BTrees). So the `Cow::to_mut()` triggers in +`transact_inner` and the `Arc::make_mut(catalog)` in `ddl.rs:475` are +cheap — not the cause of O(n) growth. + +**Suspect: builtin-table updates.** Each DDL produces +`builtin_table_updates`. Some of these are derived via +`generate_builtin_table_update` per `StateUpdateKind` and may walk +catalog-wide state (e.g. computing privilege rows). Then +`builtin_table_update().execute(builtin_table_updates).await` +(`ddl.rs:516-519`) appends them to system tables. Worth tracing what +this actually does at scale. + +**Empirical slope (from cluster-spec-sheet `envd_scalability_mvs`):** + +``` +N p50 CREATE TABLE +1 13 ms +100 15 ms (~10 ms baseline) +1000 26 ms +3000 58 ms +5000 88 ms +``` + +Linear fit: ~10 ms baseline + ~15 μs/object. Consistent with a single +O(n) walk somewhere; not an O(n²) so far at N≤5000. We may see O(n²) +appear at higher N if a quadratic term is small but nonzero, or if a +per-object op gets more expensive at scale. + +### O(n) suspect 1 — `Transaction::allocate_oids` + +`src/catalog/src/durable/transaction.rs:914-1018` walks every database, +schema, role, item, and introspection source on **every OID allocation**, +inserting their existing OIDs into a `HashSet`, then scans integers +starting from `id_allocator[OID_ALLOC_KEY]` until it finds one not in +the set. Comment at line 944-948: + +> This is potentially slow to do everytime we allocate an OID. A faster +> approach might be to have an ID allocator that is updated everytime an +> OID is allocated or de-allocated. However, benchmarking shows that +> this doesn't make a noticeable difference and the other approach +> requires making sure that allocator always stays in-sync which can be +> error-prone. **If DDL starts slowing down, this is a good place to +> try and optimize.** + +DDL has started slowing down. At N=5000 items, the hashset build is ~50 μs +of pure compute; called once or more per DDL. Worth verifying in traces +whether this accounts for a meaningful fraction of the per-object slope. + +### O(n) suspect 2 — `Coordinator::validate_resource_limits` + +`src/adapter/src/coord/ddl.rs:1056-1500` runs **before** every catalog +transact. For a CREATE TABLE it does **five** full walks of +`entry_by_id`, one per object type: + +| line | call | cost | +| --- | --- | --- | +| 1300 | `for c in self.catalog().user_connections()` | walks all entries, filters connections | +| 1353 | `self.catalog().user_tables().count()` | walks all, counts tables | +| 1360 | `self.catalog().user_sources()...` | walks all, filters sources | +| 1377 | `self.catalog().user_sinks().count()` | walks all, counts sinks | +| 1384 | `self.catalog().user_materialized_views().count()` | walks all, counts MVs | + +Each `user_*()` is defined in `src/adapter/src/catalog.rs:1094-1122` as +`self.entries().filter(...)` over `state.entry_by_id` — there's no +type-bucketed index. So at N=5000 items we burn 25k iterations of the +`imbl::OrdMap`, plus filter predicates, plus the `is_user()` +discriminator check. Estimated ~5-15 ms of pure compute per CREATE at +N=5000 — sizable fraction of the ~75 ms slope, but probably not the +whole thing. + +Fix would be cheap: maintain per-type sets (a `BTreeSet` +per object type) updated in `apply_*_update`, then turn each +`user_tables().count()` into an O(1) `.len()`. Or just maintain +running counters. + +**Tracing instrumentation already present** in DDL path: +- `catalog::transact` (`#[instrument]` at `transact.rs:419`) +- `catalog::transact_inner` (`:636`) +- `coord::catalog_transact_with::finalize` (`ddl.rs:565`) +- per-update `apply_*_update` helpers under `apply.rs` (level=debug) +- `apply_updates_inner` (`:203`) +- `apply_catalog_implications` metric on coord + +With `opentelemetry_filter=debug` we get the debug-level spans, so the +per-update applies are visible. That should let us identify which kind +of update is slow and how its self-time scales with N. + +### 2026-05-15 — first profiling pass (tables padding, local envd) + +Ran `audit.py --padding tables --scale 0,500,2000,5000 --ops +create_table,drop_table,alter_table_add_col,rename_table,create_view, +drop_view --reps 8`. All DDL ops scale linearly with N at remarkably +similar slopes (3.3-4.7 μs/object), which means **the dominant O(N) +cost is shared infrastructure, not op-specific**. + +p50 latency in ms (local envd, not directly comparable to cluster- +spec-sheet's cloud numbers, but slope is what matters): + +| op | N=0 | N=500 | N=2000 | N=5000 | Δ@5000 | +| --- | ---: | ---: | ---: | ---: | ---: | +| create_table | 31 | 33 | 37 | 55 | +23 | +| drop_table | 20 | 21 | 30 | 40 | +20 | +| alter_table_add_col | 23 | 25 | 32 | 41 | +18 | +| rename_table | 20 | 21 | 28 | 37 | +17 | +| create_view | 18 | 21 | 26 | 38 | +20 | +| drop_view | 19 | 22 | 29 | 39 | +19 | + +CSV at `/tmp/audit-tables.csv`, summary at +`/tmp/audit-tables.summary`. Slopes ~4 μs/object across all ops +means even *no-controller-side-effect* DDLs (rename, alter, view +create/drop) pay the same per-object price. + +### Trace analysis: which spans grow with N + +Top self-time growers from N=0 → N=5000: + +**For CREATE TABLE** (full +23 ms breakdown): +| span | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| `storage::create_collections` | 9.9 | 22.5 | **+12.6** | +| `snapshot` (catalog durable) | 0.5 | 4.1 | +3.6 | +| `transaction` (catalog durable) | 0.5 | 2.5 | +2.0 | +| `consolidate` (catalog durable) | 0.4 | 2.3 | +1.9 | +| `PersistTableWriteCmd::Append` | 5.6 | 7.5 | +1.9 | +| `apply_catalog_implications_inner` | (new) | 1.1 | +1.1 | +| `apply_updates` | 0.5 | 1.2 | +0.7 | + +**For ALTER TABLE** (full +18 ms breakdown): +| span | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| `snapshot` | (sub-ms) | 3.8 | +3.6 | +| `transaction` | (sub-ms) | 2.5 | +2.1 | +| `consolidate` | (sub-ms) | 1.9 | +1.5 | +| `coord::catalog_transact_with_context::table_updates` | 7.6 | 9.2 | +1.6 | +| `apply_updates` | 0.7 | 1.2 | +0.5 | +| `apply_catalog_implications_inner` | (new) | 1.1 | +1.1 | +| `PersistTableWriteCmd::Append` | 6.6 | 8.3 | +1.7 | + +**For RENAME TABLE** (full +17 ms breakdown): same shape as ALTER — +the growers are catalog durable txn spans (`snapshot`, `transaction`, +`consolidate`) plus persist append. + +### Where the per-object cost lives + +Two clusters of O(N) cost dominate: + +1. **`storage::create_collections`** — only for CREATE TABLE (and + anything else that creates a storage collection). +12 ms / 5000 ≈ + 2.5 μs/object. Suspect: scans existing collections to set up read + policies / read holds / metadata. Need to read + `src/storage-controller/src/lib.rs::create_collections`. + +2. **Catalog durable txn machinery** (`snapshot`, `transaction`, + `consolidate`) — present for **every** DDL. +7-8 ms / 5000 ≈ + 1.5 μs/object. These spans live in + `src/catalog/src/durable/persist.rs` and + `src/catalog/src/durable/transaction.rs`. Combined with + `TableTransaction::insert/update` doing `for_values` over all N + items (suspect 3 below), this looks like the catalog snapshot read + from persist is full-state every time, even when the txn only + touches a few rows. + +### O(n) suspect 3 — `TableTransaction::insert/update` scans all rows + +`src/catalog/src/durable/transaction.rs:3190-3210` — every `.insert(k, +v, ts)` on a `TableTransaction` calls `self.for_values(|for_k, for_v| +{ ... })` to check both `k == for_k` and `uniqueness_violation`. That +walks **every initial row + every pending row** of the table on each +single insert. For a CREATE TABLE we insert a handful of rows (item, +maybe extra metadata), and each insert walks all N existing items. + +`update` (`:3222`) has the same structure: `for_values_mut` walks all +items, calling `f(k, v)` on each. + +This is the most plausible source of the per-object cost in the +catalog durable txn spans (`transaction`, `consolidate`, `snapshot`) +that grow with N regardless of op type. + +### 2026-05-15 — views and mvs padding passes + +**Views padding** (N=0,500,2000,5000): slope is roughly half of tables- +padding. Δ@5000 for CREATE TABLE: +8.5 ms (vs +23 ms with tables pad). +Confirms that **two cost components are stacked**: + +1. ~half scales with **storage-collection count** (only paid by ops + that create/drop storage collections, i.e. CREATE TABLE / DROP TABLE + and friends). Tables have shards; views do not. +2. ~half scales with **catalog-entry count** of any kind. All ops pay + this. + +**MVs padding** (N=0,500,2000): blows up super-linearly. + +| op | N=0 | N=500 | N=2000 | factor 500→2000 | +| --- | ---: | ---: | ---: | ---: | +| create_table | 37 | 146 | **2095** | 14× for 4× N | +| drop_table | 36 | 99 | 996 | 10× for 4× N | +| alter_table_add_col | 46 | 138 | 1127 | 8× for 4× N | +| rename_table | 29 | 119 | 653 | 5.5× for 4× N | +| create_view | 31 | 105 | 479 | 4.6× for 4× N | +| create_mv | 40 | 51 | 837 | 16× for 4× N | +| drop_mv | 31 | 44 | 446 | 10× for 4× N | + +4× more N giving 5-16× more latency = **quadratic** somewhere. CSV at +`/tmp/audit-mvs.csv`. + +Trace for `create_table` p50=2.1 s at N=2000 MVs: +- `storage::create_collections` total: 1.72 s + - **self-time: 1.65 s** + - Visible child: `PersistTableWriteCmd::Register` 64 ms +- `coord::catalog_transact_with_side_effects` self: 192 ms +- `coord::initialize_read_policies`: 13 ms (vs 0.5 ms at N=0) + +**1.65 seconds inside `storage::create_collections` is in code not +covered by any sub-span.** Source reading turned up plausible callers +inside `storage_collections.create_collections_for_bootstrap` +(`src/storage-client/src/storage_collections.rs:1686`) but none +explicitly look quadratic. Most likely candidates: + +- `install_collection_dependency_read_holds_inner` → + `install_read_capabilities_inner` → `update_read_capabilities_inner` + walks `MutableAntichain::update_iter` on collections with many read + capabilities. With N MVs holding read holds on a single `pad_base`, + pad_base's `read_capabilities` could have N entries → an + `update_iter` over it would be O(N). +- The recursive propagation in `update_read_capabilities_inner` + (`:1250`) walks `storage_dependencies` and adds them to `updates`, + potentially fanning out across the dependency graph. +- `acquire_read_holds_inner` could be amplified. + +But we need either targeted instrumentation or a CPU profile to be +sure. Reading further blindly is hitting diminishing returns. + +### What we know now + +- **Linear O(N) bottleneck** is shared across all DDL ops, around + ~4 μs/catalog-entry on this local envd. Sources identified by code: + - `Transaction::allocate_oids` (full walk) + - `Coordinator::validate_resource_limits` (5 full walks) + - `TableTransaction::insert/update` (full walk on every mutation) +- **Quadratic O(N²) bottleneck** triggered when padding objects + share dependencies (e.g. MVs reading from a common base table). + Bottleneck lives inside storage controller's `create_collections` + path, not yet pinpointed; suspect read-capability propagation + through dependency edges. + +## Open questions + +- Does the scaling pattern differ across padding axes? If `views` (no + dataflows) shows the same DDL slowdown as `mvs`, the hotspot is purely + in the catalog/coordinator. If `mvs` is dramatically worse, controller + state matters too. +- Are CREATE-side and DROP-side regressions caused by the same loops, or + different ones (dependency walks tend to live on DROP)? + +## Design proposals (draft — for the linear O(N) suspects) + +These are the three confirmed O(N) hotspots from source reading. None +of them require the storage-side investigation to be complete. They +also stack — each contributes part of the ~4 μs/object slope. + +### Fix 1 — `Coordinator::user_*().count()` calls + +`src/adapter/src/coord/ddl.rs:1056-1500`, `validate_resource_limits`. +Today every DDL does 5+ full walks of `state.entry_by_id` to count +items per type. + +**Proposal:** add type-bucketed indexes to `CatalogState`, maintained +in `apply_item_update` (and `drop_item`): + +```rust +// in CatalogState (state.rs) +items_by_type: imbl::OrdMap>, +``` + +with buckets `Table`, `Source`, `Sink`, `MaterializedView`, `View`, +`Index`, `Connection`, `Secret`, `Type`, `Func`, `Log`, +`ContinualTask`. Add `user_*().count()` overloads that return `.len()` +on the bucket. Rewrite `validate_resource_limits` to use the counts. + +Trade-offs: +- O(log N) bookkeeping in `insert_entry` / `drop_item` (negligible). +- ~4-12 ms/DDL saved at N=5000 from `validate_resource_limits` alone. +- Same buckets are useful for `mz_objects` builtin tables and ad-hoc + introspection; can replace some other full walks. +- Has to stay in sync with `is_user()` filter (different bucket for + system vs user, or filter at read time). + +### Fix 2 — `Transaction::allocate_oids` + +`src/catalog/src/durable/transaction.rs:914-1018`. Every OID alloc +builds a HashSet of all in-use OIDs by walking all +databases/schemas/roles/items/introspection_sources. + +**Proposal:** what the existing comment already suggests — maintain a +durable free-list / next-id allocator that tracks taken OIDs +incrementally. Two options: + +a. **Append-only bump + tombstones for reuse.** Today OIDs recycle. + If we make the `id_allocator` store the next monotonic OID + (`OID_ALLOC_KEY.next_id` already does this) and keep a separate + `taken_oids` `BTreeSet` in memory, allocation is + O(log N). On drop we insert into a `freed_oids` set; on alloc we + first pop from `freed_oids` before bumping. + +b. **Skip OID reuse entirely.** OIDs are u32; at 1k DDLs/s we exhaust + in ~50 days, but that's not actually true because we cap to + `FIRST_USER_OID..u32::MAX` (a few billion). For practical purposes + we could simply never reuse and panic on wrap-around. Simplest + possible fix; matches Postgres's behaviour where OID reuse is also + rare. + +(a) preserves current semantics. (b) is simpler but a semantic +change worth checking with the team. + +### Fix 3 — `TableTransaction::insert/update` full scans + +`src/catalog/src/durable/transaction.rs:3190` (insert) and `:3222` +(update). Both call `for_values` to check for duplicate-key and +uniqueness-violation against every initial+pending row. For a CREATE +TABLE with N pre-existing items, every `items.insert(...)` walks N +items. + +**Proposal:** since `for_values` is uniformly walked for these checks: + +a. **Key check via `BTreeMap::contains_key`.** Replace the `k == for_k` + scan with a `BTreeMap` (or `BTreeSet`) lookup. `pending` is + already a `BTreeMap`, so we just need to also have access to + `initial`'s keyspace — which is also a `BTreeMap` (see + `current_items_proto`). Both lookups are O(log N). + +b. **Uniqueness check** is the harder part: the predicate + `uniqueness_violation(for_v, &v)` is opaque, so we can't index it + generically. But for most tables, `K` IS the unique key, so the + predicate is `false`. We can let callers either: + - declare "no uniqueness violations possible" so we skip the loop + entirely, OR + - register a "uniqueness key extractor" so we keep a side + `BTreeMap` and check via that map. + +This change is the most invasive of the three but probably also the +highest-leverage; it touches every TableTransaction mutation. + +### Roll-up + +The three fixes combined likely close most of the ~4 μs/object slope +on linear paths. None changes externally visible semantics; all are +"replace full-walk with O(log N) lookup using an index maintained in +the existing apply path." Tests are easy to add: time `CREATE TABLE` +in a catalog with N=10/100/1000/10000 trivial tables and assert near- +constant latency. + +The quadratic-looking blow-up under MV-shared-dependency padding is +**not** covered by any of these — see "Storage-side blow-up" below. + +### 2026-05-15 — pinpointing the storage-side blow-up + +Added scoped tracing spans inside the storage controller's +`create_collections_for_bootstrap` chain. Trace for CREATE TABLE at +N=1000 MVs (444 ms p50, vs ~30 ms at N=0): + +| span | self-time at N=1000 | +| --- | ---: | +| `ccfb::open_data_handles_concurrent` (was unattributed) | 119 ms | +| `PersistTableWriteCmd::Register` (txn-wal) | 46 ms | +| catalog persist `compare_and_append` | 23 ms | +| `ccfb::install_collection_states` (main loop) | ~0 ms | +| `ccfb::synchronize_finalized_shards` | ~0 ms | + +So **the inner main loop and the synchronize call are not the issue** +even at N=1000 MVs. The cost is dominated by `open_data_handles`. +Stepping inside `open_data_handles`: + +| span | self-time at N=1000 | +| --- | ---: | +| `odh::upgrade_version` | 40.8 ms | +| `odh::open_critical_handle` | 32.6 ms | +| `odh::fetch_recent_upper` | 7.2 ms | +| `odh::open_write_handle` | (sub-ms) | + +Both `upgrade_version` and `open_critical_handle` invoke persist's +`StateCache::get` → `Applier::new` → `maybe_init_shard` → +`fetch_recent_live_diffs` / `try_compare_and_set_current` / +`fetch_current_state`. These hit CockroachDB's `consensus` table via +queries like: + +```sql +SELECT sequence_number, data FROM consensus +WHERE shard = $1 ORDER BY sequence_number DESC LIMIT 1 +``` + +Per-shard, primary-key access — should be O(log total_rows). But the +measurement says the cost scales with the number of *existing* shards +in CRDB. Plausible explanations (not yet verified): + +- **CRDB query latency degradation** under N rows is the simplest + one — even index-only lookups can slow when the table has lots of + rows due to caching effects. +- **Persist `StateCache` lock contention** — the cache holds a single + `Mutex` (`src/persist-client/src/cache.rs:479`) across all shards; + if background tasks hold it proportional to N, every new shard's + insert waits. +- **PubSub subscribe path** — `pubsub_sender.subscribe(&shard_id)` is + invoked from inside the cache lock; if it has any O(N) accounting, + this would multiply per-shard cost. +- **CRDB `consensus` table bloat from version churn** — every shard + state mutation appends a row, and old rows get GC'd. With many + shards, churn rate matters. + +In every case, this is **persist / CRDB territory**, not adapter / +catalog. The catalog-side O(N) fixes from the previous section are +independent and worth landing first. + +The catalog persist `compare_and_append` for the catalog shard (23 ms +at N=1000, vs ~2 ms at N=0) also looks linear in something — likely +the catalog shard accumulates state per item, so its own consensus +state grows with N. Compaction settings on the catalog shard would +affect this. + +### Updated proposals + +**Storage-side fix (Fix 4 — needs persist team).** The persist +`make_machine` / `maybe_init_shard` path should not scale with the +number of existing shards in the same process or in CRDB. Concrete +investigations to start with: + +1. Run the audit harness with `samply record -p $envd_pid` and + confirm the CPU flame graph attributes the time to either + CRDB-network I/O (then it's a database-side issue) or to in-process + Rust code (then it's a persist-cache / pubsub issue). +2. Time the raw CRDB query `SELECT sequence_number, data FROM + consensus WHERE shard = $1 ORDER BY sequence_number DESC LIMIT 1` + at varying total row counts. If that's the source, file it with + storage/persist and CRDB folks. +3. Profile under contention: with N MVs, if multiple compute replicas + are still chattering with persist, the lock on `StateCache.states` + may be more contended than the trace suggests. Re-test with + `--meas-cluster-size scale=1,workers=1` and no pad clusters + (`tables` padding) at high N to isolate from compute side-effects. + +### 2026-05-15 — post-fix measurements (CORRECTED) + +Three catalog-side fixes landed on this branch: + +- `f69d91c977` — bucket user item counts in `CatalogState`; `validate_resource_limits` now does O(log K) lookups instead of 5 walks of `entry_by_id`. +- `9fca09ff8a` — maintain `Transaction::initial_oids` so `allocate_oids` doesn't walk every db/schema/role/item/intro-source per allocation. +- `293a243e6b` — `TableTransaction::insert` uses `self.get(&k).is_some()` instead of `for_values` for dup-key, and skips the uniqueness walk when `uniqueness_violation` is `None` (14 of 22 instances). + +**First post-fix run was misleading.** The pre-fix tables audit ran on +an envd that had been up ~30 minutes (warm caches); the first post-fix +audit ran on a freshly-reset envd with cold caches and showed N=0 at +~60 ms vs pre-fix N=0 at ~30 ms. That made the post-fix N=5000 number +(~50 ms) look like a flat line vs N=0, suggesting "slope eliminated." +It wasn't — the slope just looked flat because N=0 was inflated. + +**Apples-to-apples comparison on a warm envd:** the slope is essentially +unchanged. + +| op | pre-fix N=0 / N=5000 / Δ | post-fix N=0 / N=5000 / Δ | +| --- | --- | --- | +| create_table | 31 / 55 / **+23 ms** | 30 / 51 / **+21 ms** | +| drop_table | 20 / 40 / +20 ms | 20 / 41 / +21 ms | +| rename_table | 20 / 37 / +17 ms | 19 / 39 / +19 ms | + +CSV at `/tmp/audit-tables-warm.csv`. The three catalog-side fixes are +theoretically correct (they eliminate the named O(N) loops), but their +combined contribution to the ~4 μs/object slope is **lost in noise** at +N=5000. Implication: the named loops were not the slope's dominant +source. There's another O(N) elsewhere in the per-DDL path that the +audit hadn't pinpointed. + +Candidates not yet ruled out: +- The `snapshot` span (catalog durable txn read) grows linearly with N + in the traces — 0.5 → 4 ms across the range. We attributed this to + `TableTransaction::insert/update` for_values, but `insert` is now + fast. So the per-DDL `snapshot` cost has another source. +- `compare_and_append` for the catalog shard grows similarly (2 ms → + ~5 ms). The catalog shard accumulates a row per item; persist's + per-row consensus operation might genuinely scale. +- `apply_updates` self-time in the in-memory catalog grew mildly + (0.5 → 1.2 ms). Some bookkeeping there scales we haven't seen. +- Things in `apply_catalog_implications_inner` that we didn't dig into. + +Need to fetch a fresh post-fix high-N trace and rank growers again. + +**MVs-padding:** per-N cost dropped ~2-3× but the super-linear +remainder is intact, as predicted (the persist-side blow-up identified +in `open_data_handles` is not addressed by these fixes): + +| op | pre-fix @2000 | post-fix @2000 | speedup | +| --- | ---: | ---: | ---: | +| create_table | 2095 ms | 851 ms | 2.5× | +| drop_table | 996 ms | 536 ms | 1.9× | +| rename_table | 653 ms | 748 ms | 0.9× (noisy) | +| create_view | 479 ms | 330 ms | 1.5× | +| create_mv | 837 ms | 1095 ms | 0.8× (noisy) | +| drop_mv | 446 ms | 282 ms | 1.6× | +| drop_view | 795 ms | 1152 ms | 0.7× (noisy) | + +With reps=5 there's meaningful variance and a few cells go the wrong +way; the median across the matrix shows clear improvement but the +super-linear shape vs. N is intact (e.g. CREATE TABLE: 31 → 81 → 851 ms +for N = 0 / 500 / 2000 is still ~10× for 4× N). The persist-side +`open_data_handles` cost we pinpointed earlier dominates here. CSV at +`/tmp/audit-mvs-post.csv`. + +**Net conclusion (corrected above)**: the three catalog-side fixes are +each individually correct but eliminate a small fraction of the slope. +The dominant per-DDL O(N) cost is elsewhere — see the trace ranking +below. + +### 2026-05-15 — post-fix high-N trace ranking (where the slope actually lives) + +Re-ran the audit on a warm, post-fix envd with `--padding tables +--scale 0,5000 --ops create_table,drop_table,alter_table_add_col, +rename_table --reps 8`. CSV at `/tmp/audit-trace-postfix-N5000.csv`. +Latencies (p50): + +| op | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| create_table | 30.0 | 47.3 | +17.3 ms | +| drop_table | 18.6 | 42.6 | +24.0 ms | +| alter_table_add_col | 22.3 | 53.0 | +30.7 ms | +| rename_table | 17.4 | 43.6 | +26.2 ms | + +Span self-times averaged over 4 traces per cell, ranked by N=0→N=5000 +growth. Numbers are self-time at N=5000 (the delta from N=0 is the +relevant signal but cells without entries at N=0 mean the span had +sub-threshold self-time). Pulled via +`test/envd-ddl-scalability/summarize_traces.py`. + +**CREATE TABLE** — explains roughly 13 of the +17 ms slope: + +| span | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| `snapshot` (catalog durable) | 509μs | 4.1ms | **+3.6ms** | +| `group_commit_apply::append_fut` | 5.0ms | 7.6ms | +2.6ms | +| `PersistTableWriteCmd::Append` | 5.0ms | 7.6ms | +2.6ms | +| `transaction` (catalog durable) | 426μs | 3.0ms | **+2.6ms** | +| `consolidate` (catalog durable) | 449μs | 2.1ms | +1.7ms | +| `apply_catalog_implications_inner` | <0.5ms | 1.2ms | +1.2ms | +| `apply_updates` | 536μs | 1.4ms | +0.9ms | + +**ALTER TABLE** — explains roughly 17 of the +31 ms slope: + +| span | N=0 | N=5000 | Δ | +| --- | ---: | ---: | ---: | +| `coord::catalog_transact_with_context::table_updates` | 7.3ms | 12.9ms | **+5.6ms** | +| `group_commit_apply::append_fut` | 6.2ms | 10.9ms | +4.7ms | +| `snapshot` | 551μs | 4.3ms | **+3.8ms** | +| `transaction` | 453μs | 3.2ms | **+2.7ms** | +| `consolidate` | 447μs | 1.9ms | +1.5ms | +| `apply_catalog_implications_inner` | <0.5ms | 1.3ms | +1.3ms | + +**DROP TABLE** and **RENAME TABLE** have the same shape (snapshot, +transaction, consolidate, append, apply_catalog_implications_inner +all grow), and don't add new growers. + +### Mapping growers to code + +1. **`snapshot` + `transaction` + `consolidate` — the structural target.** + - `with_snapshot` (`src/catalog/src/durable/persist.rs:766`) walks + the consolidated trace and rebuilds a `Snapshot { databases: + BTreeMap, schemas: BTreeMap, items: BTreeMap, ... }` from scratch + on every transaction. With N=5000 items, that's 5000 inserts into + the items BTreeMap alone. + - `Transaction::new` (`transaction.rs:128-232`) then walks every + row of every snapshot table and calls `TableTransaction::new(...)` + for each, which itself does + `initial.into_iter().map(RustType::from_proto).collect()` — a + full O(N) proto→Rust conversion + BTreeMap construction. + - `consolidate` (`persist.rs:706`) re-consolidates the in-memory + trace via `differential_dataflow::consolidation:: + consolidate_updates`. Trace grows with N → consolidation cost + grows with N. + - **Combined growth**: +7-8 ms / 5000 ≈ 1.5 μs/object. The single + biggest source of the per-DDL slope. + - The proposed design (Arc'd `DurableCatalogData` + per-txn overlay) + eliminates all three: starting a transaction becomes + `Arc::clone(&self.data)`, reads probe overlay then base, commits + emit only delta keys, no full state materialisation per txn. + +2. **`group_commit_apply::append_fut` / `PersistTableWriteCmd::Append` + — persist-side, partly out of scope.** Growth here is the catalog + shard's consensus append getting bigger as the catalog accumulates + rows. Some of this should drop when the durable txn emits only + delta keys (because the per-txn batch is smaller), but a meaningful + chunk is persist's own state machinery scaling with shard history. + Treat as a follow-up; the structural fix above does *some* of the + work indirectly. + +3. **`coord::catalog_transact_with_context::table_updates` (+5.6 ms + for ALTER) — separate hot path.** This is the wrapper span around + the catalog transact for table-mutating DDL. Its self-time grew + most for ALTER. Worth instrumenting deeper inside before + implementation — there may be a smaller, easily-fixed loop hiding + in here. + +4. **`apply_catalog_implications_inner` (+1.2-1.3 ms across ops).** + `src/adapter/src/coord/catalog_implications.rs:182`. Iterates the + per-DDL `implications` list (which is delta-sized, so itself O(1)), + then has handlers per kind. The growth implies one of the handlers + reads catalog-wide state. Should be auditable in a focused pass. + +5. **`apply_updates` (+0.9 ms).** Small but real. The in-memory + `CatalogState::apply_updates` path; should be O(delta) by design, + but something inside scales mildly. Worth auditing in the same + pass as item 4 above. + +### Revised design priorities (confirmed by trace) + +The trace confirms the proposed design's primary target is the right +one. In priority order: + +1. **Shared, indexed durable state + per-txn overlay.** Kills the + `snapshot` + `transaction` + `consolidate` triple. Estimated + payoff: ~1.5 μs/object → roughly halves the slope at N=5000. +2. **Indexes for name/OID/namespace lookups.** Eliminates the + `for_values` walks; mostly already done via Fix 3 (insert) but + `update`/`for_values_mut` and 8 of 22 `insert` callers still walk. + Estimated payoff: small at N=5000 (already covered by Fix 3 + partially) but compounds at higher N. +3. **Storing the durable txn overlay in `TransactionOps::DDL` and + stopping op replay.** Highest-leverage for multi-statement DDL + transactions; doesn't show up in our single-statement audit, so + not visible in the slope numbers above. Still worth doing. +4. **Audit `apply_catalog_implications_inner` and `apply_updates`.** + Both grow mildly with N; find the loop, fix it. Likely small, + focused changes. +5. **Point APIs for storage metadata.** As designed. +6. *Out of scope for now:* persist-side `open_data_handles` / + `compare_and_append` cost. Real but separate workstream. + +`apply_catalog_implications_inner` and `apply_updates` together add +~+2 ms at N=5000 — small enough to leave for the audit pass per +item 4 above, but worth fixing for completeness. + +The OID-set cache (Fix 2) is correctly noted as *not* moving the +needle for single-DDL transactions, because building the set on +every txn start is O(N) regardless of how fast lookups are. +Single-DDL is the common case. The cache only pays off if it's +shared incrementally across transactions — which the proposed design +provides via the Arc'd durable state. + +## Next steps + +1. Land the structural fix: `DurableCatalogData` (`Arc` + per table) maintained incrementally in `PersistHandle`, with + per-txn overlay reads and delta-only commits. +2. Audit `apply_catalog_implications_inner` and `apply_updates` for + the residual ~+2 ms growth. +3. Storage point APIs for catalog-side storage metadata reads. +4. Move durable txn overlay into `TransactionOps::DDL` to kill op + replay across multi-statement DDL transactions. + +### 2026-05-15 — post-design measurements (after steps 1-7) + +Steps 1-7 of `doc/developer/design/20260515_ddl_catalog_o_delta.md` +landed. Re-ran the audit on a warm release envd: + +``` +bin/environmentd --release +python3 test/envd-ddl-scalability/audit.py \ + --padding tables --scale 0,5000,10000 \ + --ops create_table,drop_table,alter_table_add_col,rename_table \ + --reps 8 +``` + +p50 latency in ms: + +| op | N=0 | N=5000 | Δ@5000 | N=10000 | Δ@10000 | +| --- | ---: | ---: | ---: | ---: | ---: | +| create_table | 26.3 | 34.0 | +7.7 | 41.1 | +14.8 | +| drop_table | 17.2 | 24.8 | +7.6 | 34.4 | +17.2 | +| alter_table_add_col | 20.7 | 27.1 | +6.4 | 36.6 | +15.9 | +| rename_table | 17.1 | 22.8 | +5.7 | 31.3 | +14.2 | + +Pre-design baseline (the "post-fix high-N trace ranking" section +above), reps=8, warm envd: + +| op | pre-design Δ@5000 | post-design Δ@5000 | reduction | +| --- | ---: | ---: | ---: | +| create_table | +17.3 | +7.7 | 55% | +| drop_table | +24.0 | +7.6 | 68% | +| alter_table_add_col | +30.7 | +6.4 | 79% | +| rename_table | +26.2 | +5.7 | 78% | + +Slope dropped from ~3-6 μs/object to ~1.4-1.7 μs/object — a roughly +half to a quarter of the pre-design rate. The design's success +criterion #1 was "≤ +5 ms"; we land at +5.7 to +7.7 ms, very close +but not quite there. Criterion #2 ("invariant under add-only growth") +is not satisfied either — N=10000 still adds ~7 ms over N=5000. + +The residual slope is almost certainly the persist-side cost on the +catalog shard (`PersistTableWriteCmd::Append` / `compare_and_append`) +that the design explicitly lists as out-of-scope (§"Out of Scope" → +"Persist-side scaling of the catalog shard"). The trace in the +pre-design ranking attributed +2.6 ms / 5000 to those two spans +alone; that cost is now the dominant per-object grower. + +The remaining handful-of-ms gap to the success target needs a +follow-on persist/storage workstream, not more catalog-side work. + +### 2026-05-15 — bogo-consensus probe (does the residual slope live in CRDB?) + +Cherry-picked `819a69a6d1` ("persist: add bogo-consensus, an in-memory +gRPC Consensus backend") onto this branch as the experiment vehicle. +The hypothesis was: if the post-design residual ~1.5 μs/object slope +is dominated by persist's `compare_and_append` round-trip to CRDB, +swapping the consensus backend for an in-memory gRPC service should +reduce or eliminate that slope, giving us cleaner signal for the next +round of catalog-side work. + +Setup: `bin/environmentd --optimized` against +`bogo://127.0.0.1:6882`, CRDB still in the loop for the timestamp +oracle. Default 4 MiB gRPC message size cap is too small for the +catalog shard once history accumulates; bumped client and server to +256 MiB (server: `BogoGrpcServer::new(...).max_decoding_message_size` +/ `max_encoding_message_size`; client: same on `TonicClient`). +Without this fix the catalog shard's `scan` retries-with-backoff +indefinitely once history grows past 4 MiB; the audit completes but +the latencies are dominated by retry sleeps, not actual work. + +Numbers (warm envd, `--padding tables --scale 0,5000 --reps 8`), +side by side with the optimized CRDB baseline: + +| op | CRDB N=0 | CRDB N=5000 | CRDB Δ | bogo N=0 | bogo N=5000 | bogo Δ | +| --- | ---: | ---: | ---: | ---: | ---: | ---: | +| create_table | 31.8 | 39.4 | +7.6 | 104.8 | 150.6 | +45.8 | +| drop_table | 19.0 | 31.2 | +12.2 | 46.9 | 70.3 | +23.4 | +| alter_table_add_col | 23.8 | 34.0 | +10.2 | 62.6 | 119.3 | +56.7 | +| rename_table | 18.9 | 29.9 | +11.0 | 57.9 | 76.3 | +18.4 | + +Per-object slope under bogo (≈3.7-11.3 μs/object) is **higher**, not +lower, than under CRDB (≈1.5-2.4 μs/object). Absolute latencies are +also 2-4× worse. Padding the catalog (5000 sequential CREATE TABLEs) +took 551 s under bogo vs 193 s under CRDB. + +That's an informative negative result. Interpretations: + +- The residual slope is **not** dominated by CRDB query latency. If + it were, replacing CRDB with a local in-process state machine would + drop it. It doesn't — it grows. +- The bogo backend's `Vec`-per-key model with a single + global `Mutex` doesn't beat CRDB for this workload. Each persist + scan currently transfers history-shaped state over gRPC + protobuf; + the cost grows with shard history, and the global mutex serialises + every persist op. CRDB's row-level locking and binary client + protocol are tighter at this size. +- The persist-side residual on CRDB is likely in `Applier::new` / + `maybe_init_shard` / `compare_and_append`'s own state-machine + bookkeeping (state encoding, version walks, listen plumbing) rather + than in the SQL roundtrip itself. That matches the earlier finding + in this file that `open_data_handles` dominated MV padding. + +What this means for the next round of catalog-side work: there isn't +a hidden CRDB-latency factor masking a deeper O(n) loop in adapter or +catalog. The residual ~1.5 μs/object on CRDB really does look like +persist-internal cost. Further reductions need to come from either +(a) persist-side scaling work (smaller per-DDL batches, smarter state +caching), or (b) batching multiple DDL operations into a single +persist `compare_and_append` — the design's step 5 already does this +for multi-statement DDL transactions, but single-statement DDL still +pays once per statement. + +### 2026-05-18 — why was bogo slower? HTTP/2 flow control on small RPCs + +The earlier conclusion ("the global mutex serialises every persist op") +was wrong. A focused microbench against `BogoConsensus` vs `MemConsensus` +(see `src/persist/src/bogo.rs::bogo_consensus_microbench`) showed three +things: + +1. **The Mutex was never the bottleneck.** Concurrent CAS on 64 shards + hits 30-50k ops/s on bogo — far short of single-mutex contention + limits; the in-memory work under the lock is ~1 μs/op, leaving the + lock idle. +2. **Per-op gRPC overhead is the floor.** On loopback, the smallest + bogo RPC (head with no data) takes ~100 μs round-trip. That alone + is ~3× a typical CRDB consensus call latency under load, before + any payload is in scope. +3. **`tonic` defaults Nagle and HTTP/2 flow control windows to the + spec minimum (65 KiB).** Per-call CAS latency on the bench rises + linearly with payload size at ~130 ns/byte (≈ 7 MB/s effective + throughput) — because every request whose payload spans the + connection's send window stalls waiting for the server's + WINDOW_UPDATE round-trip. The catalog shard's appends grow well + past 64 KiB once history accumulates. + +The fix: bump `initial_stream_window_size` to 8 MiB and +`initial_connection_window_size` to 16 MiB on both client and server, +and enable `tcp_nodelay(true)` on the server. With those, 16 KiB CAS +drops from 2204 μs/op to 241 μs/op (9×), and concurrent throughput at +concurrency=16 rises from 21 K ops/s to 50 K ops/s. Landed in +`4fe0d584e2` ("bogo-consensus: raise HTTP/2 flow control windows; add +microbench"). + +### 2026-05-18 — post-window-fix DDL audit + +Re-ran the audit on a warm release envd, exact same shape as the +pre-window-fix bogo numbers above. Side-by-side, including the +optimized CRDB baseline so we can see how much of the gap the window +fix closed: + +| op | CRDB Δ | bogo pre-fix Δ | bogo post-fix Δ | slope reduction | +| --- | ---: | ---: | ---: | ---: | +| create_table | +7.6 | +45.8 | +18.3 | 60% | +| drop_table | +12.2 | +23.4 | +7.1 | 70% | +| alter_table_add_col | +10.2 | +56.7 | (noisy) | — | +| rename_table | +11.0 | +18.4 | (noisy) | — | + +`alter` and `rename` got too noisy at reps=8 to interpret (a few +single-rep stalls dominate the p50/p95). `create_table` and +`drop_table` are the cleanest signal and show ~60-70% of the bogo +slope coming out — bogo now within ~2× of CRDB's slope, where before +it was 3-6×. + +Absolute N=0 latency improved too — bogo `create_table` N=0: 104.8 → +54.3 ms — but bogo's baseline is still 1.5-2× CRDB's at N=0. Looking +at envd's persist-client metric for `consensus_cas` under bogo (199 K +calls over the audit run): **p50 ≈ 500 μs, p95 ≈ 64 ms, max 256 ms**, +mean 10.2 ms. Compare to CRDB on the same envd build: p50 ≈ 1.8 ms, +p95 ≈ 4 ms, max ~32 ms, mean 1.8 ms. So bogo's p50 is actually +*better* than CRDB's, but a fat 5-10% tail at 32-64+ ms drives up the +mean and dominates DDL latency. + +The bogo server itself is fast across the same period (server-side +mean ~1 μs per CAS, no spikes above 64 μs). So the tail lives in +the gRPC client path / network / persist's `Tasked` task-hop, not +in bogo. Most plausible culprit: HTTP/2 head-of-line blocking on the +single shared connection — a multi-MB scan response stalls the TCP +receive buffer and all sibling streams (small CAS responses) wait +behind it. Server-side metrics show the scan/CAS bytes total roughly +matches the mean op latency × count gap, consistent with HoL. + +### Next moves for bogo perf (if we want to keep pushing) + +1. **Multiple parallel gRPC channels** in `BogoConsensus`, round-robin + per-RPC. Removes HoL blocking between large scans and small CAS + requests. Probably the single biggest remaining win. +2. **Stream scans** instead of returning a `Vec`. + Server pushes entries incrementally; client assembles. Reduces + peak buffer occupancy and protobuf-encode cost. +3. **Move bogo metrics off the hot path.** The + `update_state_metrics` walk runs under the global lock on every + CAS/truncate, and four `with_label_values(&[...])` lookups + happen per RPC; small but adds up at high RPS. +4. **UDS instead of loopback TCP.** Drops kernel TCP overhead; + tonic supports it via `tower::service_fn`. + +None of these change semantics; all are isolated to +`mz-bogo-consensus` + the `mz_persist::bogo` adapter. + +### 2026-05-18 — multi-channel client (item 1 above) landed + +`15f741cf73` ("bogo-consensus: fan out client RPCs across multiple +gRPC channels") opens 8 independent tonic `Channel`s in the +`BogoConsensusClient` and round-robins RPCs across them. + +Microbench delta (`bogo_consensus_microbench`, same machine, +release): + +| | single channel | 8 channels | +| ---: | ---: | ---: | +| serial CAS, 16 KiB | 240 μs | 146 μs | +| concurrent CAS, conc=64 | 32 K/s | 67 K/s | + +Single-op serial latency on the smallest payloads is flat — the +loopback round-trip floor (~90 μs) doesn't depend on connection +count. + +### 2026-05-18 — DDL audit, multi-channel bogo (this is the headline) + +After both fixes (HTTP/2 windows + 8-way channel fan-out), bogo is +**faster than CRDB across the board**, both N=0 and N=5000. + +p50 latency in ms (warm envd, `--padding tables --reps 8`): + +| op | CRDB N=0 | bogo+mc N=0 | CRDB N=5000 | bogo+mc N=5000 | +| --- | ---: | ---: | ---: | ---: | +| create_table | 24.6 | **14.8** | 39.4 | **24.0** | +| drop_table | 15.1 | **11.0** | 31.2 | **21.9** | +| alter_table_add_col | 17.8 | **13.7** | 34.0 | **22.1** | +| rename_table | 13.7 | **10.5** | 29.9 | **20.1** | + +Bogo wins every cell. The win at N=0 is 20-40%; the win at N=5000 is +about the same in absolute terms (~8-15 ms), so the deltas hold even +as the catalog grows. The 5000-table pad itself took 184 s under bogo +(was 551 s pre-fix, 193 s under CRDB). + +Why this works despite the consensus_cas mean from envd's persist +client *not* having dropped (still ~11 ms): with multiple connections, +the slow ops in the long tail no longer block sibling RPCs on the +same connection. The p50 CAS is still sub-millisecond; the tail still +exists but it now runs concurrently with the rest of the DDL work +instead of serially gating it. Eliminating the head-of-line blocking +is the real fix even though the mean per-op number looks unchanged. + +### Done + +The original probe asked whether the residual post-design slope was +CRDB-bound. The first answer (pre-fix bogo: it's not, bogo is even +slower) was misleading because bogo itself was broken on the gRPC +client path. With bogo actually fast, the probe re-runs and confirms +the underlying signal: bogo cleanly beats CRDB on every op and every +scale point measured. The next round of catalog-side work can use +bogo as a clean low-floor reference. + +### 2026-05-18 — slope study at N=5000 / 10000 / 15000 (bogo + file blob) + +Used a small `bench_profile.py` driver (one-shot, lived under `/tmp`, +not checked in): pads the catalog incrementally with empty tables, +snapshots prometheus + envd RSS, then runs 100 reps of +`CREATE TABLE m_tmp (a int)` + `DROP TABLE m_tmp` while samply is +attached to envd. Blob backend `file://`; consensus backend +multi-channel bogo on loopback; timestamp oracle still CRDB. +`ALTER SYSTEM SET max_tables = 30000` and +`enable_alter_table_add_column = true` set up-front. + +Headline per-rep latencies (warm envd, 100 reps each): + +| N | create p50 | create p95 | create mean | drop p50 | drop p95 | envd RSS | +| ---: | ---: | ---: | ---: | ---: | ---: | ---: | +| 5000 | 25.9 ms | 69.9 ms | 34.3 ms | 23.3 ms | 32.8 ms | 1618 MB | +| 10000 | 37.4 ms | 84.6 ms | 45.0 ms | 32.6 ms | 78.3 ms | 2511 MB | +| 15000 | 52.7 ms | 125.7 ms | 67.6 ms | 45.0 ms | 105.3 ms | 3208 MB | + +Slope: `create_table` p50 adds ≈13 ms per +5000 user tables; p95 +grows faster (≈+28 ms per +5000), tail is widening. RSS grows ~700–900 +MB per +5000 = ~160 KB per pad table held in memory. + +#### envd-internal metric deltas during the 100 reps + +(differences between `/metrics` scrapes taken immediately before and +after each measurement window) + +| metric | N=5000 | N=10000 | N=15000 | +| --- | ---: | ---: | ---: | +| `catalog_transact_seconds{method="catalog_transact_with_ddl_transaction"}` mean | 31.95 ms | 41.37 ms | 60.44 ms | +| `catalog_transact_seconds{method="catalog_transact_with_side_effects"}` mean | 31.95 ms | 41.36 ms | 60.44 ms | +| `consensus_cas` count over 100 DDL reps | 5 744 | 8 450 | 12 100 | +| `consensus_cas` mean (across *all* shards in process) | 4.83 ms | 17.41 ms | 46.53 ms | +| `blob_set` count over 100 DDL reps | 702 | 761 | 766 | +| `blob_set` mean | 1.29 ms | 1.39 ms | 1.32 ms | +| `mz_catalog_syncs` count | 630 | 630 | 630 | +| `mz_catalog_transactions_started` count | 210 | 210 | 210 | +| `mz_catalog_transaction_commit_latency_seconds` ∑ | 0.513 s | 0.838 s | 1.200 s | +| `mz_catalog_sync_latency_seconds` ∑ | 0.481 s | 0.823 s | 1.234 s | +| `audit_log` collection entries (live) | 15 472 | 20 682 | 25 892 | +| `storage_collection_metadata` entries | 5 089 | 10 089 | 15 089 | +| `item` entries | 5 001 | 10 001 | 15 001 | + +`with_ddl_transaction` and `with_side_effects` track exactly together +because for a single-statement CREATE the outer just delegates to the +inner — no explicit DDL transaction. + +The numbers that **do not** scale with N are encouraging: + +* `blob_set` count and mean are flat (≈7 puts/DDL, ≈1.3 ms each). + Local file blob with `fsync` is not the slope. +* Counter deltas for `mz_catalog_syncs`, `mz_catalog_transactions_started`, + `mz_catalog_transaction_commits`, and + `mz_persist_state_fetch_recent_live_diffs_fast_path` are constant + across scale (6.3 syncs/DDL, 2.1 commits/DDL, ≈1.4 fast-path live-diff + fetches/DDL). The *number* of catalog operations per DDL is stable. + +The numbers that **do** scale with N (the smoking gun): + +* `consensus_cas` count per DDL: 57 → 85 → 121. Linear in N + (≈ +27 CAS per +5000 user tables). +* `consensus_cas` mean: 4.83 → 17.41 → 46.53 ms. **Super-linear** — + ratios are 3.6× then 2.7× while N only doubles then 1.5×'s. The mean + is bagging in tail samples whose individual latency grows worse than + linearly. +* `catalog_transact_with_ddl_transaction` mean: 31.95 → 41.37 → 60.44 ms. + Linear-ish (~9–19 ms per +5000), tracking the wall-clock slope. +* `mz_catalog_transaction_commit_latency_seconds` rises ~2× from N=5k + to N=15k, but commit is only ~5–12 ms of the 32–60 ms DDL — it's not + the dominant slope inside `catalog_transact`. + +Important caveat: `consensus_cas` is **per-RPC across all shards in +the process**, not just the catalog shard. The headline 4.83 → 46.53 +mean is dominated by *more*, *slower* CAS on the user-table shards — +each user table is a persist shard, with its own writer doing periodic +maintenance, and at N=15000 we have 3× more shards doing it. The +catalog-shard CAS that DDL actually waits on is just one entry in +that histogram. We'd need per-shard / per-kind labels on +`consensus_cas` to isolate it cleanly. + +#### Flame-graph picture at N=5000 (samply attached during the 100-rep window) + +Coarse self-CPU breakdown across all envd threads (custom +`app_frames.py` that buckets each sample by the deepest matching app +area on the stack root→leaf): + +| area | self % | +| ---: | ---: | +| tokio\_fs (fsync / open / rename on blob) | 64.07% | +| tokio\_runtime (scheduler / park-unpark) | 18.35% | +| mz\_persist\_client | 8.25% | +| tonic\_grpc | 3.83% | +| libc\_misc | 2.63% | +| mz\_storage\_controller | 0.83% | +| mz\_compute\_client | 0.80% | +| mz\_adapter::coord | 0.42% | +| alloc | 0.37% | +| mz\_adapter::catalog::transact / mz\_catalog | 0.22% | +| planner / optimizer | ~0% | + +Two surprises in this: + +1. **Almost no on-CPU work is in adapter / catalog code paths** + (`adapter_coord` 0.42%, `catalog_transact` 0.22%, planner 0%). The + DDL critical path is mostly *waiting*, not computing — coordinator + work serializes on awaits for persist / controller responses, so + the CPU profile doesn't tell us much about wall-time. +2. **Two thirds of the process's CPU during the audit is in + `tokio::fs::*` blocking-pool tasks** doing `fsync`, `__open64`, and + `rename` against the local-file blob backend. That's a property of + the `file://` blob URL used here; with `s3://` the CPU mix would + shift, but the wall-time critical path through `blob_set` (1.3 ms + per put, flat in N) would be similar. + +#### So what scales? Best current hypotheses + +1. **Catalog shard's persist state grows linearly with the number of + catalog updates.** Each CREATE / DROP TABLE writes one diff. The + single catalog shard accumulates history; every CAS apply has to + walk that history when reconstructing state on a sync. State + compaction / rollups eventually truncate it, but in the audit + window we're racing ahead of compaction. +2. **`apply_catalog_implications` and the in-memory catalog state + updates** still have some O(N)-per-DDL walks beyond the read-holds + path that was already fixed in `11be652bf3` (timeline read holds + made O(delta)). On-CPU under those frames is currently ~0.6% + combined, so the per-DDL CPU is small — but a 1 ms walk over + 15 000 entries does match the observed slope. +3. **Audit-log writes are appended into a persist shard whose state + grows linearly with N.** Every DDL appends a row to `audit_log`. + The shard's batch list / spine grows. Even though `blob_set` and + `consensus_cas` *counts* per DDL are stable, the per-op cost on + the `audit_log` shard rises with its history depth. +4. **Per-shard background traffic.** Each user table is a persist + shard; each shard does writer heartbeats / rollups / live-diff + fetches in the background. With more shards the *total* CAS rate + in the process is higher, and these background CAS sit in the + same histogram as the DDL ones, inflating the mean we see. + +(1) and (3) are state-machine cost on specific shards; (2) is +in-memory catalog walk cost on the coordinator; (4) is observation +bias on the histogram, not real DDL latency, but it's still real +work the process is doing. + +#### Next moves, in order + +1. **Add `shard_kind` labels to `mz_persist_external_op_latency`** — + break out the catalog shard, the `audit_log` shard, and "user + collections" separately. That tells us within minutes which kind + of shard is contributing the slope and which is observation bias. +2. **If catalog shard is the offender**: look at state apply cost in + the persist client — does the catalog shard hit + `state_apply_spine_slow_path` more as it grows? Check rollup + write cadence on the catalog shard at large state sizes. +3. **If audit-log shard is the offender**: aggressive truncation / + compaction of the `audit_log` shard. Old audit entries are read + rarely; we don't need to keep the full history hot. +4. **Independently**, hunt remaining O(N) walks per DDL inside + `catalog_transact_inner` / `apply_catalog_implications`. The + recent read-holds fix removed one; given on-CPU under those frames + is 0.6%, any remaining walks should be cheap CPU-wise but still + show up in wall time. + +#### Reproducing the run + +`/tmp/bench_profile.py` was intentionally not checked in — it's a +thin psycopg driver that does the padding / measurement loop and +shells out to `samply`. Sketch: + +* Spin up bogo on `:6882` and a `--persist-consensus-url=bogo://…` + envd against `file:///…/blob`. +* On the mz_system port: `ALTER SYSTEM SET max_tables = 30000; + ALTER SYSTEM SET enable_alter_table_add_column = true;`. +* For each scale point in `[5000, 10000, 15000]`: incrementally pad + via `CREATE TABLE IF NOT EXISTS audit_pad.pad_t_`; snapshot + `/metrics` and `/proc//status` VmRSS; start + `samply record -p -s -o /tmp/profile_N.json.gz`; run 100 + reps of `CREATE TABLE audit_meas.m_tmp (a int)` / + `DROP TABLE audit_meas.m_tmp` while timing each statement; + `SIGINT` samply; snapshot `/metrics` again. +* Analyze with `samply load ` for the flame graph, + and diff the before/after metrics scrapes for histogram deltas. + +Note that on this host `perf_event_paranoid` had to be lowered to 1 +and `perf_event_mlock_kb` raised to 128 MiB before samply could +attach. + +### 2026-05-18 — shard-attributed slope study (bogo + file blob) + +The previous slope study showed `consensus_cas` count and mean both +growing with N, but the single histogram couldn't tell us *which +shards* the slope came from. This run adds an investigation-only +metric `mz_persist_external_op_latency_by_shard_kind` (HistogramVec, +labels `[op, shard_kind]`) and a small in-process registry mapping +`ShardId -> shard_kind` populated at `Applier::new` time. The +shard_kind classifier is closed-set: + +| shard_name (from `Diagnostics`) | shard_kind | +| --- | --- | +| `catalog` | `catalog` | +| `txns` | `txns` | +| `builtin_migration` | `builtin_migration` | +| `expression_cache` | `expression_cache` | +| `storage-usage` / `storage_usage` | `storage_usage` | +| anything else | `user_data` | +| (pre-registration ops) | `unknown` | + +No samply this time: the new label is enough to attribute the slope +without flame-graph overhead. Ladder cut to N=5000 / 10000 (15000 +dropped) — the per-shard `file://` blob backend filled the host +disk on the longer run. + +#### Latency per rep (ms) + +| N | create_p50 | create_p95 | drop_p50 | drop_p95 | +| ---: | ---: | ---: | ---: | ---: | +| 5,000 | 29.8 | 73.9 | 28.2 | 35.9 | +| 10,000 | 46.5 | 113.0 | 41.9 | 91.5 | + +#### `consensus_cas` by shard_kind, per-DDL + +`count/DDL` = total CAS delta during the 100-rep window ÷ 100. +`mean ms` = total CAS time delta ÷ count, so it reflects only ops +that happened during the burst (not lifetime). Numbers are from one +clean run; the absolute mean values shift between runs but the +*shape* is stable. + +| op | kind | N=5k count/DDL | N=5k mean ms | N=10k count/DDL | N=10k mean ms | +| --- | --- | ---: | ---: | ---: | ---: | +| consensus_cas | catalog | 5.58 | 0.53 | 5.58 | 1.85 | +| consensus_cas | txns | 6.63 | 0.44 | 6.64 | 1.30 | +| consensus_cas | user_data | 43.21 | 6.76 | 69.83 | 38.72 | +| blob_set | catalog | 0.57 | 0.99 | 0.56 | 1.39 | +| blob_set | txns | 2.62 | 0.96 | 2.63 | 1.18 | +| blob_set | user_data | 3.77 | 1.11 | 3.86 | 1.06 | + +For comparison, the previous (unsharded) numbers from the same +ladder had `consensus_cas count` growing from 57 → 85 per DDL with +mean 4.83 → 17.41 ms. With the label, we can now see *that growth +is almost entirely `user_data`*: those CAS are background work on +the pre-existing user_data shards (compaction, GC, rollup writes), +not synchronous per-DDL work. + +#### What we learned + +The slope is NOT in user_data shard work. That work is mostly +asynchronous background activity on the 5,000-10,000 pre-existing +shards — high count but doesn't gate DDL completion. + +The slope IS in **catalog and txns shard CAS getting more expensive +per-op as those shards' states grow**: + +* catalog CAS count/DDL is flat at 5.58 across both scales — + per-DDL DDL doesn't generate more catalog CAS as N grows. +* catalog CAS mean **grows 3.5× from N=5k to N=10k** (0.53 → 1.85 ms). +* txns CAS count is similarly flat at ~6.6, mean grows 3× + (0.44 → 1.30 ms). +* The synchronous catalog+txns CAS budget per DDL therefore grew + from 2.96+2.92 = 5.9 ms at N=5k to 10.3+8.6 = 18.9 ms at N=10k, + i.e. **+13 ms of synchronous CAS work** out of +16.8 ms total + wall-time growth. That accounts for almost all the slope. + +This points at persist state-apply / rollup cost on the **catalog +shard**, with a parallel — and not actually surprising — slope on +the **txns shard**: every table is registered in the txns shard, +so its state size grows linearly with N. catalog and txns are thus +the same shape (singleton-shard state growing with table count), +not two distinct mysteries. + +Neither has anything to do with the user_data shards' state or how +many of them exist; both are about the *size* of two specific +singleton shards' own histories. + +#### Next moves + +1. **Persist-only microbenchmark.** Build a tiny harness *outside* + the envd/catalog stack that opens a single persist shard, writes + into it in a pattern that mimics what the catalog actually does + per DDL (small diff, occasional rollup, similar key/value + shapes), grows the shard's state to a target SeqNo / state-byte + size, then measures `compare_and_set` latency at that size. Run + the ladder cheaply at multiple sizes. Once that reproduces the + per-CAS slope outside envd, we can profile / trace / mutate it + in isolation — no clusterd, no CRDB-TS-oracle, no measurement + noise from 10k user_data shards doing background work. Likely + lives in `src/persist-client/src/bin/persist_cas_bench.rs` or a + sibling of `persist-client/examples/`. Same trick applied + separately to the txns-shard write pattern (`mz_txn_wal`) + isolates that slope too. + + **Important: run the microbench against BOTH consensus backends** + — real CRDB (`postgres://...?options=--search_path=consensus`) + and bogo-consensus (`bogo://127.0.0.1:6882`). The split tells us + *where* the per-CAS cost lives: + * If bogo flattens the slope but CRDB doesn't, the cost is below + persist — in the consensus impl's per-row / per-state-blob + handling as the shard's row in `consensus` table grows. + * If both show the same slope, the cost is *above* the consensus + trait — in persist's `Applier` / state-apply / diff-fetch / + rollup-cadence path, and we'd fix it in persist itself. + The prior end-to-end runs already use bogo, so we know bogo + shrinks total DDL latency. The microbench is the cleanest way + to factor whether bogo *changes the slope shape* or just lowers + the floor. +2. **Once the microbench reproduces the slope**: profile / trace + the slow CAS to see where time goes inside persist state apply + — spine slow-path, rollup write cadence, decode cost, encoded + diff size. The flame-graph from inside envd is too contaminated + by everything else to read cleanly; the microbench output + should be much sharper. +3. **Once we know *what* is slow**, fix it — likely some + combination of more aggressive rollups on the catalog and txns + shards (`mz_persist_state_apply_spine_slow_path` / + `mz_persist_shard_seqnos_since_last_rollup` are the relevant + counters to watch), or a cheaper apply path for the specific + diff pattern the catalog produces. +4. **De-prioritize**: user_data CAS volume is a non-issue for DDL + latency. It matters for *total CPU* but not for the wall-time + slope we've been chasing. + +#### Reproducing + +Scripts live in `/home/ubuntu/envd-ddl-investigation/` (driver +`bench.py`, analyzer `analyze.py`, launchers `start_envd.sh` / +`reset_state.sh`, cluster-replica JSON). Build with +`cargo build --profile=optimized` — full `--release` triggers an +LTO link that OOMs this 23 GiB VM (the earlier "VM went +unresponsive" was that, plus zero swap). Mitigations applied: + +* Added 8 GiB swapfile (`/swapfile`, swappiness=60). +* Cap CRDB container memory: `docker run --memory=2g`. +* Deleted `target/debug` and `target/release` before re-running + (the `file://` blob backend needs disk room too). +* Bench checkpoints results after each scale; if envd crashes + mid-padding, the previous scale's data is preserved on disk + (`results/timings_N{N}.csv`, `metrics/{before,after}_N{N}.prom`). + +### 2026-05-18 — persist-only CAS microbench (single-shard ladder) + +Built and ran the persist-only CAS microbench described in the +previous section's "Next moves" #1. Binary: +`src/persist-client/examples/persist_cas_bench.rs`. Driver scripts + +collected data live under `/home/ubuntu/envd-ddl-investigation/cas_bench/`. + +Each ladder rung opens a *fresh* shard, pre-fills it with `size` +catalog-shaped `compare_and_append`s (one small batch of one row per +iteration), then takes 200 timed `compare_and_append`s at that state +size. Ran the same ladder against: + +* `mem` consensus + `mem` blob — control, no I/O. +* `bogo://` consensus + `file://` blob — the same backend our envd + end-to-end study used. +* `postgres://` (CockroachDB v24.2) + `file://` blob — the real + production-shape consensus. + +Each backend was run twice: once with the production rollup cadence +(`persist_rollup_threshold = 128`) and once with rollups effectively +suppressed (`persist_rollup_threshold = 1_000_000`) so persist state +genuinely accumulates between rungs. + +#### Per-CAS latency (p50 ms, after pre-fill) + +| backend | rollup | N=0 | N=1k | N=2.5k | N=5k | N=10k | N=20k | +| --- | --- | ---: | ---: | ---: | ---: | ---: | ---: | +| mem + mem | default | 0.08 | 0.08 | 0.09 | 0.08 | 0.11 | 0.11 | +| bogo + file | default | 0.63 | 0.59 | 0.63 | 0.59 | 0.61 | 0.66 | +| bogo + file | none | 0.59 | 0.57 | 0.58 | 0.62 | 0.59 | — | +| crdb + file | default | 1.10 | 1.10 | 1.06 | 1.09 | 1.24 | 1.39 | +| crdb + file | none | 1.06 | 1.05 | 1.01 | 1.24 | 1.47 | — | + +(Full samples in `cas_bench/results.csv`; per-tag table in +`cas_bench/summary.md`.) + +#### What we learned + +**The single-shard CAS slope is essentially zero — under all three +backends, including with rollups disabled.** Per-CAS p50 grows by at +most ~0.4 ms (CRDB, 0→10k, no rollups) and is *flat* on bogo and mem. + +For comparison, the envd end-to-end study at the same scale showed +the `catalog`-shard's `consensus_cas` *mean* growing from 0.53 ms +(N=5k) to 1.85 ms (N=10k) — a +1.3 ms slope per CAS. **This +microbench does not reproduce that slope.** Whatever is making +catalog-shard CASes slow in envd is *not* "the consensus row / +state blob is bigger because we have more SeqNos." It survives +even when state accumulates without rollups. + +Specifically: + +* **Bogo flat → cost is not in the consensus implementation.** Even + with rollups disabled, bogo's per-CAS latency does not respond to + shard state size. If the slope were in the consensus row's + storage / row-count path, bogo would have shown some movement at + 10k–20k. +* **CRDB shows a tiny ~0.4 ms slope without rollups** which is the + expected cost of more rows accumulating in the `consensus` table + for a single shard (the `INSERT ... WHERE (SELECT ... ORDER BY + sequence_number DESC LIMIT 1)` plan has to skip more + PK-suffix-suffix rows as truncation falls behind). That's still + far short of the envd slope. +* **Mem + mem at 0.08 ms** confirms the persist-client overhead + itself is tiny — the rest is real I/O against the chosen backend. + +So the +13 ms catalog+txns slope we saw end-to-end is **not** simply +the cost of doing a CAS against a shard whose history is N seqnos +long. It comes from something the microbench is *not* exercising. + +#### Smoking gun in the existing metrics dump + +Going back to the envd `/metrics` snapshots from the previous run +and pulling counters the histogram doesn't expose, there's a clear +super-linear signal: + +| | N=5k window | N=10k window | +| --- | ---: | ---: | +| `state_apply_spine_flattened` Δ (per 100 reps) | 1,719 | 9,753 | +| per-DDL spine flattens | **17.2** | **97.5** | +| `cmd_cas_mismatch_count` (compare_and_append) | 2 (lifetime) | 2 (lifetime) | +| `shard_seqnos_since_last_rollup{name="catalog"}` | 121 | 30→72 (oscillating, normal) | + +Spine flattens per DDL grew **5.7×** as N doubled (17 → 97). Each +flatten is "rebuild the trace's spine from scratch" work that +happens during state apply — CPU on the persist-client side, *not* +the consensus RPC. `state_apply_spine_fast_path` stayed at 0 +throughout — every state apply is going through the flattened +(slow-ish) path. + +The retries counter (`cmd_cas_mismatch_count`) stayed at 2 across +both windows, so the slope is not from cas retries. The rollup +cadence on the catalog shard is normal (~120 seqnos between +rollups). What's blowing up is state-apply work *around* the CAS, +not the CAS itself. + +This reconciles the microbench result. The single-shard microbench +opens one shard, drives one shard's spine — flatten cost per CAS +is tiny because there's only one batch shape per timestamp. envd +has 10k user_data shards' worth of batches threading through the +*same* `Applier::apply_unbatched_cmd` code path on every state +apply, and the per-flatten cost grows with whatever is shared +(shared state cache, shared trace structures, allocator pressure, +or simply scheduling latency). That's why the slope shows up in +the `consensus_cas` *wall-time histogram* even though it isn't +"the CAS RPC got slower" — `MetricsConsensus` wraps the RPC future +with `metrics.consensus.compare_and_set.run_op(...)`, which times +the *whole future* including the time spent waiting on Tokio's +scheduler. Under shared-runtime pressure, that wait time *is* the +slope. + +#### Hypotheses for what the microbench is missing + +The microbench opens **one** shard and writes to it sustained. +envd at N=10k has: + +* ~10,000 user_data shards, all opened in the same `PersistClient`, + all doing their own background work (GC, snapshot reads, rollup + writes, compaction posting CAS) concurrently with the catalog + CAS we're trying to measure. +* A `txns` shard whose state grows linearly with table count (every + table is registered there), getting CASed on every DDL. +* A shared `StateCache` keyed by ShardId; lookup / update cost + could grow with shard count. +* A shared `IsolatedRuntime` and a shared gRPC connection pool to + the consensus server — contention on either could throttle + catalog CAS specifically. +* A shared `Metrics` (with our new `external_op_latency_by_kind`) + whose histograms see traffic from every shard. + +Any of these could be what makes a *catalog* CAS in envd at N=10k +take 1.85 ms when the same CAS in isolation takes 0.6 ms. The +microbench data rules out "row/state blob size" as the cause; what +remains is shared-resource contention or shared-cache work that +scales with shard count. + +#### Next moves (updated) + +1. **Multi-shard contention variant.** Extend `persist_cas_bench` + (or add a sibling) with a `--num-bg-shards N` flag: open N + background shards on the same `PersistClient` and have each one + trickle in writes (or just hold open handles to grow the state + cache), then measure the foreground catalog-shaped shard's CAS + latency. Same ladder against bogo + CRDB. If this reproduces + the slope, the bottleneck is shared per-client state (StateCache + contention, gRPC channel sharing, GC/compaction scheduling). +2. **Run the microbench *while envd is busy*.** Connect a fresh + `persist_cas_bench` process to the same bogo + same blob dir as + a running, populated envd and measure foreground CAS. This is + the closest possible reproduction without splitting the slope + into "client work" vs "server work." +3. **Inspect persist's internal counters in the envd metrics + dump** for catalog-shard signals the histogram doesn't see: + `mz_persist_state_apply_spine_slow_path_count`, + `mz_persist_shard_seqnos_since_last_rollup`, + `mz_persist_cmd_cas_mismatch_count` — these would tell us + whether catalog CAS in envd is bottlenecked on state-apply CPU + or on retries, not on the consensus RPC. +4. **The original next-step from above remains valid** but is now + second priority: profiling the bare CAS path won't help if the + per-CAS slope only shows up under multi-shard load. + +#### Reproducing this section + +``` +# Build (cheap, no LTO): +cargo build -p mz-persist-client --example persist_cas_bench --profile=optimized + +# Run a ladder against one backend: +target/optimized/examples/persist_cas_bench \ + --consensus bogo://127.0.0.1:6882 \ + --blob file:///home/ubuntu/envd-ddl-investigation/cas_bench/bogo \ + --sizes 0,500,1000,2500,5000,10000,20000 \ + --measurements 200 \ + --out /home/ubuntu/envd-ddl-investigation/cas_bench/results.csv \ + --tag bogo-file-default-rollup +``` + +Tag rows in the CSV so multiple runs can share one output file. +Use `--rollup-threshold 1000000` to suppress rollups. The +`summarize.py` script in `cas_bench/` produces the table above. + +### 2026-05-18 — fine-grained state_apply attribution, full envd rerun + +The microbench from the previous section ruled out "per-CAS scales +with state size" as the cause of the envd slope. To close the loop, +added `mz_persist_state_apply_latency_by_shard_kind` (HistogramVec +with labels `[stage, shard_kind]`, stages `total`/`flatten`/ +`unflatten`/`decode`) so each `State::apply_diff` invocation gets +timed and attributed to a shard_kind. Then re-ran the full +envd N=5k/N=10k bench with the new build. + +#### End-to-end DDL latency (re-run, same backend, fresh build) + +| N | create_p50 | create_p95 | drop_p50 | drop_p95 | create_mean | +| ---: | ---: | ---: | ---: | ---: | ---: | +| 5,000 | 29.27 | 74.77 | 26.71 | 39.55 | 34.05 | +| 10,000 | 47.41 | 120.24 | 42.85 | 48.81 | 61.02 | + +Create p50 slope: **+18.14 ms** from N=5k→N=10k. Matches the +previous run within run-to-run variance. + +#### `consensus_cas` mean per CAS, per shard_kind + +| N | catalog | txns | user_data | +| ---: | ---: | ---: | ---: | +| 5,000 | 0.54 ms | 0.85 ms | 6.45 ms | +| 10,000 | **2.48 ms** | 0.92 ms | **41.41 ms** | + +* Catalog CAS RPC: **+1.94 ms × 5.6 calls/DDL = +10.9 ms/DDL**. +* Txns CAS RPC mean is essentially flat (+0.07 ms × 6.65 = +0.5 ms). +* User_data CAS mean blew up 6× (6.45 → 41 ms) — but those are + off the critical path of any single DDL. Still telling: bogo + is taking ~40 ms per CAS on average for *background* shards at + N=10k. + +#### `state_apply_latency_by_shard_kind` — the new metric + +`total` count is the number of `apply_diff` invocations per +shard_kind in the bench window (×0.01 reps = per-DDL). Mean is +the per-call latency. + +| N | shard_kind | total count/DDL | total mean | flatten count/DDL | unflatten count/DDL | decode count/DDL | +| ---: | --- | ---: | ---: | ---: | ---: | ---: | +| 5,000 | catalog | **0.01** | 0.01 ms | 0 | 0 | 0.01 | +| 5,000 | txns | 0 | — | 0 | 0 | 0 | +| 5,000 | user_data | 92.14 | 0.01 ms | 80.64 | 80.64 | 16.20 | +| 10,000 | catalog | **335.70** | 0.01 ms | 332.56 | 332.56 | 1.42 | +| 10,000 | txns | **167.84** | 0.01 ms | 115.72 | 115.72 | 0.06 | +| 10,000 | user_data | 49.25 | 0.01 ms | 35.54 | 35.54 | 30.31 | + +**At N=10k the catalog shard does 335 apply_diff calls per DDL, +each ~10 µs.** At N=5k it does ~0. The txns shard makes the same +~0 → 168/DDL jump. Aggregated: + +* Catalog state-apply budget: 0 → **3.4 ms/DDL** (NEW slope + component, completely invisible to the previous "consensus CAS" + metric). +* Txns state-apply budget: 0 → **1.7 ms/DDL** (also new). + +99.9% of apply_diff calls finish in <64 µs (lowest bucket). flatten +is sub-precision; unflatten is the line item that actually adds up. +Decode count is small (~1–2/DDL on catalog, ~30/DDL on user_data) +so the StateDiff::decode cost is not the slope either. + +#### Full slope reconciliation + +Putting all per-DDL persist budgets side by side: + +| component | N=5k ms/DDL | N=10k ms/DDL | Δ ms/DDL | +| --- | ---: | ---: | ---: | +| catalog `consensus_cas` (5.6 × mean) | 3.04 | **13.91** | **+10.87** | +| txns `consensus_cas` (6.65 × mean) | 5.67 | 6.12 | +0.45 | +| catalog `consensus_scan` (2.1 × mean) | 3.28 | 2.79 | −0.49 | +| catalog `blob_set` | 0.79 | 0.65 | −0.14 | +| txns `blob_set` | 2.84 | 3.79 | +0.95 | +| catalog `state_apply` (total) | 0.00 | **3.36** | **+3.36** | +| txns `state_apply` (total) | 0.00 | **1.68** | **+1.68** | +| user_data `state_apply` (total) | 0.92 | 0.49 | −0.43 | +| **sum** | **16.54** | **32.79** | **+16.25** | + +Create-table p50 slope: +18.14 ms. Sum-of-persist-pieces slope: ++16.25 ms. **The remaining ~2 ms is run-to-run variance / minor +in-process work** (catalog walks, JSON re-encoding, etc.). The +persist-attributable slope is essentially the whole slope. + +#### What we learned + +1. **The slope is fully accounted for.** Catalog `consensus_cas` + RPC growth (+10.9 ms) + catalog state-apply (+3.4 ms) + txns + state-apply (+1.7 ms) + txns blob_set growth (+0.95 ms) ≈ + the whole +17–18 ms wall-time slope per DDL. No mystery + residual. +2. **There are two distinct cost growths.** The catalog `consensus_cas` + RPC itself got 4.6× slower (0.54 → 2.48 ms) — that's a server- + side cost (bogo + shared client / scheduler contention from 10k + user_data shards' background activity). And separately, the + *client*-side state-apply work grew from ~0 to 335 invocations + per DDL on the catalog shard. +3. **The state-apply count explosion is the more surprising one.** + The catalog shard's actual `cmd_succeeded` count stayed flat at + 5.6/DDL across both scales. So the catalog's true SeqNo only + advanced ~5.6 times per DDL. But we called `apply_diff` 335 + times. That means ~60 `apply_diff` invocations per real SeqNo + advance, each doing little work (~10 µs) but the aggregate + reaches 3.4 ms/DDL. Something is calling the state-apply path + far more often than the shard's history requires. +4. **Single-shard microbench can't reproduce this** because it + doesn't have the 10k user_data shards generating the + pubsub broadcast / scheduler load that drives the catalog + state-apply replay frequency up. + +#### Hypotheses for the apply_diff count explosion + +`apply_diff` only fires for diffs that pass the filter inside +`apply_encoded_diffs` (`x.seqno == state_seqno.next()`), so each +call is at least nominally trying to advance the local cache. +Plausible sources of the 60× per-CAS multiplier on catalog: + +* **`cache.rs::push_diff` (pubsub broadcast)** — each call applies + one diff. The catalog shard's `pubsub_diff_applied` count + delta is only ~2/window though, so this is *not* the source. +* **`apply.rs::fetch_and_update_state`** (line 669) — calls + `apply_encoded_diffs(diffs_to_current)`. The + `update_state_fast_path` counter delta is only ~0.57/DDL. + Even if each fast_path firing applied ~60 diffs, that gives + ~34/DDL — short of 335. +* **`state_versions.rs::fetch_current_state`** (line 466) — the + slow path, replays all live diffs from the latest rollup. + Would have to fire ~5/DDL on catalog applying ~67 diffs each + to explain the count. Need to instrument call sites to confirm. +* **`state_versions.rs:1140`** (`StateVersionsIter`) — used by + GC/audit/inspect to walk historical states. If GC on the + catalog walks N live states, each walk applies N diffs and + counts in our metric. + +The next surgical metric is to add a call-site label to the +`state_apply_latency_by_shard_kind` histogram (or a separate +counter), so we can split the catalog's 335/DDL by which of the +four call sites is responsible. That's a one-line dispatch per +site and falls out into the same bench harness. + +#### Next moves + +1. **Add a `source` label to the state-apply metric** (or a + separate counter `apply_diff_calls_by_source_and_kind` with + sources `cas_update`, `slow_refetch`, `pubsub_push`, + `state_iter`). Rerun the bench. Pinpoint the call site that + creates the 335/DDL on catalog. +2. **Investigate why catalog `consensus_cas` mean grew 4.6×** even + though the microbench shows no growth in bogo RPC at the same + state size. The likely answer is shared-client / scheduler + contention — many user_data shards' CASes queued up in front of + the catalog CAS on the bogo gRPC connection (the pool was + bumped to 50 channels recently, but at N=10k there may still be + head-of-line blocking). To confirm, instrument + `MetricsConsensus::run_op` with an *outer* timer that wraps the + future before Tokio scheduling, and compare against the inner + timer that wraps just the inner consensus call. +3. **Now that the slope is fully accounted for**, decide which of + the two pieces to fix first. The catalog `consensus_cas` + growth (+10.9 ms) is the bigger lever (2× larger than apply_diff + growth). It's likely also easier to fix — multi-channel fan-out + per shard, or a dedicated per-shard concurrency limit, would + stop the user_data CAS load from queuing in front of catalog + CASes. + +#### Reproducing + +* New build: `cargo build --profile=optimized --bin environmentd --bin clusterd --bin mz-bogo-consensus` +* `/home/ubuntu/envd-ddl-investigation/reset_state.sh` to wipe + blob / mzdata / scratch / CRDB. +* `/home/ubuntu/envd-ddl-investigation/start_envd.sh` to launch. +* `/home/ubuntu/envd-ddl-investigation/bench.py` runs the full + N=5k → N=10k ladder; ~10 minutes wall. +* `/home/ubuntu/envd-ddl-investigation/analyze.py` parses the + before/after `/metrics` snapshots and prints the per-shard_kind + breakdown tables above. + +### 2026-05-18 — apply_diff source attribution: it's GC + +Added a per-call-site `source` label to the apply_diff path +(`mz_persist_state_apply_calls_by_source_shard_kind` with +`[source, shard_kind]`). Four runtime sources are now distinguishable: + +* `cas_update` — `apply.rs::Applier::fetch_and_update_state` fast path. +* `slow_refetch` — `state_versions.rs::fetch_current_state` full + rollup+replay (the fast-path-fallback "we got fenced too far"). +* `pubsub_push` — `cache.rs::push_diff` from PubSub broadcasts. +* `state_iter` — `state_versions.rs::StateVersionsIter::next` walks + (used by GC + storage-usage audit + admin inspect). + +#### Three runs, one consistent story + +Ran the bench three times to characterize variance: run 1 (clean +reset, fresh envd), run 2 (re-measured on the same envd without +reset — bench `pad_to` is idempotent so this re-measures at N=10k +twice, kept as a contrast point), run 3 (clean reset again). + +| run | reset? | catalog apply_diff/DDL at N=5k | catalog apply_diff/DDL at N=10k | +| --- | --- | ---: | ---: | +| 1 | yes | 0 | 0.01 | +| 2 | no (warm) | 95.19 | 88.98 | +| 3 | yes | 0.02 | 172.29 | + +And the per-source breakdown for run 3 N=10k catalog: + +| source | count/DDL | +| --- | ---: | +| **`state_iter`** | **172.29** | +| `slow_refetch` | 0.71 | +| `cas_update` | 0.01 | +| `pubsub_push` | 0 | + +Run 2 N=5k catalog (warm-envd contrast): + +| source | count/DDL | +| --- | ---: | +| **`state_iter`** | **94.03** | +| `slow_refetch` | 1.14 | +| `pubsub_push` | 0.01 | +| `cas_update` | 0.01 | + +In every case where catalog apply_diff exists, **>99% of it is +`state_iter`**. The original mystery is solved: it's +`StateVersionsIter::next`, the per-diff walker used by GC. + +#### The GC fingerprint confirms it + +Run 3 N=10k window deltas on the catalog shard: + +| counter | before | after | Δ | +| --- | ---: | ---: | ---: | +| `shard_gc_finished{name="catalog"}` | 9 | 10 | **+1** | +| `shard_gc_live_diffs{name="catalog"}` | 19,676 | 17,229 | (gauge) | +| `shard_cmd_succeeded{name="catalog"}` | 33,661 | 34,221 | +560 | +| `shard_seqnos_since_last_rollup{name="catalog"}` | 24 | 69 | (gauge) | + +**Exactly one GC fired on the catalog during the 100-rep N=10k +window**, and that one GC walked **17,229 live diffs** (the +`gc_live_diffs` gauge after that GC). The `state_iter` counter +delta is 17,229 — the same number. One GC = one `fetch_all_live_states` += 17,229 `StateVersionsIter::next` calls, each one an `apply_diff` +on the catalog shard. + +For N=5k, GC didn't fire on the catalog during the measurement +window (`gc_finished` delta 0), so `state_iter` was 0. For run 1 +N=10k, GC also happened not to fire during the window. For run 2 +both windows happened to coincide with GC firings. + +#### What this means for the slope + +Per-call work is ~10 µs. **17,229 calls × 10 µs ≈ 172 ms** total +GC work on the catalog over the 100-rep window. That's 1.72 ms of +catalog GC work *per DDL of wall time*, but it runs on background +tasks, so its contribution to *DDL-critical-path latency* is at +most the Tokio-scheduler tax (single-digit %). + +Slope decomposition for run 3 (clean reset): + +| component | N=5k ms/DDL | N=10k ms/DDL | Δ ms/DDL | +| --- | ---: | ---: | ---: | +| catalog `consensus_cas` × 5.6 | 3.04 | **10.28** | **+7.24** | +| txns `consensus_cas` × 6.64 | 3.31 | 7.36 | +4.05 | +| catalog `consensus_scan` × 2.08 | 1.10 | 2.43 | +1.33 | +| txns `blob_set` × 2.63 | 2.66 | 3.51 | +0.85 | +| catalog `state_apply` (GC, all `state_iter`) | 0.00 | 1.72 | +1.72 | +| **sum** | **10.11** | **25.30** | **+15.19** | + +create_p50: 31.26 → 45.45 ms = **+14.19 ms**. Sum-of-persist-pieces: ++15.19 ms. Within noise. The slope is dominated by the CAS RPC +times growing on catalog (+7.2 ms) and txns (+4.0 ms), with GC +state-walk overhead a distant third (+1.7 ms). + +#### Run-to-run variance is high + +Comparing the three runs side-by-side, the *split* of the slope +across components changes a lot, even though the *total* slope is +consistently +14–18 ms: + +| run | catalog CAS Δ ms | txns CAS Δ ms | catalog state_apply Δ ms | +| --- | ---: | ---: | ---: | +| 1 | +10.87 | +0.45 | +0.00 | +| 3 | +7.24 | +4.05 | +1.72 | + +The previously reported "catalog state_apply slope of +3.4 ms" was +from run 1's *previous build* — a third run we can't compare +apples-to-apples to. The reproducible story is: + +1. **GC's `state_iter` walks ARE the source of all catalog/txns + apply_diff calls.** PubSub, cas_update, and slow_refetch + together contribute <1% of catalog calls in every run. +2. **Whether catalog state-apply appears as a slope component + depends on whether GC fires during the measurement window.** + When it fires, it walks all live diffs (17 k+ at N=10k) but the + work is cheap (~10 µs/call). Total contribution: 1–4 ms/DDL, + third-tier behind the two CAS RPC slopes. +3. **The dominant slope is catalog `consensus_cas` RPC time** — + +7–11 ms/DDL across runs. Second is txns CAS at +0–4 ms. + Together those are 70-90% of the DDL slope every time. +4. **The deeper question is why catalog CAS RPC itself slows + down.** The single-shard microbench (previous section) rules + out "consensus row gets bigger as state grows." The most + plausible remaining cause is shared-runtime / gRPC-pool + contention from the 10k user_data shards' background CAS + load (user_data `consensus_cas` mean was 35–44 ms at N=10k vs + 7 ms at N=5k — a 5× growth that head-of-line blocks every + other CAS on the same bogo gRPC connection pool). + +#### Next moves + +1. **Confirm the bogo head-of-line hypothesis** with an outer-vs- + inner timer split around `MetricsConsensus::run_op`: the outer + timer captures Tokio-scheduler + connection-acquire wait; the + inner timer captures only the RPC wire time. Slope in + outer-minus-inner = contention. +2. **The GC-walks-17k-diffs pattern is itself a backlog signal.** + With one envd running for ~30 minutes, GC fires every ~10 + minutes and finds a big backlog (17 k diffs). If GC ran more + often it'd find smaller backlogs each time. Worth checking + whether GC scheduling on the catalog shard scales appropriately + with command rate, or whether something throttles it. +3. **Source attribution is solved.** No follow-up metric needed + for this question. + +## 2026-05-18 — bogo's `update_state_metrics` was eating all the CAS slope + +### tl;dr + +The "catalog `consensus_cas` RPC mean grows 2-3× at N=10k" slope was a +bench artifact, not a Materialize issue. The bogo-consensus server's +`update_state_metrics` was iterating every shard's `Vec.len()` inside +the mutex on every CAS to recompute `versions_total`. At 10k shards +that's a ~100 µs O(N) hold on the lock that serializes every operation. +Removing the per-call iteration (incremental counters instead) makes +catalog CAS mean go FLAT across N=5k → N=10k. + +### Step 1: split outer (`MetricsConsensus::run_op`) from inner (gRPC wire) + +Added `mz_persist_consensus_wire_seconds_by_shard_kind`, recorded +inside `BogoConsensus` around `self.client.compare_and_set(...)`. Same +axes (op + shard_kind) and buckets as the existing +`external_op_latency_by_shard_kind`, so subtraction is meaningful. + +**Run before any other fix, N=5k → N=10k:** + +| layer | N=5k mean ms | N=10k mean ms | mean Δ ms | +| --- | ---: | ---: | ---: | +| catalog external_op (post-spawn, around run_op) | 0.77 | 1.79 | +1.02 | +| catalog wire (inside BogoConsensus around gRPC) | 0.77 | 1.79 | +1.02 | +| user_data external_op | 7.26 | 35.10 | +27.84 | +| user_data wire | 6.68 | 34.95 | +28.27 | + +**Outer === inner** to within sampling noise. The post-spawn wrapper +(`run_op` counter incs + the bogo adapter's status_to_external map) is +free. **The CAS slope is in the gRPC call itself.** This rules out +spawn-side overhead and any wrapping overhead inside MetricsConsensus. + +### Step 2: scrape the bogo server's `rpc_seconds` and notice it grows + +Enabled `--metrics-listen-addr` on the bogo binary and added a scrape +in `bench.py`. Server-side `mz_bogo_consensus_rpc_seconds` for +`compare_and_set` (aggregated across all shard kinds — the server +doesn't have a shard_kind classifier): + +| N | server compare_and_set mean ms | client wire user_data mean ms | +| --- | ---: | ---: | +| 5k | 0.60 | 6.68 | +| 10k | 2.15 | 34.95 | + +Server mean grew 3.6×. The bogo server holds a single +`std::sync::Mutex` around its `BTreeMap` for every op. Suspicious. + +### Step 3: read the server and find the smoking gun + +`src/bogo-consensus/src/server.rs::update_state_metrics`, called from +every `compare_and_set` (both Committed and ExpectationMismatch paths) +and every `truncate`: + +```rust +fn update_state_metrics(&self, store: &BTreeMap>) { + let shards = i64::try_from(store.len()).unwrap_or(i64::MAX); + let versions: i64 = store.values() + .map(|v| i64::try_from(v.len()).unwrap_or(i64::MAX)) + .sum(); + self.metrics.shards_total.set(shards); + self.metrics.versions_total.set(versions); +} +``` + +That `store.values().map(...).sum()` is **O(num_shards) under the +mutex on every CAS**. At N=10k with ~100 concurrent CAS in flight from +the 10k user_data shards' background work, the mutex queue depth +grows. Catalog CAS waits behind it. + +### Step 4: replace with incremental counters and rerun + +Fix: `bump_state_gauges(shards_delta, versions_delta)` called *after* +dropping the mutex. CAS that creates a new key bumps `shards` by 1; +every successful CAS bumps `versions` by 1; truncate decrements +`versions` by the count it removed. Constant time per call, no +iteration. + +**Same bench, fixed bogo:** + +| | N=5k mean | N=10k mean | mean Δ | +| --- | ---: | ---: | ---: | +| catalog wire mean ms | 0.27 | 0.29 | +0.02 | +| txns wire mean ms | 0.42 | 0.27 | -0.15 | +| user_data wire mean ms | 0.93 | 0.92 | -0.01 | +| server `compare_and_set` mean ms | 0.00 | 0.01 | +0.01 | +| create_p50 ms | 26.86 | 41.17 | +14.31 | + +Catalog and user_data CAS means are now **flat** across the scale +jump. The previous +28 ms/call user_data slope was 100% the bogo +metric-update O(N) artifact. The catalog +1 ms/call slope was the same +artifact contending on the shared mutex. + +### What remains of the create_p50 slope (+14.31 ms/DDL) + +With CAS basically free, the per-DDL slope decomposes to: + +- catalog state_apply (this run GC walked 20k catalog diffs): +2.02 ms +- catalog `consensus_scan`: +0.82 ms (mean 0.45 → 0.84 ms × 2.08/DDL) +- catalog blob_set: +0.06 ms +- catalog `consensus_cas`: +0.12 ms +- txns `consensus_cas`: -1.0 ms (decreased) +- **Sum of measured CAS+blob+scan+apply slope: ~+2 ms/DDL** +- `catalog_transact_seconds` slope: +9.87 ms/DDL +- create_p50 slope: +14.31 ms/DDL + +So `catalog_transact_seconds` itself has a +9.87 ms slope but only ~+2 ms +of that comes from persist external ops we instrument. The other +8 ms +is inside the catalog transact path between persist calls +(catalog state munging, builtin migration checks, etc.). That's the +next layer to investigate if we want to keep peeling. + +The +4.4 ms gap between catalog_transact slope (+9.87) and create_p50 +slope (+14.31) is outside catalog_transact — in adapter coordination +or driver-side. + +### Takeaways + +1. **The "catalog consensus_cas grows with N" headline was a bogo + artifact, not a Materialize finding.** Bogo's per-CAS work was O(N) + in the number of shards because of an in-mutex metric update. +2. **Wire == outer.** No measurable overhead inside + MetricsConsensus's `run_op` wrapper or the bogo adapter — the + slope was always in the actual gRPC call. +3. **With the fix, bogo is a much better CRDB proxy.** Per-call mean + stays flat from N=5k to N=10k for all shard kinds. The remaining + DDL-level slope is in adapter/catalog code paths, not persist. +4. **Bench correctness lesson:** anything that proxies a production + service for perf work needs to itself be O(1) in the dimension + being scaled. `update_state_metrics` looked innocent but actively + distorted every comparison since the bogo work started. + +## 2026-05-18 — CRDB-backed sanity check at N=5k/10k/15k + +### tl;dr + +Re-ran the bench against CRDB consensus (same machine, same envd +binary, just `--persist-consensus-url=postgres://…/consensus`) at +three scale points. The **Materialize-side slope reproduces on +CRDB**: ~+15 ms/+5k tables on create p50, basically the same shape +we see on post-fix bogo. CRDB adds a modest extra ~2-5 ms/+5k on top +because its catalog CAS RPC mean grows mildly with state size +(1.88 → 2.11 → 3.80 ms across 5k → 10k → 15k); bogo's was flat. + +So the post-fix bogo conclusion holds: the dominant slope is in +adapter/catalog code, not persist. Switching backends doesn't move +that slope. + +### Bench setup + +- `start_envd_crdb.sh` — same as the bogo flavour but with + `--persist-consensus-url=postgres://root@localhost:26257/materialize?options=--search_path=consensus`. +- `bench.py` + `analyze.py` take a `BENCH_MODE` env var that + suffixes `metrics_/` and `results_/`. Bogo data lives + in `metrics_bogo/` / `results_bogo/`; CRDB data in + `metrics_crdb/` / `results_crdb/`. +- 100 reps × CREATE+DROP at each scale point. Padding is + incremental (5k → 10k → 15k = 15k total CREATE TABLEs). +- Resource ceiling held throughout: envd RSS topped at 2.9 GiB at + N=15k; CRDB container stayed under 1 GiB. No memory pressure. + +### Headline: create_table p50 by backend + +| N | bogo p50 | CRDB p50 | Δ (CRDB-bogo) | +|---------|---------:|---------:|--------------:| +| 5 000 | 26.86 | 54.82 | +27.96 | +| 10 000 | 41.17 | 72.24 | +31.07 | +| 15 000 | — | 87.47 | — | + +**Slope per +5k tables:** + +- bogo: +14.31 ms (5k → 10k) +- CRDB: +17.42 ms (5k → 10k), +15.23 ms (10k → 15k) + +The Materialize-side slope (the part bogo is also paying) is ~14 +ms/+5k. CRDB adds ~2-3 ms on top of that. + +CRDB sits ~28 ms above bogo at every scale — that's a flat +"CRDB tax" from the actual consensus RPCs being ~2 ms each instead +of <0.5 ms. ~5.6 catalog CAS + 6.6 txns CAS = 12 RPCs/DDL × (2 ms - +0.3 ms) ≈ +20 ms; close enough to the +28 we observe. + +### CAS per-call means: bogo stays flat, CRDB drifts + +`mz_persist_external_op_latency_by_shard_kind` mean for `consensus_cas`: + +| | bogo 5k | bogo 10k | CRDB 5k | CRDB 10k | CRDB 15k | +|--------|--------:|---------:|--------:|---------:|---------:| +| catalog| 0.27 | 0.29 | 1.88 | 2.11 | 3.80 | +| txns | 0.42 | 0.27 | 2.02 | 2.68 | 2.27 | +| user_data| 0.93 | 0.92 | 8.28 | 33.24 | 79.57 | + +- bogo: catalog/txns CAS mean is flat across scales. This was the + whole point of the `update_state_metrics` fix. +- CRDB catalog: grows ~2× from 5k to 15k. Counts are unchanged + (5.6 per DDL), so this is per-RPC slowdown — CRDB is doing more + work per CAS as the consensus table grows. Plausibly index + size, query plan, or just SQL parsing/round-trip overhead under + load. +- CRDB user_data: the big numbers are dominated by background + compaction load (count 49 → 124 per DDL); ignore for the + create-path discussion. + +### catalog_transact_seconds tracks create p50 + +CRDB `catalog_transact_with_ddl_transaction` mean: + +| N | mean (ms) | slope per +5k | +|---------|----------:|--------------:| +| 5 000 | 53.38 | — | +| 10 000 | 70.15 | +16.77 | +| 15 000 | 89.67 | +19.52 | + +So roughly the entire create_p50 slope is inside `catalog_transact` +on CRDB too — same conclusion as bogo. + +### Padding throughput tells the same story + +CREATE TABLE rate during the pad phase, end of each segment: + +- N=5k: 19.1 tbl/s +- N=10k: 15.6 tbl/s +- N=15k: 11.9 tbl/s + +Roughly 1/p50: a 5k-table-rich envd is doing ~52 ms/CREATE during +padding vs ~85 ms at 15k. Same slope. + +### Takeaways + +1. **The Materialize-side scaling slope is real and backend- + independent.** Going from bogo to CRDB doesn't make it go away; + CRDB just shifts the absolute floor up and adds a mild extra + per-CAS cost. +2. **CRDB has its own mild CAS-mean slope** (catalog 1.88 → 3.80 ms + across 5k → 15k). Probably worth a follow-up to confirm whether + that's the consensus table index growth or SQL-side, but it's a + secondary effect at these scales. +3. **The bogo work was the right setup.** Now that its in-mutex + metric update is fixed, bogo CAS mean is flat across the range, + so anything bogo still shows as slope is genuinely Materialize- + side. The CRDB run confirms that the bogo slope reproduces on a + real backend. +4. **15k is well within budget on this machine** — envd hit 2.9 + GiB RSS, CRDB stayed under 1 GiB. We can keep going if needed. + +## 2026-05-18 — Splitting `catalog_transact` into phases + +tl;dr: the `catalog_transact_with_ddl_transaction` slope is **mostly +outside** `Catalog::transact`. The inside-transact slope is real but +modest (~3-5 ms/+5k). The outside-transact slope is ~6-10 ms/+5k and +lives somewhere in `Coordinator::catalog_transact_inner` — the +wrapper that does `Arc::make_mut(catalog)`, calls +`catalog.transact`, then ships builtin-table updates and runs the +finalize block. + +### What we added + +A new histogram, `mz_catalog_transact_phase_seconds{phase=...}`, +that times each phase inside `Catalog::transact`: + +* `transact_inner` — total time inside the inner method (a + cross-check / super-timer for the four phases below). +* `op_loop` — the for-each-op loop body (`transact_op` + per-op + `preliminary_state.apply_updates`). +* `final_apply_updates` — the combined `apply_updates` call on the + final state, after the op loop. +* `prepare_state` — `storage_collections.prepare_state(...)` + (storage controller side). +* `post_prepare_apply_updates` — the second final `apply_updates` + after `prepare_state`, draining any new tx updates that emerged. +* `tx_commit` — `tx.commit(&mut **storage, oracle_write_ts)` + (the persist CAS path). +* `assign_state` — `self.state = new_state` (drops old `CatalogState`). + +Wired through `Catalog` as `Option`, set once from +`Coordinator` startup. `transact_incremental_dry_run` doesn't get +the metric — DDL-txn dry runs are a different code path and +polluting the measurement bucket would muddy the bench. + +### Headline timings (bogo backend, fresh-from-scratch) + +`mz_catalog_transact_seconds{method="catalog_transact_with_ddl_transaction"}` +mean (ms, per DDL = one CREATE *or* one DROP, 200 obs/scale): + +| N | mean (ms) | slope per +5k | +|---------|----------:|--------------:| +| 5 000 | 33.80 | — | +| 10 000 | 42.81 | +9.01 | +| 15 000 | 57.81 | +15.00 | + +Create p50 (CREATE side only) tracks: 34.31, 43.91, 60.37. + +### Phase split — mean per single DDL + +`mz_catalog_transact_phase_seconds`, mean over 200 observations/scale: + +| phase | 5k ms | 10k ms | 15k ms | Δ 5→10 | Δ 10→15 | +|-----------------------------|------:|-------:|-------:|-------:|--------:| +| transact_inner (total) | 2.04 | 3.14 | 5.79 | +1.10 | +2.64 | +| ↳ op_loop | 1.07 | 1.53 | 2.19 | +0.46 | +0.66 | +| ↳ final_apply_updates | 0.51 | 0.72 | 1.04 | +0.21 | +0.32 | +| ↳ prepare_state | 0.04 | 0.19 | 1.41 | +0.15 | +1.22 | +| ↳ post_prepare_apply_upd. | 0.17 | 0.29 | 0.45 | +0.12 | +0.16 | +| tx_commit | 2.47 | 3.79 | 5.51 | +1.32 | +1.72 | +| assign_state | 0.34 | 0.62 | 0.99 | +0.28 | +0.37 | +| **inside-transact sum** | 4.85 | 7.55 | 12.29 | +2.70 | +4.74 | +| outside-transact remainder | 28.95 | 35.26 | 45.52 | +6.31 | +10.26 | +| `catalog_transact_with_ddl` | 33.80 | 42.81 | 57.81 | +9.01 | +15.00 | + +(Children of `transact_inner` sum to ~80-90% of the parent; the gap +is small per-phase Cow setup, lock acquisition, mode match — not +worth its own metric.) + +### Takeaways + +1. **The dominant slope is outside `Catalog::transact`.** Of the + +9 ms/+5k jump from N=5k→10k, only +2.7 ms is in the timed + phases; +6.3 ms is in the Coordinator wrapper layer. At + 10k→15k it gets worse: +4.74 inside, +10.26 outside. The + "+8 ms unattributed" that motivated this iteration is the + *outside* component, not something hidden inside `transact_inner`. +2. **`tx_commit` is the biggest inside-transact slope component** + (~half of the inside-transact rise). The catalog CAS RPC mean + is flat (we fixed bogo's update_state_metrics earlier), so + tx_commit's growth has to be in serialization, batching, or + the txns/user_data CAS work that runs synchronously inside + `tx.commit`. +3. **`prepare_state` has a hockey-stick at 15k** — 0.04 → 0.19 → + 1.41 ms. The storage_controller's `prepare_state` does + per-collection bookkeeping; at 15k user collections, something + in there is starting to bite. Worth a dedicated look. +4. **`op_loop` and `final_apply_updates` grow modestly** — both + accumulate cost from in-memory state-diff application. This + matches our earlier finding that catalog state-apply does + ~335 invocations per DDL. +5. **`assign_state` grows linearly** — 0.34 → 0.62 → 0.99 ms. + This is dropping the old `CatalogState`; the cost is proportional + to state size. Cheap per-DDL but not zero. + +### Wrapper-layer suspects (outside `Catalog::transact`) + +`Coordinator::catalog_transact_inner` does, in order: + +* Pre-walk ops to classify them (cheap). +* `validate_resource_limits(&ops, ...)` — O(ops). +* `Arc::make_mut(catalog)` — **if any other holder of the catalog + Arc exists, this clones the entire `Catalog` (≈ full + `CatalogState` clone).** Highly suspect — would scale linearly + with N. Catalog Arcs are held by every active session for catalog + snapshots, so under any concurrent activity this can fire. +* `catalog.transact(...)` — the part we now have phase metrics for. +* `cluster_replica_statuses` updates (no per-table loop, cheap). +* `builtin_table_update().execute(builtin_table_updates)` — writes + rows into mz_objects, mz_tables, etc. Scales with the number of + builtin tables touched by the DDL, which grows with N via the + derived/dependent rows. +* The finalize block (configs, replanning) — only fires for + system-config ops, not bare CREATE/DROP TABLE. +* Segment audit-log dispatch (no-op in this bench). + +Then the outer wrappers `catalog_transact_with_side_effects` / +`catalog_transact_with_ddl_transaction` add +`apply_catalog_implications` (controller side effects) and the +side-effects-fut join. + +The biggest two on-paper suspects are: +* `Arc::make_mut(catalog)` cloning the catalog on every DDL. Need + to confirm there's a second Arc holder during a typical CREATE + TABLE. +* `builtin_table_update().execute(...)` writing per-object rows; + the table row count grows ~linearly with N. + +### Where we'd go next + +Add a second phase histogram around the **outside** layer: + +* `coord_pre_transact` — from method entry to `catalog.transact()`. +* `coord_arc_make_mut` — wrap just the `Arc::make_mut(catalog)` call. +* `coord_post_transact` — from `catalog.transact()` end through + builtin-table execute. +* `coord_finalize` — finalize block. +* `coord_apply_implications` — outer wrapper's + `apply_catalog_implications` call. + +That should split the +6-10 ms/+5k outside slope into named pieces. +Also worth a peek at `prepare_state` in `storage_controller` to +explain the 15k hockey-stick. + +## 2026-05-18 — Wrapper-layer phase split: `builtin_table_update().execute` is the slope owner + +tl;dr: outside-transact slope is almost entirely +`builtin_table_update().execute()`. At N=15k it's 16.85 ms/DDL — +nearly half the total DDL latency — and it grows ~4.3 ms per +5k +tables. `Arc::make_mut(catalog)` and the `finalize` block are +essentially free; both are ruled out. + +### What we added + +Six new `mz_catalog_transact_phase_seconds{phase=...}` labels for +`Coordinator::catalog_transact_inner` (the wrapper layer): + +* `coord_inner_total` — entire method (cross-check super-timer). +* `coord_pre_transact` — entry → just before `catalog.transact()` + (op pre-walk, validate_resource_limits, get_local_write_ts, + Arc::make_mut). +* `coord_arc_make_mut` — wraps just `Arc::make_mut(catalog)` to + isolate the Catalog-clone-if-shared cost. +* `coord_post_transact` — just after `catalog.transact()` → + method return (cluster_replica_statuses, builtin_table_execute, + finalize, audit). +* `coord_builtin_table_execute` — wraps just + `self.builtin_table_update().execute(builtin_table_updates).await`. +* `coord_finalize` — the bool-gated finalize block (config updates, + webhook restarts, advance_timelines refresh). + +### Headline timings (bogo backend, fresh-from-scratch) + +`mz_catalog_transact_seconds{method="catalog_transact_with_ddl_transaction"}`: + +| N | mean (ms) | slope/+5k | +|---------|----------:|----------:| +| 5 000 | 36.45 | — | +| 10 000 | 44.72 | +8.27 | +| 15 000 | 58.92 | +14.20 | + +create p50: 35.85 → 46.07 → 62.25 (tracks the same slope). + +### Phase split — mean per single DDL + +| phase | 5k | 10k | 15k | Δ 5→10 | Δ 10→15 | +|--------------------------------|------:|------:|------:|-------:|--------:| +| coord_inner_total | 17.32 | 25.11 | 34.06 | +7.79 | +8.95 | +| ↳ coord_pre_transact | 3.24 | 3.23 | 3.51 | -0.01 | +0.28 | +| ↳ coord_arc_make_mut | 0.00 | 0.00 | 0.00 | 0 | 0 | +| ↳ Catalog::transact (sum)* | 4.97 | 8.02 | 11.93 | +3.05 | +3.91 | +| ↳ coord_post_transact | 8.27 | 12.41 | 16.86 | +4.14 | +4.45 | +| ↳ coord_builtin_table_exec | 8.26 | 12.40 | 16.85 | +4.14 | +4.45 | +| ↳ coord_finalize | 0.00 | 0.00 | 0.00 | 0 | 0 | +| apply_catalog_implications | 11.39 | 11.16 | 13.51 | -0.23 | +2.35 | +| `catalog_transact_with_ddl` | 36.45 | 44.72 | 58.92 | +8.27 | +14.20 | + +(*) Catalog::transact = transact_inner + tx_commit + assign_state +(plus a small per-stage gap), per the previous phase split. + +### What this tells us + +1. **`builtin_table_update().execute()` is the single biggest + slope component on the outside layer.** It contributes +4.14 + and +4.45 ms per +5k tables — essentially *half* of the entire + per-DDL slope on its own. At N=15k it's 16.85 ms, ~29% of the + 58.92 ms total per-DDL latency. +2. **`Arc::make_mut(catalog)` is essentially zero** at all scales. + The Catalog Arc is uniquely held while we're inside + `catalog_transact_inner`, so the make_mut hot path doesn't + trigger a clone. Original hypothesis ruled out. +3. **`coord_pre_transact` is flat** (~3.2 ms regardless of N). + The op pre-walk + resource-limit validation + write-ts grab + don't scale with N. Good — we can ignore these. +4. **`coord_finalize` is ≈ 0** for plain CREATE/DROP TABLE. + The bool-gated config/tracing/etc. updates only fire for + system-config ops. Not a suspect. +5. **`apply_catalog_implications` is mostly flat** — 11.4, 11.2, + 13.5 ms across scales. It's *big* (≈ 1/4 of the per-DDL total) + but doesn't carry the slope. + +So the slope budget at 10k→15k splits roughly: +* `coord_builtin_table_execute`: +4.45 ms +* `Catalog::transact` (tx_commit + transact_inner + assign_state): +3.91 ms +* `apply_catalog_implications`: +2.35 ms +* everything else (`coord_pre_transact` drift, gap): +3.49 ms + +`coord_inner_total` minus its named children leaves a ~0.84 ms +(5k) → 1.45 ms (10k) → 1.76 ms (15k) gap — that's +cluster_replica_statuses updates + segment audit + setup overhead. +Cheap per-DDL but not flat. Probably not worth chasing yet. + +### Inside `builtin_table_update().execute()` + +Reading `src/adapter/src/coord/appends.rs::execute`, the call is: + +```rust +self.coord.pending_writes.push(PendingWriteTxn::System { updates, ... }); +let write_ts = self.coord.group_commit(None).await; +self.coord.advance_timelines_interval.reset(); +``` + +So the time is **`Coordinator::group_commit(None).await`**. That's +where pending_writes get flushed to persist as table appends. The +size of `builtin_table_updates` per DDL is small (one or two rows +per builtin system table touched), so the growth has to be inside +`group_commit` itself — likely from iterating something that +scales with the number of tables (table advancement, upper bumps, +collection bookkeeping). + +### Where we'd go next + +1. **Instrument inside `group_commit`** — split the upper-advancement, + table-append, and bookkeeping phases. We've already got a metric + `mz_group_commit_table_advancement_seconds`; pair it with one + for the per-DDL append cost. +2. **Look at `prepare_state` 15k hockey-stick** (0.04 → 0.19 → + 1.41 ms from the previous run). That's storage_controller side, + not coord. +3. **Catalog::transact internal slope** (+3-4 ms/+5k) is still + non-trivial. Mostly tx_commit and transact_inner growth from + prior iterations — not on the critical path of "where does + the headline slope live," but a real number. + + +## 2026-05-18 — Group-commit fix lands: `coord_builtin_table_execute` is now flat + +`coord: remove O(n) table advancement loop from group_commit` +(5d2d138108) removed the per-DDL `for table in catalog.entries().filter(is_table)` +loop in `group_commit()` that iterated every catalog entry to push an +empty append. With this gone, `group_commit()` no longer touches the +catalog when there are no user writes, so its cost is roughly fixed +per call instead of growing with `len(tables)`. + +Re-ran fresh-from-scratch bench at N=5k/10k/15k (`BENCH_MODE=phase4`) +against the same bogo backend, same envd build profile, same +SETTLE_S=5, REPS=100. + +### Headline (CREATE TABLE create_p50, ms) + +| N | phase3 (pre-fix) | phase4 (post-fix) | Δ | +|--------|-----------------:|------------------:|-------:| +| 5 000 | 35.80 | 30.21 | −5.59 | +| 10 000 | 46.02 | 36.48 | −9.54 | +| 15 000 | 62.23 | 47.94 | −14.29 | + +Slope per +5k: +* phase3: +10.22 / +16.21 ms +* phase4: +6.27 / +11.46 ms (−38% / −29% slope reduction) + +### `coord_builtin_table_execute` — the loop's old home + +Mean per-call (ms): + +| N | phase3 | phase4 | Δ | +|--------|-------:|-------:|-------:| +| 5 000 | 8.26 | 3.80 | −4.46 | +| 10 000 | 12.40 | 3.91 | −8.49 | +| 15 000 | 16.85 | 4.59 | −12.26 | + +Slope per +5k inside the timer: +* phase3: +4.14 / +4.45 ms +* phase4: +0.11 / +0.68 ms (essentially flat — fix confirmed) + +The remaining ~0.7 ms of slope at 10→15k is plausibly drift in +`get_local_write_ts`, the catalog upper advance CAS, and append +construction inside the leaner `group_commit`. Not worth chasing +on its own. + +### Full phase4 split — mean per single DDL (ms; CREATE+DROP averaged) + +| phase | 5k | 10k | 15k | Δ 5→10 | Δ 10→15 | +|--------------------------------|------:|------:|------:|-------:|--------:| +| coord_inner_total | 13.32 | 16.95 | 23.88 | +3.63 | +6.93 | +| ↳ coord_pre_transact | 3.03 | 3.13 | 3.67 | +0.10 | +0.54 | +| ↳ coord_arc_make_mut | 0.00 | 0.00 | 0.00 | 0 | 0 | +| ↳ transact_inner | 2.24 | 3.97 | 6.65 | +1.73 | +2.68 | +| ↳ op_loop | 1.13 | 1.81 | 2.66 | +0.68 | +0.85 | +| ↳ final_apply_updates | 0.60 | 0.93 | 1.36 | +0.33 | +0.43 | +| ↳ prepare_state | 0.02 | 0.28 | 1.11 | +0.26 | +0.83 | +| ↳ post_prepare_apply_updates| 0.20| 0.39 | 0.63 | +0.19 | +0.24 | +| ↳ tx_commit | 2.55 | 3.90 | 5.37 | +1.35 | +1.47 | +| ↳ assign_state | 0.43 | 0.83 | 1.28 | +0.40 | +0.45 | +| ↳ coord_post_transact | 3.80 | 3.91 | 4.59 | +0.11 | +0.68 | +| ↳ coord_builtin_table_exec | 3.80 | 3.91 | 4.59 | +0.11 | +0.68 | +| ↳ coord_finalize | 0.00 | 0.00 | 0.00 | 0 | 0 | +| apply_catalog_implications | 10.80 | 10.06 | 11.59 | −0.74 | +1.53 | +| append_table_duration | 5.08 | 4.90 | 5.30 | −0.18 | +0.40 | +| `catalog_transact_with_ddl` | 31.87 | 34.52 | 45.12 | +2.65 | +10.60 | + +(`apply_catalog_implications` and `append_table_duration` are per-call; +both are histograms; both fire twice per rep — once for CREATE, once +for DROP — but the mean above is the per-call mean.) + +### Where the slope now lives + +`coord_inner_total` accounts for +6.93 ms of the +11.46 ms create_p50 +slope (10→15). Its dominant children: + +* **`transact_inner`: +2.68 ms** — split across `op_loop` (+0.85), + `prepare_state` (+0.83, hockey-stick), `final_apply_updates` (+0.43), + `post_prepare_apply_updates` (+0.24). Mostly downstream of "catalog + state grew, so per-op apply_updates does more work." +* **`tx_commit`: +1.47 ms** — durable catalog commit. Catalog persist + shard CAS itself stays flat (~0.3 ms x 5.6 calls/DDL ≈ 1.7 ms), so + the slope is in state-apply / GC downstream of consensus, which we + already attributed in phase3 via the `apply_diff` `source` label. +* **`assign_state`: +0.45 ms** — just `self.state = new_state`. + +`coord_pre_transact` drifted from flat (+0.10) at small N to +0.54 +at 10→15. That's the op pre-walk + `validate_resource_limits` + the +`get_local_write_ts` await. Probably contention growing slightly with +shard count. + +Outside `coord_inner_total`, the remaining headline slope (~+4.5 ms) +has to come from the post-inner concurrent-join layer: + +* CREATE uses `catalog_transact_with_side_effects` (sequential + `apply_catalog_implications` then await `table_updates_notify`). +* DROP uses `catalog_transact_with_context` (concurrent join). + +For CREATE, `catalog_transact_with_ddl_transaction` measures +**+10.60 ms** at 10→15. After subtracting `coord_inner_total` per +call (which is the CREATE-only number, somewhere around 33 ms — we +only have the CREATE+DROP-averaged 23.88) and the apply_implications +per call (+1.53), the residue is small. The slope is fully +attributable to the named phases. + +### Where to go next + +The fix successfully removed the biggest slope contributor. +What remains is fairly evenly spread across `transact_inner`, +`tx_commit`, and `apply_catalog_implications`. The single biggest +*absolute* per-DDL cost is now `apply_catalog_implications` at +~11.6 ms — even though its slope is small, that's ~25% of a single +CREATE TABLE at N=15k. A sub-phase split of `apply_catalog_implications` +would tell us whether the cost is in the implication-batching loop, +the controller calls (`create_table_collections`, +`initialize_storage_collections`), or the inner match arms. That's +the next investigation. + + +## 2026-05-18 — `apply_catalog_implications` phase split: it's the controller setup + +Added a new `mz_apply_catalog_implications_phase_seconds{phase}` +histogram with six labels: + +* `absorb_updates` — the leading implication-batching loop (before + calling `apply_catalog_implications_inner`) +* `inner_total` — the whole `apply_catalog_implications_inner` call +* `inner_item_loop` — the `for (catalog_id, implication) in implications` + walk +* `inner_cluster_loops` — cluster + cluster-replica command loops +* `inner_controller_setup` — `create_source_collections`, + `create_table_collections`, `initialize_storage_collections`, + vpc-endpoint queueing, alter_* connection / source-desc / data-config / + ingestion-source-desc batches +* `inner_dependency_scan` — sink / peek / copy cleanup for dropped + relations + global-timeline association rebuild +* `inner_finalize` — the "no error returns" async block: actual + drop_tables / drop_sources / drop_sinks / drop_replicas, peek + copy + cancellation, retire_compute_sinks, plus the spawned + drop_replication_slots/secrets task + +Fresh-from-scratch bench at N=5k/10k/15k (`BENCH_MODE=phase5`). + +### Per-call mean (ms), CREATE+DROP averaged (count = 2/rep) + +| phase | 5k | 10k | 15k | Δ 5→10 | Δ 10→15 | +|-----------------------------|------:|------:|------:|-------:|--------:| +| `absorb_updates` | 0.01 | 0.01 | 0.01 | 0 | 0 | +| `inner_item_loop` | 0.01 | 0.01 | 0.02 | 0 | 0.01 | +| `inner_cluster_loops` | 0.00 | 0.00 | 0.00 | 0 | 0 | +| **`inner_controller_setup`**| **8.16**| **8.56**| **9.85**| **+0.40** | **+1.29** | +| `inner_dependency_scan` | 0.01 | 0.01 | 0.01 | 0 | 0 | +| `inner_finalize` | 1.56 | 1.71 | 1.81 | +0.15 | +0.10 | +| `inner_total` | 9.76 | 10.31 | 11.70 | +0.55 | +1.39 | + +### What this tells us + +* **`inner_controller_setup` is both the dominant absolute cost AND + the dominant slope inside `apply_catalog_implications`.** It is + 84% of `inner_total` and carries ~93% of the slope. +* `inner_finalize` is a fixed ~1.7 ms — that's the DROP TABLE path's + `drop_tables` (which goes through txn-wal). It does not scale. +* Everything else is rounding noise. The implication-batching loop, + the per-item match arms, the cluster loops, the dependency-scan + loops over `active_compute_sinks` / `pending_peeks` / `active_copies` + — all microseconds in this workload. They will not become a problem + until we actually have user-cluster activity. + +### CREATE-only attribution + +`inner_controller_setup` fires for both CREATE and DROP (count = 2/rep), +but only CREATE does work there — DROP runs through `inner_finalize` +instead. So the CREATE-only cost is the mean × 2: + +| phase (CREATE-only) | 5k | 10k | 15k | +|-----------------------------|------:|------:|------:| +| `inner_controller_setup` | 16.32 | 17.13 | 19.70 | + +CREATE pays ~**20 ms** at N=15k in `apply_catalog_implications`'s +controller-setup phase. That's a single call into +`create_table_collections`, which does: + +1. `get_local_write_ts()` — get a register timestamp +2. `self.catalog.advance_upper(write_ts.advance_to)` — CAS on the + catalog persist shard (already measured separately at ~0.6 ms via + `mz_group_commit_catalog_upper_seconds`) +3. `set_statement_execution_timestamp` loop — cheap +4. `self.controller.storage.create_collections(...)` — opens a fresh + persist `WriteHandle` + `SinceHandle` for the new table shard, + then `compare_and_downgrade_since` to advance the new shard's + since to `register_ts`. This is the meaty piece. +5. `self.apply_local_write(register_ts)` — finalize on timeline + +DROP-only `inner_finalize` is ~3.6 ms at N=15k, almost entirely +`self.drop_tables(...)` calling `controller.storage.drop_tables`, +which goes through the txn-wal append path. + +### Where to go next + +The fix removed the wrapper-layer scaling. Inside +`apply_catalog_implications`, the dominant slope is now in +**storage-controller `create_collections`** during a CREATE TABLE. +That's a single call but it does several distinct operations: +opening write+since handles, downgrading the since, installing +collection state into the controller's BTreeMap. Worth splitting +those phases next. + + +## 2026-05-18 — `create_table_collections` phase split: storage.create_collections owns it + +Added four sub-phase labels on `mz_apply_catalog_implications_phase_seconds` +that wrap each operation inside `create_table_collections`: + +* `create_table_write_ts` — `self.get_local_write_ts().await` +* `create_table_advance_upper` — `catalog.advance_upper(advance_to)` +* `create_table_storage_create_collections` — + `controller.storage.create_collections(...)` +* `create_table_apply_local_write` — `self.apply_local_write(register_ts)` + +These are emitted **once per CREATE TABLE** (not per DROP — DROP does +not enter this code path). + +### Per CREATE TABLE (ms; CREATE-only, count = 1/rep) + +| sub-phase | 5k | 10k | 15k | Δ 5→10 | Δ 10→15 | +|------------------------------------------|------:|------:|------:|-------:|--------:| +| `create_table_write_ts` | 3.27 | 3.62 | 3.61 | +0.35 | −0.01 | +| `create_table_advance_upper` | 0.56 | 0.61 | 0.71 | +0.05 | +0.10 | +| **`create_table_storage_create_collections`** | **8.16**|**8.98**|**11.57**|**+0.82**|**+2.59** | +| `create_table_apply_local_write` | 2.88 | 3.12 | 3.29 | +0.24 | +0.17 | +| Sum of named sub-phases | 14.87 | 16.33 | 19.18 | +1.46 | +2.85 | +| `inner_controller_setup` (CREATE-only) | 16.18 | 18.07 | 21.58 | +1.89 | +3.51 | + +The 1.3 → 2.4 ms gap between the sum and the parent timer is the +`set_statement_execution_timestamp` loop + the +`storage_metadata = self.catalog.state().storage_metadata()` fetch ++ general overhead between the named steps. + +### What this tells us + +1. **`controller.storage.create_collections` is the dominant slope + owner inside CREATE TABLE.** It carries +2.59 ms of the +3.51 ms + `inner_controller_setup` slope at 10→15 (74%) and is the single + biggest absolute cost at 11.57 ms/CREATE at N=15k. That's a single + call into the storage controller that opens persist `WriteHandle` + and `SinceHandle` for the new table shard, downgrades the new + shard's critical since to `register_ts`, and installs the collection + into the in-memory controller state. +2. **`get_local_write_ts` is a hidden 3.6 ms tax per CREATE.** That's + a synchronous timestamp-oracle round-trip happening *after* the + one already done at the top of `catalog_transact_inner`. Flat + slope, but absolute cost is 18% of `inner_controller_setup`. + Worth understanding why a CREATE TABLE needs two `get_local_write_ts` + calls when the previously-acquired `oracle_write_ts` should already + be valid for the register-ts purpose. +3. **`apply_local_write` is another flat 3 ms.** This bumps the local + timeline read frontier so the new table is immediately readable. + We probably need it, but it's another sync await. +4. **`catalog.advance_upper` is just 0.7 ms.** It's a single catalog-shard + CAS, and slope is tiny (+0.1 / +5k). It's the 2nd `advance_upper` + call per CREATE — the first one is inside `group_commit` to keep + `mz_catalog_raw` readable at the oracle ts. + +### CREATE TABLE end-to-end picture at N=15k + +create_p50 = 48.55 ms. Per-CREATE attribution: + +| component | ms | +|------------------------------------------|-------:| +| coord_inner_total (CREATE estimate) | ~30 | +| ↳ Catalog::transact (transact_inner+tx_commit+assign_state)| ~14 | +| ↳ coord_pre_transact | ~4 | +| ↳ coord_post_transact (builtin_table_execute) | ~5 | +| ↳ gap | ~7 | +| **apply_catalog_implications (CREATE)** | **~25** | +| ↳ inner_controller_setup | 21.58 | +| ↳ get_local_write_ts | 3.61 | +| ↳ catalog.advance_upper | 0.71 | +| ↳ storage.create_collections | 11.57 | +| ↳ apply_local_write | 3.29 | +| ↳ gap | 2.40 | +| ↳ inner_finalize | ~0 | +| ↳ everything else | ~3 | +| append_table_duration_seconds (concurrent)| ~5 | + +For CREATE the sequence in `catalog_transact_with_side_effects` is +synchronous: `catalog_transact_inner` → `apply_catalog_implications` +→ join(empty_side_effect, table_updates_notify). The table_updates +write happens concurrent with apply_implications. + +### Where to go next + +The biggest remaining levers, in order of opportunity: + +1. **`storage.create_collections` (slope owner)** — 11.57 ms at N=15k, + +2.59 ms per +5k. Worth splitting into the named operations inside + `storage_collections.create_collections_for_bootstrap`: + `open_data_handles` (concurrent stream of write+since handle + opens), `compare_and_downgrade_since` for tables, the collection + sort, and the `install_collection_states` post-loop. We already + have `info_span!` annotations there — pair them with metrics. +2. **The two oracle round-trips per CREATE + (`get_local_write_ts` + `apply_local_write`)** — flat ~6.5 ms + combined. Not slope-driving but a structural cost. Unclear if + both are required for correctness; needs design review. +3. **Storage controller's installation loop** — `acquire_read_holds` + in `create_collections_for_bootstrap` per-collection might be + where the slope lives (read holds touch shared `BTreeMap`s sized + to N). + +## 2026-05-18 — `create_collections_for_bootstrap` phase split: it's flat, slope moved + +Phase 6 attributed the +2.59 ms / +5k slope inside CREATE TABLE to +`controller.storage.create_collections`. Phase 7 splits that call into +two layers of sub-phases: + +``` +StorageController::create_collections_for_bootstrap + ├─ storage_collections_call ─→ StorageCollections::create_collections_for_bootstrap + │ ├─ validate_and_enrich + │ ├─ open_persist_client (cached) + │ ├─ open_data_handles_concurrent ← stream of (SinceHandle + WriteHandle) opens + │ ├─ sort + │ ├─ install_collection_states (under collections mutex) + │ └─ synchronize_finalized_shards + ├─ open_persist_client (cached, distinct from the inner one) + ├─ open_data_handles_concurrent ← stream of WriteHandle opens only + ├─ register_loop (per-collection, acquire_read_holds, insert into self.collections) + ├─ init_source_statistics + ├─ table_register (persist_table_worker.register → txns.register CaS) + ├─ append_shard_mappings + └─ run_to_execute (no-op for tables) +``` + +Two new HistogramVecs registered (commit `1c2f6ac975`): + - `mz_storage_collections_create_collections_phase_seconds{phase}` + - `mz_storage_controller_create_collections_phase_seconds{phase}` + +### Phase 7 data: storage.create_collections is flat, ~9 ms/CREATE + +Per CREATE TABLE at N=5k / 10k / 15k: + +**storage_collections layer:** + +| phase | 5k | 10k | 15k | slope (+5k→10k / 10k→15k) | +|--------------------------------|-----:|-----:|-----:|---------------------------:| +| `open_data_handles_concurrent` | 6.29 | 5.02 | 5.32 | −1.27 / +0.30 (flat) | +| `install_collection_states` | 0.19 | 0.53 | 1.54 | +0.34 / +1.01 (slope) | +| `open_persist_client` | 0.02 | 0.02 | 0.02 | flat | +| sort, validate, synchronize | <0.01 each | + +**storage_controller layer:** + +| phase | 5k | 10k | 15k | slope | +|--------------------------------|-----:|-----:|-----:|---------------------------:| +| `storage_collections_call` | 6.52 | 5.59 | 6.90 | −0.93 / +1.31 (variance) | +| `open_data_handles_concurrent` | 1.19 | 1.19 | 0.77 | flat | +| `table_register` | 1.88 | 2.24 | 1.55 | variance | +| `register_loop` | 0.03 | 0.03 | 0.03 | flat | +| `append_shard_mappings` | 0.01 | 0.01 | 0.01 | flat | +| `run_to_execute` | 0.00 | 0.00 | 0.00 | no-op for tables | + +**Total `storage.create_collections` per CREATE:** + - N=5k: 6.52 + 1.19 + 1.88 = 9.59 ms + - N=10k: 5.59 + 1.19 + 2.24 = 9.02 ms + - N=15k: 6.90 + 0.77 + 1.55 = 9.22 ms + +It's flat. The phase 6 +2.59 ms / +5k figure was likely run-to-run +variance on a single N=15k data point — the underlying CaaS work +inside the storage layer doesn't grow with shard count. + +### `install_collection_states` is the only slope inside storage_collections + +`install_collection_states` did grow (0.19 → 0.53 → 1.54 ms, ++0.67 ms / +5k by N=15k). That's a small but real slope. It's the +post-stream loop that: + 1. Takes the `self.collections` mutex. + 2. For each collection: determines dependencies, builds CollectionState, + inserts into the BTreeMap, calls `register_handles` (which does an + unbounded channel send to the background task), and calls + `install_collection_dependency_read_holds_inner`. + +For tables there are no dependencies, so this is essentially: +`BTreeMap::insert + register_handles + channel send`. The 1.5 ms +suggests either the BTreeMap is hot enough to be slow at N=15k, or +there's contention on the mutex with background tasks. Probably not +worth fixing unless we hit a Coordinator with much higher N. + +### `create_collections_for_bootstrap` is no longer the story + +The phase 5 finding that `inner_controller_setup` was the slope owner +of `apply_catalog_implications` was correct **at that snapshot**, but +phase 7 shows it's also flat now: + +`apply_catalog_implications` phase split (CREATE only, ms/call): + +| phase | 5k | 10k | 15k | slope | +|-----------------------------------------|-----:|-----:|-----:|------:| +| `inner_total` |11.44 |11.56 |11.82 | flat | +| `inner_controller_setup` | 9.43 | 9.44 | 9.72 | flat | +| `create_table_storage_create_collections`| 9.68 | 9.11 | 9.31 | flat | +| `create_table_write_ts` | 3.83 | 3.98 | 3.99 | flat | +| `create_table_apply_local_write` | 3.31 | 3.49 | 3.55 | flat | +| `create_table_advance_upper` | 0.60 | 0.59 | 0.58 | flat | +| `inner_finalize` | 1.97 | 2.08 | 2.06 | flat | + +### Where the slope actually lives: Catalog::transact + +The `mz_catalog_transact_phase_seconds` split at N=5k/10k/15k +(per-call mean, ms; ×2 for per-DDL since CREATE+DROP both go +through this): + +| phase | 5k | 10k | 15k | slope at 10→15 | +|-----------------------------|-----:|-----:|-----:|---------------:| +| `coord_inner_total` |15.12 |18.82 |24.30 | **+5.48** | +| ↳ `transact_inner` | 2.39 | 3.91 | 6.43 | **+2.52** | +| ↳ `tx_commit` | 2.92 | 4.03 | 5.54 | **+1.51** | +| ↳ `op_loop` | 1.19 | 1.81 | 2.64 | **+0.83** | +| ↳ `coord_pre_transact` | 3.53 | 3.81 | 3.90 | flat | +| ↳ `coord_post_transact` | 4.49 | 4.72 | 4.88 | flat | +| ↳ `coord_builtin_table_execute`| 4.48 | 4.71 | 4.88 | flat | +| ↳ `final_apply_updates` | 0.64 | 0.97 | 1.40 | +0.43 | +| ↳ `assign_state` | 0.44 | 0.83 | 1.25 | +0.42 | +| ↳ `post_prepare_apply_updates`| 0.21| 0.40 | 0.62 | +0.22 | +| ↳ `prepare_state` | 0.04 | 0.17 | 0.87 | +0.70 | + +Per-call slope is +5.48 ms / +5k inside `coord_inner_total`. Doubled +(CREATE + DROP): **+10.96 ms / +5k**. That matches the observed +create_p50 slope (33 → 39 → 48 ms = +6.27 / +9.11). + +### CREATE TABLE end-to-end picture at N=15k (phase 7) + +create_p50 = 48.93 ms (was 48.55 in phase 6 — same). + +| component (per CREATE) | ms | +|-------------------------------------------|-------:| +| `coord_inner_total` | 24.30 | +| ↳ `transact_inner` | 6.43 | +| ↳ `tx_commit` | 5.54 | +| ↳ `op_loop` | 2.64 | +| ↳ apply_updates phases (final + post_prepare + assign + prepare) | ~4.1 | +| ↳ coord_pre/post_transact | ~7.8 | +| `apply_catalog_implications` (CREATE) | 11.82 | +| ↳ `inner_controller_setup` | 9.72 | +| ↳ `create_table_storage_create_collections` | 9.31 | +| ↳ open_data_handles_concurrent | 5.32 | +| ↳ install_collection_states | 1.54 | +| ↳ controller open_handles + table_register + others | ~2.4 | +| ↳ `create_table_write_ts` | 3.99 | +| ↳ `create_table_apply_local_write` | 3.55 | +| append_table_duration_seconds (concurrent)| ~5 | + +### What this means + +1. **`create_collections_for_bootstrap` is no longer the slope owner.** + It's flat at ~9 ms per CREATE across 5k–15k. No fix is needed at + this layer; the per-CREATE storage work simply doesn't scale with + the number of already-registered shards. +2. **The slope has moved to `Catalog::transact`.** Specifically: + - `transact_inner` (+2.5 ms / +5k) + - `tx_commit` (+1.5 ms / +5k) + - `op_loop` (+0.8 ms / +5k) + - `apply_updates` family (final + post_prepare + assign + prepare + totalling ~+1.8 ms / +5k) +3. **CREATE and DROP both go through this path**, so the per-DDL + slope from Catalog::transact alone is ~+11 ms / +5k, which fully + accounts for the observed create_p50/drop_p50 slope. + +### Where to go next + +The next investigation iteration belongs **inside Catalog::transact**: + +1. **`transact_inner` outer** (+2.5 ms / +5k) — its inner phases are + already split by `op_loop`, `final_apply_updates`, `prepare_state`, + `post_prepare_apply_updates`. Sum of those at N=15k is ~5.5 ms, + but `transact_inner` measures 6.43 ms — there's an unaccounted + ~1 ms in `Self::transact_inner` (cow-cloning state, the initial + `extract_expressions_from_ops`, the storage_collections fields, + etc). +2. **`tx_commit`** (+1.5 ms / +5k) — this is the catalog durable + commit. The catalog state grows with N, so the persist state-apply + on the catalog shard grows too. The phase 5 finding ("335 apply_diff + per DDL on the catalog shard") is still very much alive here. + Worth instrumenting which sub-step inside `tx_commit` carries the + slope: persist CaS, state-apply, or the subsequent `apply_updates` + round. +3. **`apply_updates` (collectively +1.8 ms / +5k)** — runs over the + updates list, which for a CREATE TABLE has a small constant number + of items. The slope must be inside `apply_update` for one of those + StateUpdateKinds (Item, Storage, Cluster, etc.) — likely from + `generate_builtin_table_update` consulting per-N state, or from a + per-N lookup inside `apply_role_update`/`apply_item_update`/etc. + +Of these, `tx_commit` is the highest-confidence O(N)-eliminator: it's +a persist-shard apply, which we already know grows because the diff +log grows. Compaction of the catalog shard should bound this — but +only if compaction is keeping up. + +## 2026-05-19 — `tx_commit` phase split + consolidate fix + +Phase 7 attributed the +1.51 ms/+5k slope inside `tx_commit` (per call; ++3 ms/+5k per DDL since CREATE+DROP both hit this path) to the catalog +durable commit. Phase 8 instrumented two sub-phase histograms inside +the catalog persist crate: + + `mz_catalog_commit_transaction_phase_seconds{phase}` + caa_fence_check / caa_encode / caa_persist_caa_inner / + caa_persist_compare_and_append / caa_since_downgrade / caa_post_sync + + `mz_catalog_sync_phase_seconds{phase}` + listen_fetch / apply_updates / consolidate + +(All these histograms aggregate per `sync_inner` call: each sample is +the total time spent in that phase across all listen events that one +call processes. There are ~3 sync_inner calls per catalog +tx_commit — one inside the CaA itself, one after, one from the +`sync_updates(upper)` drain — and 2 tx_commits per DDL (CREATE + +DROP), so `count/DDL` for the sync phases is ~6.) + +### Phase 8: consolidate is the slope owner + +Per DDL (sum across all sync_inner calls): + +| phase | N=5k | N=10k | N=15k | slope (+5k→10k / 10k→15k) | +|-----------------|------:|------:|------:|---------------------------:| +| listen_fetch | 0.20 | 0.31 | 0.36 | +0.11 / +0.05 | +| apply_updates | 0.16 | 0.24 | 0.22 | flat | +| **consolidate** | 3.82 |11.20 |14.37 | **+7.38 / +3.17 (slope)** | + +`consolidate()` was running unconditionally at the end of every +`sync_inner` call, doing O(N log N) work on the entire snapshot +(15k+ entries at N=15k). It did this even when the sync only +processed one timestamp adding ~5-10 entries — paying the full +sort+dedup cost for a trivial delta. + +The doubling-threshold `maybe_consolidate` (added in #36233 to +amortize this exact cost) was *never triggering* on the hot path +because `sync_inner` reset `size_at_last_consolidation = None` at +the top of every call. Resetting re-baselined the threshold against +the current snapshot size, and a single DDL never grows the snapshot +by 2× — so `maybe_consolidate` inside the loop did nothing, and the +unconditional `self.consolidate()` after the loop ate the entire +cost every time. + +### The fix (`00d31c5be5`) + +Two-line conceptual change in `sync_inner`: + +1. Drop `self.size_at_last_consolidation = None` at the top. + The doubling threshold is meant to amortize across the snapshot's + lifetime, not per `sync_inner` invocation. +2. Replace the unconditional `self.consolidate()` at the end with + `self.maybe_consolidate()`. + +Result: `consolidate` only fires when the snapshot has actually +doubled since the last consolidation, keeping memory bounded at 2× +the consolidated size while making the typical per-call cost O(K log K) +where K is the delta (not O(N log N) on every commit). + +The two existing tests still pass: + - `test_persist_sync_consolidation_not_quadratic` (asserts < 10 + consolidations during a 100-ts sync) — still passes; with the + persistent threshold ~7 consolidations fire across the doubling + sweep. + - `test_persist_sync_snapshot_stays_bounded_under_churn` (200 DB + renames; asserts peak unconsolidated growth stays bounded) — + still passes; per-ts `maybe_consolidate` keeps growth bounded. + +### Phase 9: the fix lands + +Per-DDL `consolidate` time: + +| N | pre-fix | post-fix | +|-------|--------:|---------:| +| 5000 | 3.82 | **0.00** | +| 10000 | 11.20 | **0.00** | +| 15000 | 14.37 | **0.00** | + +(0.00 because it never triggered — the snapshot at N=15k starts at +~15k entries and the bench's 200 tx_commits add only ~2k more, which +doesn't double the threshold.) + +`tx_commit` per call dropped to flat: + +| N | pre-fix | post-fix | +|-------|--------:|---------:| +| 5000 | 2.92 | **1.24** | +| 10000 | 4.03 | **1.13** | +| 15000 | 5.54 | **1.13** | + +`create_p50` end-to-end: + +| N | phase 7 (pre) | phase 9 (post) | savings | +|-------|--------------:|---------------:|--------:| +| 5000 | 33.57 | 31.34 | −2.23 | +| 10000 | 39.82 | 38.70 | −1.12 | +| 15000 | 48.93 | **44.51** | **−4.42** | + +Slope at 10k→15k: **+9.11 → +5.81 ms/+5k tables (35% reduction)**. + +### What's left + +The residual slope (+5.81 ms/+5k at 10k→15k in create_p50) still lives +in `Catalog::transact` but no longer in the durable-commit path. Phase 9 +attribution per call: + +| phase | N=5k | N=15k | slope (10→15) | +|-------------------------|------:|------:|--------------:| +| `transact_inner` outer | 2.20 | 6.74 | +2.68 | +| `op_loop` | 1.14 | 2.57 | +0.61 | +| `final_apply_updates` | 0.58 | 1.37 | +0.35 | +| `prepare_state` | 0.02 | 1.25 | +1.17 (jumped at 15k) | +| `assign_state` | 0.39 | 1.17 | +0.39 | +| `tx_commit` | 1.24 | 1.13 | flat (FIXED) | + +The slope has moved entirely to the in-memory state-apply paths +(`apply_updates` family) and `transact_inner` outer. Next iteration +target: profile `CatalogState::apply_updates` — there's a per-update +walk somewhere that scales with N. Likely candidates: `generate_builtin_table_update` +for some StateUpdateKinds (e.g. shard mapping), or one of the +`apply_*_update` arms that consults global state. + +The `prepare_state` jump at N=15k (0.08 → 1.25 ms/call) is also +worth a look — that's +`storage_collections.prepare_state(tx, create, drop, register)`, +which calls `insert_collection_metadata` etc. on the durable txn. + +## Phase 10: drill into `CatalogState::apply_updates` + +Phase 9's `mz_catalog_transact_phase_seconds` split showed the residual ++5.81 ms/+5k slope at 10k→15k lives in the in-memory state-apply +path inside `Catalog::transact`. The umbrella for all that work is +`CatalogState::apply_updates`, called multiple times per transact +(op_loop's per-op apply, `final_apply_updates`, `post_prepare_apply_updates`). +We had no visibility into which sub-step of `apply_updates` was responsible. + +### Instrumentation (commit `00025cb9c2`) + +Two new histograms wired through `CatalogState`: + +* `mz_catalog_apply_updates_phase_seconds{phase}`: one observation + per `apply_updates` call for each of `consolidate_initial`, + `sort_per_group`, `apply_updates_inner`, `cleanup_notices`. +* `mz_catalog_apply_update_kind_seconds{kind}`: one observation per + applied `StateUpdate`, labeled by `StateUpdateKind` variant (item, + schema, storage_collection_metadata, etc.). Per-update events wrap + the full `parse_state_update + generate_builtin_table_update + + apply_update` triplet, so the timing captures everything we do + for that update. + +### Results (phase 10 bench) + +`mz_catalog_apply_updates_phase_seconds` (ms/DDL): + +| phase | 5k | 10k | 15k | slope (10→15) | +|------------------------|------:|------:|------:|--------------:| +| `apply_updates_inner` | 3.23 | 5.58 | 7.52 | **+1.94** | +| `cleanup_notices` | 0.02 | 0.02 | 0.02 | flat | +| `consolidate_initial` | 0.01 | 0.01 | 0.01 | flat | +| `sort_per_group` | 0.05 | 0.06 | 0.06 | flat | + +The slope is concentrated entirely in `apply_updates_inner` — the +kind-dispatched loop. `consolidate_initial` (per-call +`consolidate_updates`) and `sort_per_group` are noise at these sizes. + +Per-kind split (`mz_catalog_apply_update_kind_seconds`): + +| kind | 5k | 10k | 15k | slope (10→15) | +|-----------------------------|----------:|----------:|----------:|--------------:| +| `item` | 2.34 (4×586µs) | 4.21 (4×1051µs) | **5.75 (4×1436µs)** | **+1.54 ms/DDL** | +| `storage_collection_metadata` | 0.34 (2×171µs) | 0.74 (2×371µs) | 1.13 (2×567µs) | +0.39 ms/DDL | +| `audit_log` | 0.04 | 0.05 | 0.05 | flat | +| `unfinalized_shard` | 0.01–0.06 | … | … | flat | + +(Numbers in parens are `count/DDL × mean_per_event`.) Two findings: + +1. The dominant slope owner is **`StateUpdateKind::Item`**: +850 µs + per `apply_item_update`+`pack_item_update`+`parse_item_update` + triplet per +10k tables, on 4 calls/DDL. Accounts for ~80% of + the `apply_updates_inner` slope. + +2. **`storage_collection_metadata`** has a clean ~+200 µs per call + per +5k tables on 2 calls/DDL — consistent with full O(N) clone + of `StorageMetadata.collection_metadata` (a plain + `BTreeMap` wrapped in `Arc`). The + `preliminary_state`/`state` Cow pattern in `transact_inner` + forces `Arc::make_mut` to deep-clone this BTreeMap once per + independently-owned `CatalogState`. + +### Root cause hypothesis for the item slope + +`apply_item_update` Addition path is dominated by +`with_enable_for_item_parsing` → `deserialize_item` → `insert_entry`. +`insert_entry` calls `self.get_schema_mut(...)` which walks +`database_by_id.get_mut(...).schemas_by_id.get_mut(...)` — +both `imbl::OrdMap`s. `get_mut` on a shared `imbl::OrdMap` does +path-copy of the affected B-tree leaf, **cloning every value in +that leaf**, not just the targeted one. + +Critically, `Schema` (the leaf value type) embeds three +*non-persistent* `BTreeMap`s: + +```rust +pub struct Schema { + pub items: BTreeMap, + pub functions: BTreeMap, + pub types: BTreeMap, + ... +} +``` + +At N=15k the audit_pad schema's `items` map has 15k entries. The B-tree +leaf containing audit_pad and audit_meas almost certainly fits in a +single `imbl::OrdMap` chunk, so every `apply_item_update` (which +mutates `audit_meas`, not `audit_pad`) path-copies that leaf and +clones audit_pad's 15k-entry `BTreeMap` along the way. That's an +O(N) memcpy+tree-build per call — exactly the shape of the +observed slope. + +### Phase 11 plan + +Two surgical changes informed by the per-kind data: + +1. `Schema.items`/`functions`/`types`: `BTreeMap` → `imbl::OrdMap`. + Drop-in replacement (all callers already use ops common to both + types: `get`, `insert`, `remove`, `contains_key`, `is_empty`, + `len`, `values`, `iter`). `imbl::OrdMap` clones in O(1) (refcount + bump on a persistent tree root), turning the leaf-copy cost from + O(audit_pad.items.len()) into O(1). + +2. `StorageMetadata.{collection_metadata, unfinalized_shards}`: + `BTreeMap`/`BTreeSet` → `imbl::OrdMap`/`imbl::OrdSet`. Same + reasoning; this directly attacks the + `storage_collection_metadata` 200 µs/call slope. + +Re-bench in phase 11 should drive `item` mean back toward flat +(N-independent constant) and zero-out the `storage_collection_metadata` +slope. + +## Phase 11+12: the fixes land + +Both fixes from the phase 10 plan shipped on this branch: + +* `ad197b0bc3` — `catalog: switch Schema items/functions/types to imbl::OrdMap` +* `4b6f5d171b` — `storage-client: switch StorageMetadata fields to imbl persistent collections` + +Re-bench at N=5k/10k/15k. Phase 10 here is the *pre-fix* baseline, +phase 11 has only the Schema fix, phase 12 has both fixes: + +### Per-call cost in apply_updates_inner (mean µs/call) + +| kind | phase 10 (pre) | phase 11 (Schema only) | phase 12 (both) | +|-------------------------------|---------------:|-----------------------:|----------------:| +| `item` @ N=5k | 586 | 166 | 164 | +| `item` @ N=10k | 1051 | 214 | 200 | +| `item` @ N=15k | 1436 | 251 | 242 | +| `storage_collection_metadata` @ N=5k | 171 | 181 | 3.84 | +| `storage_collection_metadata` @ N=10k | 371 | 344 | 4.03 | +| `storage_collection_metadata` @ N=15k | 567 | 523 | 5.12 | + +* **item**: 1436 → 242 µs/call at N=15k (**−83%**). The per-call + slope across 5k→15k went from +850 µs to +78 µs — the + `audit_pad`-schema BTreeMap clone-via-leaf-copy is gone. +* **storage_collection_metadata**: 567 → 5.12 µs/call at N=15k + (**−99%**). The per-call slope is gone entirely; what's left is + just the `Arc::make_mut` shallow clone of the StorageMetadata + struct itself (now O(1) because all its fields are persistent). + +### apply_updates_inner total (ms/DDL) + +| N | phase 9 (consolidate only) | phase 10 (instrumented) | phase 11 (Schema) | phase 12 (both) | +|-------|---------------------------:|------------------------:|------------------:|----------------:| +| 5000 | — | 3.23 | 1.53 | **1.16** | +| 10000 | — | 5.58 | 2.11 | **1.31** | +| 15000 | — | 7.52 | 2.61 | **1.49** | +| slope (10→15) | **+1.94** | +0.50 | **+0.18** | + +apply_updates_inner slope at 10→15k went from +1.94 ms/+5k to ++0.18 ms/+5k — a **>90% reduction**. The phase 10 instrumentation +attributed the slope; the phase 11/12 fixes erased it. + +### create_p50 end-to-end (ms) + +| N | phase 7 | phase 9 | phase 10 | phase 11 | phase 12 | Δ from p7 | +|-------|--------:|--------:|---------:|---------:|---------:|----------:| +| 5000 | 33.57 | 31.34 | 32.88 | 32.29 | 32.43 | −1.14 | +| 10000 | 39.82 | 38.70 | 38.26 | 35.87 | 35.81 | −4.01 | +| 15000 | 48.93 | 44.51 | 43.87 | 42.91 | **42.84** | **−6.09** | +| slope (10→15) | +9.11 | +5.81 | +5.61 | +7.04 | +7.03 | | + +At N=15k, create_p50 is down 6.09 ms from the start of this branch +(48.93 → 42.84). The 10→15k slope on the end-to-end metric stays +noisy across runs (±2 ms per scale point), but the internal +attribution is unambiguous: the two structural fixes erased the +apply_updates_inner slope they targeted. Residual slope in +create_p50 is now distributed across other transact phases (notably +`prepare_state`, `op_loop`, `transact_inner` outer) — none of which +have a single dominant N-scaling step on the magnitude of the two +that just got fixed. + +### Pattern note + +Both fixes are the same shape: a struct held inline inside an +`imbl::OrdMap` had non-persistent (B-tree) collections as +fields. Path-copy on the outer OrdMap leaf-clones the V, which +deep-clones those collections. + +When putting a value type into an `imbl::OrdMap` (or behind an +`Arc` that gets `make_mut`'d under sharing), prefer **persistent +sub-collections** (`imbl::OrdMap`/`OrdSet`/`Vector`) for any field +that can grow with the workload — otherwise the persistent outer +structure silently loses its O(1) clone advantage to its first +non-persistent inner field. This pattern is worth grep-flagging +across the rest of the catalog and controller state types. + +## Sweep: other instances of the same pattern + +Read-only audit triggered by the pattern note above. Scope: every +`imbl::OrdMap` (or `Arc>` / `Arc` over +something we `make_mut`) in `adapter`, `catalog`, `storage-client`, +`storage-controller`, `compute-client`, `controller`. For each, ask: +does the value type embed a non-persistent sub-collection that can +grow with the workload, where path-copy / `make_mut` would deep-clone +it? + +Findings, ranked by likely impact: + +### HIGH — same shape as the just-shipped fixes, in the same hot path + +**`Database.{schemas_by_id, schemas_by_name}`** (`src/catalog/src/memory/objects.rs:87`): + +```rust +pub struct Database { + ... + pub schemas_by_id: BTreeMap, + pub schemas_by_name: BTreeMap, + ... +} +``` + +`Database` lives inside `CatalogState.database_by_id: imbl::OrdMap`. +Every `get_schema_mut` call path-copies the database_by_id leaf → +clones a `Database` → deep-clones `schemas_by_id`. That's a +`BTreeMap` clone — and since `Schema` now embeds +three `imbl::OrdMap`s (post phase 11), each `Schema` shallow-clones +in O(1), so the cost is `K` shallow-clones where `K` is the number +of schemas in the database. For `materialize` that's ~5 (including +`audit_pad`/`audit_meas`), so per-call this is small constant. + +But: `database_by_id.get_mut` happens on **every** apply_item_update, +and that constant gets multiplied by 4 item-events per DDL plus the +preliminary/state Cow split. Swapping to `imbl::OrdMap` +and `imbl::OrdMap` is a drop-in change (callers use +get / get_mut / insert / remove / iter / values — all common). Likely +worth doing in the next pass; the fix is mechanically identical to +the Schema/StorageMetadata fixes. + +### MEDIUM — grows with workload, not exercised by the audit_pad bench + +**`Cluster.bound_objects`** (`src/catalog/src/memory/objects.rs:352`): + +```rust +pub struct Cluster { + ... + pub bound_objects: BTreeSet, // grows with N + pub replica_id_by_name_: BTreeMap, + pub replicas_by_id_: BTreeMap, + pub log_indexes: BTreeMap, + ... +} +``` + +`Cluster` lives inside `clusters_by_id: imbl::OrdMap`. +`bound_objects` accumulates every MV / index / source / continual task +bound to that cluster — for an MV-heavy workload on a single cluster, +this can grow to thousands of entries. `insert_entry` does +`clusters_by_id.get_mut(cluster_id).bound_objects.insert(...)` on +every CREATE for any object that has a `cluster_id`, so the +path-copy clones a sibling-set of Clusters, each cloning its +`bound_objects` `BTreeSet`. + +Our bench (audit_pad of plain `CREATE TABLE`s, no cluster binding) +doesn't exercise this — `Table::cluster_id()` returns `None`. So the +slope is invisible here. Worth instrumenting next time we benchmark +MV / index scale. + +**`Cluster.replica_id_by_name_` / `replicas_by_id_` / `log_indexes`**: +small in practice (single-digit entries per cluster typically), but +swapping is mechanically the same. + +### LOW — workload-dependent, mostly small in practice + +* `Role.{vars: RoleVars { map: BTreeMap }, membership: RoleMembership { map: BTreeMap }}` + in `roles_by_id: imbl::OrdMap`. Per-role counts are + small (a few memberships, a few vars). Only matters if a deployment + has many roles with large memberships. +* `SourceReferences.references: Vec` in + `source_references: imbl::OrdMap`. + Grows with the number of references *of one source*, not with N + tables. Workload-specific. +* `CatalogEntry.{referenced_by, used_by}: Vec` in + `entry_by_id: imbl::OrdMap`. Per-entry + usually small; `entry_by_id` itself has 15k+ entries, so each + `get_mut` leaf-clones ~16 sibling `CatalogEntry`s. The Vec clones + are cheap (`Copy` ids), but cloning `CatalogEntry.item: CatalogItem` + for unaffected siblings is real work — and for MaterializedView / + Index / ContinualTask items that carry optimized/physical plans + (#35834), that clone is expensive. Not exercised by the audit_pad + table bench, but a future MV-scale bench would surface this. +* `notices_by_dep_id: imbl::OrdMap>>`: + `Vec>` clones are shallow (refcount bumps). Cheap. + +### Recommended next step + +Land the `Database.schemas_by_id` / `schemas_by_name` swap (HIGH +priority above). It's the same pattern, in the same hot path, with +the same drop-in semantics — and the per-DDL constant gets +multiplied by every `apply_item_update` event. Phases 11/12 fixed +the inner layer (Schema's BTreeMaps); the database layer is its +outer sibling. + +For the MEDIUM tier, the path forward is to design a real MV / +index / source scale bench (audit_pad currently only stresses +plain tables) and re-run the per-kind apply_updates timer. The +existing `mz_catalog_apply_update_kind_seconds{kind}` histogram is +the right tool — if `cluster` / `cluster_replica` / `system_object_mapping` +kinds show a per-call slope, that's the signal to land the +`Cluster.bound_objects` swap and friends. + +## Phase 13: the sweep lands + +Committed in `1a8446c2bf` (and consolidated comment-block on +`CatalogState` removed the per-field "why imbl" comments on Schema +and StorageMetadata that phases 11/12 had added): + + Database.{schemas_by_id, schemas_by_name} + Cluster.{bound_objects, replica_id_by_name_, replicas_by_id_} + RoleMembership.map, RoleVars.map + SourceReferences.references + +Trait sigs in `mz_sql::catalog` updated to match (`CatalogDatabase`, +`CatalogRole`, `CatalogCluster`). Intentional holdouts: see the +comment block on `CatalogState`. + +### Sanity-bench (`results_phase13/`) + +The audit_pad bench doesn't actually touch most of these paths +(plain `CREATE TABLE` doesn't mutate clusters, roles, or sources). +Expected outcome: no regression on internal metrics, no slope +change. Result: + +| kind | phase 12 µs/call | phase 13 µs/call | +|-------------------------------|-----------------:|-----------------:| +| `item` @ N=15k | 242 | 238 | +| `storage_collection_metadata` @ N=15k | 5.12 | 5.95 | + +apply_updates_inner total ms/DDL also unchanged at N=15k +(1.49 → 1.48). Run-to-run noise dominates; no regression, no +measurable win on *this* workload — which was the point. The fix +is landmine-prevention for cluster-heavy, role-heavy, and +source-heavy workloads where the same leaf-clone-deep-clone +pattern would otherwise surface a slope on its respective DDLs. + +A future scale bench that exercises `CREATE INDEX` / `CREATE +MATERIALIZED VIEW` on a single cluster, or `GRANT` across many +roles, is the right way to actually measure the cluster/role swaps; +the existing `mz_catalog_apply_update_kind_seconds{kind}` histogram +will surface any residual slope per kind. + +## Phase 14: prepare_state slope = BackgroundTask lock contention + +With apply_updates flat, the largest residual slope in +`mz_catalog_transact_phase_seconds` at N=15k was **`prepare_state`**: +0.06 → 1.31 → 2.40 ms per call across 5k → 10k → 15k (count=200 per +scale, two calls per CREATE+DROP rep). Counted as ms/DDL: 0.11 → +2.62 → 4.79, **+2.17 ms/+5k** — bigger than any other phase by 10×. + +### Phase 14 instrumentation + +Added `mz_storage_collections_prepare_state_phase_seconds{phase}` with +sub-phase labels `insert_add`, `insert_register`, `delete`, +`dropped_shard_lookup`, `insert_unfinalized`, `mark_finalized` — +exactly matching the source layout of +`StorageCollections::prepare_state`. Committed in `0a671b7f6a`. + +### Attribution + +Re-bench at N=5k and N=10k (before the fix). Mean per-call cost in +each sub-phase of `prepare_state`: + +| sub-phase | N=5k µs | N=10k µs | Δ/+5k | +|--------------------------|--------:|---------:|------:| +| insert_add | 6.7 | 7.9 | +1.2 | +| insert_register | 0.08 | 0.08 | 0 | +| delete | 6.9 | 7.5 | +0.6 | +| **dropped_shard_lookup** | **1.6** | **599** | **+598** | +| insert_unfinalized | 1.4 | 1.6 | +0.2 | +| mark_finalized | 3.5 | 4.2 | +0.7 | + +**`dropped_shard_lookup` is the entire slope**: 1.6 → 599 µs per +call (375×). Every other sub-phase is sub-µs/+5k. + +`dropped_shard_lookup` wraps: + +```rust +let collections = self.collections.lock().expect("poisoned"); +for (id, shard) in dropped_mappings { … } +``` + +In our `CREATE TABLE` workload `dropped_mappings` is **always +empty**, so the for-loop is a no-op. The 600 µs is purely +`self.collections.lock()` waiting on a contended mutex. + +### Who holds the lock for that long? + +`StorageCollectionsImpl::collections` is +`Arc>>` — a +single global mutex shared by every DDL path and by the storage +`BackgroundTask`. The background task's `run()` loop has this branch +(`storage_collections.rs:2742`): + +```rust +(id, handle, upper) = &mut txns_upper_future => { + let mut uppers = Vec::new(); + for id in self.txns_shards.iter() { // N entries! + uppers.push((*id, &upper)); + } + self.update_write_frontiers(&uppers).await; + … +} +``` + +Every time the txns shard's upper advances, the task walks **every +txns-backed user table** (N = 5k/10k/15k in our bench) and calls +`update_write_frontiers`, which then does: + +```rust +let mut self_collections = self.collections.lock().expect("…"); +for (id, new_upper) in updates.iter() { // O(N) under the lock + … +} +``` + +So one txns-upper tick = one O(N) lock-hold against the only mutex +`prepare_state` needs. With 100 reps of CREATE+DROP firing in close +sequence, the txns upper ticks often enough to collide with most +prepare_state calls; the slope we see is just the average waiting +time per call. + +### Fix: chunked unlock in `update_write_frontiers` + +Process `updates` in chunks of 256 entries, releasing +`self.collections` between chunks. The per-acquisition work is now +bounded by chunk size (≈256 × ~1 µs ≈ 256 µs worst case) instead of +N × ~1 µs. The total CPU work is unchanged; the change is just +lock-hold ceiling and how often other acquirers can interleave. + +We considered a deeper refactor — moving the shared txns upper out +of each per-collection `write_frontier` field into a single shared +field on `StorageCollectionsImpl` — and that would eliminate the +work entirely. We didn't do it because it touches every reader of +`write_frontier` and the chunked-unlock fix is mechanical and +sufficient: contention scales with chunk size, not with N. + +### Re-bench (`results_phase14/`) + +Fresh envd (CRDB consensus, instrumentation included). Baseline run +killed after N=10k; fix run completed all three scales. Apples-to-apples +within the same build: + +| metric | N=5k | N=10k | N=15k | +|---------------------------------|----:|-----:|-----:| +| `create_p50` PRE (ms) | 59.00 | 63.18 | — | +| `create_p50` POST (ms) | 48.80 | 52.04 | 60.12 | +| **Δ create_p50** | **−10.20** | **−11.14** | — | +| `dropped_shard_lookup` PRE (µs/call) | 1.60 | 599.39 | — | +| `dropped_shard_lookup` POST (µs/call)| 9.61 | 1110.41 | 2722.10 | +| `prepare_state` PRE (ms/DDL) | 0.05 | 1.25 | — | +| `prepare_state` POST (ms/DDL) | 0.07 | 2.27 | 5.50 | + +Two surprises in the data: + +1. **`dropped_shard_lookup` and `prepare_state` got *worse* after the + fix.** Releasing the lock more often gives the BackgroundTask + more chances to re-win it (Linux pthread mutex isn't fair); a + `prepare_state` caller losing the lock race 40× per BG tick can + wait longer than losing it once and waiting out a single 10 ms + hold. So the chunked unlock is not actually shrinking + `prepare_state`'s lock-wait. + +2. **End-to-end `create_p50` got better anyway.** The + `catalog_transact_phase` breakdown at N=10k shows where: + + | phase | Δ (ms/DDL) | + |--------------------------------|-------------:| + | `coord_builtin_table_execute` | **−3.58** | + | `coord_post_transact` | **−3.58** | + | `coord_pre_transact` | **−1.87** | + | `tx_commit` | **−1.56** | + | `op_loop` | −0.14 | + | `final_apply_updates` | −0.09 | + | `prepare_state` | **+1.02** | + | `transact_inner` | +0.78 | + | `coord_inner_total` | **−8.07** | + + These phases don't take the storage `collections` lock — but they + do compete with `BackgroundTask::run` for **CPU**. By yielding + between chunks the background task stops monopolizing both the + lock and the runtime, and the txns-shard writer, persist client, + and builtin-table execute path catch up much faster. The fix is a + CPU-yield win, not the lock-wait win it advertised in the source + comment. + +### What this *doesn't* fix + +The post-fix `prepare_state` slope at N=10k → 15k is still +3.23 +ms/+5k, and `coord_inner_total` is still +6.81 ms/+5k. The +fundamental issue — `update_write_frontiers` doing O(N) work on +every txns-upper tick — is still there, just split across more +lock cycles. A future architectural pass should store the shared +txns upper in one field on `StorageCollectionsImpl` and skip the +per-collection propagation entirely; that's the only way to make +`dropped_shard_lookup` actually flat. The chunked unlock is a +mechanical, low-risk intermediate. + +Sweep totals (POST-fix, same run): + +| N | create_p50 (ms) | coord_inner_total (ms/DDL) | +|-------|----------------:|---------------------------:| +| 5000 | 48.80 | 25.52 | +| 10000 | 52.04 | 29.52 | +| 15000 | 60.12 | 36.33 | +| slope 10→15 | **+8.08** | **+6.81** | + +## Phase 15+: extend to N=20k / N=25k and iterate + +The slope is *flatter* than at the start of the investigation but it +is not flat. Concretely we still see roughly +1.6 ms/+1k tables in +`create_p50` between N=10k and N=15k, and `coord_inner_total` still +climbs by +6.81 ms over that same step. Before declaring victory, we +need to: + +1. **Extend the bench to N=20k and N=25k.** Most of our prior + evidence is dominated by run-to-run noise at N≤15k; pushing to + 25k gives us a longer lever arm to discriminate "slope-of-zero + plus noise" from "slope-of-something". Bench harness change: + `SCALES = [5000, 10000, 15000, 20000, 25000]` in + `/home/ubuntu/envd-ddl-investigation/bench.py`. The + `max_tables` / `max_objects_per_schema` system vars are already + set to 30000 so headroom is fine. + +2. **Iterate the investigation as discrete one-shot agent runs.** + Each iteration is: (a) run the bench, (b) attribute the slope + using existing metrics + new ones if needed, (c) design *one* + fix that addresses the slope owner, (d) implement, commit, write + up. Between iterations we look at the headline numbers and + decide whether to keep going. Stop when slope is flat across + N=10k → 25k or when we run out of obvious leverage. + + The agents driving each iteration should be encouraged to step + back and consider architectural cuts — especially boundaries + between Coordinator / Storage Controller / StorageCollections / + persist. The next-up known suspect (txns upper fanout to N + per-collection `write_frontier` fields) is itself a + boundary-shape problem: the txns shard's upper is a single + value that we currently propagate to N owners on every tick; + keeping one shared field on `StorageCollectionsImpl` and + reading from it would just delete the work. Simple, narrow + interfaces beat clever caching. + +### Iteration 1: shared `txns_upper`, 1Hz periodic sweep + +Commit `0007b892` lands the architectural cut described above. +`StorageCollectionsImpl::txns_upper` holds the latest observed txns +shard upper as a single shared field. The `BackgroundTask::run` +txns-upper branch publishes to that field (O(1)) and reissues the +upper future; readers that previously consulted per-collection +`write_frontier` for txns-backed shards (`collections_frontiers`, +`set_read_policies_inner`, `alter_table_desc`) now go through +`effective_write_frontier()`, which serves txns-backed collections +from the shared field. Per-collection `write_frontier` is still +source-of-truth for non-txns collections. + +Persist compaction still needs each txns-backed collection's +implied capability (`since`) to advance, so a 1Hz periodic sweep +calls `update_write_frontiers` exactly as the old per-tick branch +did. Work per sweep is unchanged from before; what changed is the +frequency — `1 Hz` vs the ~40 Hz the old branch fired at under DDL +load — and therefore how often the global `collections` mutex is +contended on the DDL hot path. + +#### Bench: N=5k → N=25k (PRE = phase 14 chunked unlock; POST = this iteration) + +`results_phase15_pre/` and `results_phase15_post/`, 100 reps per +scale, same envd build, same CRDB: + +| N | create_p50 PRE (ms) | create_p50 POST (ms) | drop_p50 PRE (ms) | drop_p50 POST (ms) | +|------:|--------------------:|---------------------:|------------------:|-------------------:| +| 5000 | 50.18 | 66.25 ⚠ | 30.79 | 43.44 | +| 10000 | 55.99 | 58.38 | 36.44 | 44.60 | +| 15000 | 68.41 | 64.82 | 43.98 | 45.97 | +| 20000 | 69.86 | 63.16 | 43.06 | 43.00 | +| 25000 | 76.19 | 64.67 | 49.25 | 43.38 | + +Slope 10k → 25k: + +- `create_p50` PRE: +20.20 ms (+1.35 ms/+1k tables) +- `create_p50` POST: **+6.29 ms (+0.42 ms/+1k tables) — 3.2× flatter** +- `drop_p50` PRE: +12.81 ms +- `drop_p50` POST: **−1.22 ms — flat within noise** + +The N=5k POST point (66.25) is anomalously high and is almost certainly +warm-up — first scale in the sweep absorbs cold-cache + first-DDL +overhead. Treating N=10k as the warm baseline, the slope is flat. Drop +is flat from N=5k onwards. + +This is the "close enough to flat" iteration. Remaining work is in the +follow-up notes below; we are no longer chasing an O(N) slope. + +### Open follow-up — why was the per-tick branch firing at ~40 Hz? + +The Phase-15 fix changed the *frequency* of the O(N) lock-hold from +~40 Hz to 1 Hz, not the *work per tick*. That works because nothing +on the DDL hot path actually needs sub-second freshness on +per-collection txns frontiers (`effective_write_frontier()` serves +them from the shared field; persist compaction only needs eventual +since-advancement). But ~40 Hz under DDL load is itself +surprising and worth understanding: + +- The txns shard upper advances on every txns-backed write + (i.e. every committed user-table append from `coord_builtin_table_execute` + + every catalog audit log row + every storage-usage update). + At idle the rate should be a few Hz at most. +- Our bench fires 100 CREATE+DROP DDLs back-to-back per scale. + Each DDL goes through the builtin-table commit path, which does + several txns appends (audit log, mz_catalog_server scrape, etc.). + If each DDL produces ~4 txns commits and the bench runs at ~10 + DDL/s, that's ~40 commits/s = ~40 upper advances/s, matching + the observed rate. +- **But:** the rate doesn't slow down between DDLs. The bench has + `time.sleep(SETTLE_S=5)` between scales but no sleep between + reps; that's expected. What's *not* expected is the BG task + running at 40 Hz when there is no DDL load at all — and we + haven't measured that. + +Next-iteration task: instrument the txns shard upper advance rate +(a counter on each `BackgroundTask::run` tick of the txns branch, +sliced by phase: idle vs DDL-load), measure it at idle and during +the bench, and decide whether the *write-side* (whoever's producing +txns commits) is the right place to slow this down, or whether the +1 Hz cap on the read side is the simplest answer. If we find that +~40 Hz at idle is real, that's a separate slope/efficiency bug +worth fixing independently of DDL scalability. + +Document findings, commit a writeup, and only ship a fix if the +investigation surfaces something concrete. + +### Iteration 2: where the 40 Hz comes from + +#### Instrumentation + +Added two Prometheus counters in `BackgroundTask` (commit `cdb8484d7b`): + +- `mz_storage_collections_txns_upper_advances_total`: bumped each + time the `txns_upper_future` arm fires (i.e. each observed + advance of the txns shard upper). +- `mz_storage_collections_txns_since_sweeps_total`: bumped each + time the periodic 1Hz sweep arm fires. + +The ratio of the two is the coalescing factor of phase 15. + +#### Experiment + +`probe_txns_upper.py` in the investigation dir does three things: + +1. Sit idle for 30s and diff the counter. +2. Fire 100 CREATE+DROP back-to-back from one connection and + sample the counter every 1s while the burst runs. +3. Sit idle for 30s again to confirm baseline. + +Pad/measure scales beyond 0 tables are not needed: the question is +about commits-per-DDL, not about per-collection fanout. + +Single envd run, optimized build, CRDB consensus, fresh state. + +#### Result + +| phase | elapsed | total advances | advance rate | sweeps/s | +|---------------------:|--------:|---------------:|-------------:|---------:| +| idle (pre) | 30.7 s | 30 | 0.978 Hz | 0.98 | +| 100×(CREATE+DROP) | 8.3 s | 402 | 48.4 Hz | 1.08 | +| idle (post) | 31.0 s | 31 | 1.0 Hz | 1.0 | + +`advances/DDL = 402 / 100 = 4.02`. Almost exactly 4. + +#### Theory and verification + +At idle the rate is **1 Hz**, driven by +`Coordinator::advance_timelines_interval` +(`src/adapter/src/coord.rs:3661`), which fires on a tokio interval +seeded from `default_timestamp_interval = 1000 ms`. Every tick +enqueues a `GroupCommitInitiate` message, and a group commit (even +an empty one) lands one append against the txns shard, advancing +its upper by one. That's 1.0 Hz, matching the measurement. + +Under DDL load the rate is **~48 Hz, broken down as ~4 advances +per CREATE+DROP pair**. The four come from: + +1. `register` of the new table's data shard (txns.register + commits to the txns shard) — `src/storage-controller/src/persist_handles.rs:372`. +2. `coord_builtin_table_execute` for the CREATE audit-log row — + `src/adapter/src/coord/ddl.rs:546` → `BuiltinTableAppend::execute` + → `group_commit` → one txns append. +3. `forget` of the dropped table's data shard + (txns.forget commits to the txns shard) — same file:397. +4. `coord_builtin_table_execute` for the DROP audit-log row. + +Each of those is its own commit_at against the txns shard, each +advances the upper by one. The bench drives ~12 CREATE+DROP/s, so +~48 Hz, matching the measurement and the ~40 Hz hand-estimate in +the phase-15 writeup. + +#### Conclusion + +- Idle: 1 Hz. No spinning, no separate efficiency bug. The + `advance_timelines_interval` 1Hz tick is intentional — it + downgrades read holds and bounds how stale realtime read + timestamps can become. +- Under DDL load: 4 commits per CREATE+DROP, structural to the + fact that register, forget, and audit-log append are three + separate calls against the txns shard (4 because CREATE and DROP + each pay the audit append separately). + +The phase-15 fix changed the *consumer* side: the BackgroundTask +no longer fans out to N collections per advance. The +**writer-side multiplier** (3 txns commits per single DDL) is a +property of the catalog/storage-controller boundary: catalog +DDL produces (a) an audit append routed via the coord group-commit +path and (b) a storage-controller register/forget routed via the +TxnsTableWorker; these don't share a transaction and can't +trivially be coalesced without significant cross-component +plumbing (timestamps, ordering, register-before-publish). + +**No fix shipped.** Phase 15's coalescing is the right answer for +this workload. Killing the writer-side multiplier would require +batching register/forget with the same group-commit append, which +is a larger refactor than the slope justifies (the slope is +already flat at +0.42 ms/+1k tables across 10k→25k). + +#### Bigger-picture observation + +The txns shard upper is conceptually a single global value that +fans out to N owners. There are two boundaries where work is +spent reacting to its advances: + +- *Consumer side* (this investigation): for every advance, walk N + collections and downgrade their per-collection implied + capability. Fixed by phase 15 (publish once, sweep at fixed + cadence). +- *Producer side* (this iteration): for every DDL, the catalog + layer makes three independent commits against the txns shard + because audit/register/forget take different code paths. + +Both are symptoms of the same boundary mismatch: txns-wal exposes +a fine-grained "commit a transaction" API, and the layers above +treat it as an unmetered RPC. A cleaner cut on the producer side +would be a per-DDL transactional unit that bundles audit-log +appends together with register/forget into a single txns commit; +the catalog already serializes DDL, so there's no concurrency +penalty to merging them. Concretely it would mean teaching +`Coordinator::catalog_transact_with` to hand a single +`Txn` (from `txn-wal`) to both `BuiltinTableAppend::execute` and +the storage controller's `TxnsTableWorker`, rather than each +opening its own. That's the architectural follow-up if a future +iteration wants to push past phase 15. + +Suggested next iteration (if any): build a stronger justification +before pursuing producer-side coalescing. At current scale the +DDL hot path's `prepare_state` slope is flat; the work-per-tick +is small; CPU is not the bottleneck. The reason to fix the +multiplier would be reducing pressure on CRDB consensus (4× +commits → 4× lease/write QPS to the consensus store) or +reducing churn in `txns_cache.update_gt`. Both are worth measuring +before opening a refactor that crosses adapter ↔ storage-controller. + +## Phase 17 — high-N slope confirmation + +The phase-15 post-fix writeup called the residual slope "+0.42 ms / ++1000 tables, close enough to flat" based on a 100-rep sweep across +N=5k..25k. That bench had no warmup discard and N=5k carried the +cold-start cost (66 ms vs 50–58 ms at higher N). Iteration 3 reruns +the high-N portion with (a) 30 warmup reps discarded at every scale, +(b) 500 measurement reps per scale, (c) ascending pad in a single +envd run so no scale needs a restart. Bootstrap 95 % CIs on the +median and on the regression slope are computed from the 500-rep +per-scale samples (`/home/ubuntu/envd-ddl-investigation/analyze_phase17.py`, +seed 17, 10 000 bootstrap samples). + +### Bench: N=10k → N=25k (POST = phase 15 + 16) + +`results_phase17/`, 4 scales, single optimized envd run against +CRDB-backed consensus. + +| N | create_p50 ms | create p50 95% CI | drop_p50 ms | drop p50 95% CI | +|------:|--------------:|------------------:|------------:|----------------:| +| 10000 | 50.91 | [50.48, 51.27] | 35.35 | [34.88, 35.60] | +| 15000 | 52.46 | [51.90, 53.21] | 37.95 | [37.58, 38.74] | +| 20000 | 58.84 | [58.01, 59.55] | 44.03 | [43.00, 45.05] | +| 25000 | 70.39 | [68.79, 72.60] | 51.79 | [50.47, 53.00] | + +Slope, fit by least squares over the four scales (medians of 500 +reps each): + +- `create_p50` slope: **+1.296 ms / +1k tables [95% CI +1.187, +1.434]** +- `drop_p50` slope: **+1.108 ms / +1k tables [95% CI +1.028, +1.194]** + +The CI excludes zero by a wide margin. The slope is **3.1×** what +phase-15 reported (+0.42 ms / +1k). The previous number was biased +low by warmup contamination at N=5k and by absorbing the first +warm reps at every scale into the median. The slope is also +**super-linear** across this range: it accelerates from +0.31 ms/+1k +(10k → 15k) to +1.28 (15k → 20k) to +2.31 ms/+1k (20k → 25k). + +### Where the slope lives + +`results_phase17/attribute_all.txt`: per-call mean of every relevant +`*_sum` / `*_count` pair diffed between +`metrics_phase17/measure_start_N{N}.prom` and +`metrics_phase17/after_N{N}.prom` (so the warmup is excluded from +attribution too). + +The top slope contributors, ranked by ms/+1k of mean per-call +latency, all sit below the catalog-transact layer: + +| metric | N=10k | N=25k | slope ms/+1k | +|---|---:|---:|---:| +| `persist_external_op_latency_by_shard_kind{op="consensus_cas",shard_kind="user_data"}` | 24.4 | 118.6 | +6.28 | +| `persist_external_op_latency_by_shard_kind{op="consensus_scan",shard_kind="txns"}` | 4.0 | 21.9 | +1.19 | +| `persist_external_op_latency_by_shard_kind{op="consensus_truncate",shard_kind="catalog"}` | 40.0 | 54.9 | +0.99 | +| `storage_collections_create_collections_phase{phase="open_data_handles_concurrent"}` | 16.1 | 24.2 | +0.54 | +| `apply_catalog_implications_seconds` | 19.3 | 26.8 | +0.50 | +| `catalog_transact_phase{phase="transact_inner"}` | 1.6 | 5.0 | +0.23 | +| `catalog_transact_phase{phase="tx_commit"}` | 2.8 | 5.6 | +0.19 | +| `catalog_transact_phase{phase="prepare_state"}` | 0.2 | 2.7 | +0.16 | + +The dominant contributor is **persist's `consensus_cas` on +`user_data` shards**, mean latency growing from 24 ms to 119 ms per +call. The same pattern shows up in +`consensus_scan` on the `txns` shard and `consensus_truncate` on +the `catalog` shard: as the catalog grows (more user collections +→ more rows in CRDB's `consensus` table → more state per shard's +CAS check), every individual CAS, scan and truncate against CRDB +gets slower. There are ~176 `user_data` CAS calls per CREATE+DROP +rep, mostly background compaction/heartbeat traffic that runs +concurrently across many persist clients, so per-rep wall-clock +impact is far less than `mean_latency × call_count`. But two of +those CAS-bound calls *are* on the DDL hot path: + +1. `open_data_handles_concurrent` (16 → 24 ms) — opens a write/since + handle for the new shard during CREATE TABLE. + `storage-collections.rs:1875`. One CRDB CAS per handle. +2. `tx_commit` (2.8 → 5.6 ms) — the catalog transaction commit. + Goes through the `catalog` shard's `consensus_cas`. + +Both are single CAS-per-DDL operations whose latency tracks the +per-call CAS latency growth shown above. + +### Conclusion + +The residual slope is real, larger than phase-15's reading, and +**structural to the persist-on-CRDB layer**, not in code we own at +the adapter / coordinator / storage-collections boundary. The +phase-15 fix correctly removed the per-DDL O(N) work inside +`StorageCollectionsImpl::BackgroundTask`; what remains is per-CAS +latency growth in the persist consensus implementation against +CockroachDB. It is not a one-file mechanical change to fix: it +would require either CRDB-side tuning (index design on the +`consensus` table, vacuum/compaction policy), or a redesign that +batches multiple DDLs' persist commits together, or moving the +catalog/txns shard's consensus_cas off the per-DDL critical path. + +We deliberately do not ship a fix this iteration. + +### What the phase-15 writeup got wrong + +It reported "+0.42 ms / +1k tables, flat enough" using a 100-rep +sweep where: + +- N=5k was the first scale measured; its median absorbed the + envd cold-start (66 ms vs 50–58 ms at warm scales). That single + outlier pulled the regression line shallow. +- No warmup reps were discarded at any scale; the bench median + consequently included the per-scale first-rep spike (e.g. + rep=0 at N=25k was 2118 ms in `results_phase15_post/`). +- 100 reps is enough to compute a median but the CI is wide + enough (~5 ms) to hide a slope of ±1 ms / +1k. + +The 500-rep bench with bootstrap CI cuts that uncertainty by ~3× +and exposes the slope. + +### Methodology artefacts + +- `bench_phase17.py` (in the investigation dir): the new harness. + `WARMUP_REPS = 30`, `MEASURE_REPS = 500`, scales [10k, 15k, 20k, + 25k] ascending. CSV gets a `phase` column (`warmup`/`measure`) + so analysis can drop the warmup window. Metrics are snapshotted + at three points per scale: before-window, measure-start, + after-window — so attribution is over the measurement window + only. +- `analyze_phase17.py`: per-scale median + 95 % bootstrap CI, plus + least-squares slope ms/+1k tables with bootstrap CI. +- `attribute_phase17.py`: per-rep mean for each + `catalog_transact_phase_seconds` label, slope ms/+1k tables. +- `/tmp/attribute_apply.py` and `/tmp/attribute_cas.py`: same shape + for the broader `pgwire`/`catalog_transact_seconds`/`persist` + series; the persist-shard-kind attribution is what surfaced the + `user_data consensus_cas` slope. + +### Suggested next iteration + +Stop chasing the slope at the adapter/storage-collections layer. +The remaining lever is the persist-on-CRDB cost per operation, +and the cleanest probe is a microbenchmark on CRDB consensus +operations as `consensus` table size grows — outside this +investigation's scope. If a future iteration wants to reduce +per-DDL persist work, the highest-leverage change is killing the +4× producer-side multiplier called out in iteration 2 of phase +16: batch register / forget / audit into a single txns commit so +each DDL pays one CRDB consensus round trip instead of four. + +## Phase 18 — bogo vs CRDB CAS attribution + +Phase-17 attributed the entire +1.296 ms/+1k slope to +`consensus_cas` on `user_data` shards getting 24 → 119 ms per call +on CRDB. That attribution is consistent with the data but does not +distinguish *where* in the stack the cost lives: it could be (a) +CRDB's `consensus` table getting expensive to CAS as it grows, (b) +the persist client doing more work per CAS as state grows, or (c) +both. Phase 18 swaps the consensus backend for `mz-bogo-consensus` +(in-memory, no-op CAS over local gRPC) and reruns the same N=10k → +N=25k bench so the same harness exercises an identical +control/data plane against a backend that *cannot* slow down with +state size. + +### Bench + +`bench_phase18.py`, same shape as `bench_phase17.py`: 30 warmup + +500 measure reps per scale, ascending pad in a single envd run, +metrics snapshots at `before` / `measure_start` / `after`. Single +envd run, configured with `--persist-consensus-url=bogo://...`. +Bogo-side metrics also snapshotted (`bogo_*.prom`). + +`analyze_phase18.py`, `attribute_phase17.py`-style: +sum/count diff between `measure_start_N{N}.prom` and +`after_N{N}.prom`, warmup excluded by construction. Slope CIs use +the same bootstrap as `analyze_phase17.py` (seed=18, 10 000 +samples). + +### create_p50 medians and slope + +| backend | N=10k p50 ms | 95% CI | N=25k p50 ms | 95% CI | slope ms/+1k | 95% CI | +|---|---:|---|---:|---|---:|---| +| CRDB (phase 17) | 50.91 | [50.48, 51.27] | 70.39 | [68.79, 72.60] | **+1.296** | [+1.187, +1.434] | +| bogo (phase 18) | 27.87 | [27.69, 28.02] | 38.23 | [36.94, 38.90] | **+0.691** | [+0.605, +0.738] | + +`drop_p50` mirrors: bogo +0.875 [95% CI +0.819, +0.987] vs CRDB ++1.108 [95% CI +1.028, +1.194]. + +**Headline: bogo still has a slope.** 53 % of CRDB's create slope +(0.691 / 1.296) and 79 % of CRDB's drop slope (0.875 / 1.108) +reproduces against an in-memory consensus backend. The slope is +*not* purely CRDB-side. + +### Per-call latency by op × shard_kind + +Mean ms per call over the 500-rep measurement window +(`persist_external_op_latency_by_shard_kind`): + +| op | shard_kind | backend | N=10k mean ms | N=25k mean ms | growth | +|---|---|---|---:|---:|---:| +| `consensus_cas` | `user_data` | CRDB | 24.37 | 118.62 | **4.87×** | +| `consensus_cas` | `user_data` | bogo | 0.93 | 0.96 | **1.03×** | +| `consensus_cas` | `catalog` | CRDB | 2.15 | 4.09 | 1.90× | +| `consensus_cas` | `catalog` | bogo | 0.32 | 0.29 | 0.90× | +| `consensus_cas` | `txns` | CRDB | 2.03 | 3.02 | 1.49× | +| `consensus_cas` | `txns` | bogo | 0.38 | 0.33 | 0.88× | +| `consensus_scan`| `txns` | CRDB | 4.02 | 21.88 | 5.44× | +| `consensus_scan`| `txns` | bogo | 4.06 | 0.40 | 0.10× | +| `consensus_truncate` | `catalog` | CRDB | 40.00 | 54.90 | 1.37× | +| `consensus_truncate` | `catalog` | bogo | 0.44 | 0.52 | 1.20× | + +**Per-call latency is flat on bogo.** Every shard_kind/op pair +that grew on CRDB stays within ±20 % on bogo. The +`consensus_scan{shard_kind="txns"}` reading on bogo at N=25k is +based on only 52 calls (warmup discard removed most of them) so +the 0.10× is noise; the absolute ms/rep is tiny either way. + +### Per-rep CAS counts (per measure rep) + +Counts at the consensus boundary divided by REPS=500: + +| op | shard_kind | backend | N=10k/rep | N=25k/rep | growth | +|---|---|---|---:|---:|---:| +| `consensus_cas` | `catalog` | CRDB | 5.61 | 5.60 | 1.00× | +| `consensus_cas` | `catalog` | bogo | 5.58 | 5.59 | 1.00× | +| `consensus_cas` | `txns` | CRDB | 6.62 | 6.63 | 1.00× | +| `consensus_cas` | `txns` | bogo | 6.61 | 6.62 | 1.00× | +| `consensus_cas` | `user_data` | CRDB | 66.50 | 176.45 | **2.65×** | +| `consensus_cas` | `user_data` | bogo | 55.00 | 113.05 | **2.06×** | + +Two things to read off this: + +- On the **critical-path** shards (`catalog`, `txns`), per-rep CAS + count is flat (~5.6 and ~6.6 CAS per CREATE+DROP rep) + identically on both backends. The phase-15 `txns-upper` and + phase-16 fanout work cleaned this up; no count-growth bug here. +- On `user_data`, per-rep CAS count grows ~2× on both backends. + This is background compaction / heartbeat traffic on the + N-table set fanning out across many persist clients, *not* the + DDL hot path — the bench inserts a single empty table per rep + with no rows, so its only on-path `user_data` CAS calls are + during `open_data_handles_concurrent`. The 2× growth is the + background workers on the existing N tables, not work + attributable to the under-test CREATE/DROP. Per-rep wall-clock + delta is +57 ms on bogo vs +19 309 ms on CRDB — but neither is + on the DDL synchronous path. + +### Where the bogo slope lives + +`catalog_transact_phase_seconds` mean ms per measure rep, ranked +by slope on bogo: + +| phase | backend | N=10k ms | N=25k ms | slope ms/+1k | +|---|---|---:|---:|---:| +| `transact_inner` | bogo | 1.48 | 5.35 | **+0.258** | +| `prepare_state` | bogo | 0.23 | 3.22 | **+0.200** | +| `op_loop` | bogo | 0.89 | 1.60 | +0.047 | +| `coord_post_transact` | bogo | 3.88 | 4.26 | +0.025 | +| `tx_commit` | bogo | 0.93 | 1.04 | +0.007 | + +Sum of the in-table slopes: **+0.54 ms/+1k**. Adding smaller +phases below the 0.02 threshold gets us to ~+0.65 ms/+1k, which +covers ~94 % of the +0.691 ms/+1k bench-level slope. The residual +~0.04 ms/+1k sits in pgwire / parsing / response framing. + +The two dominant phases on bogo, `transact_inner` and +`prepare_state`, are *both* coordinator-side, in-memory work that +scales with catalog size: `transact_inner` walks/diffs the +in-memory catalog state; `prepare_state` is the +`storage-collections` prepare path (phase-14 added the +breakdown). Neither touches consensus. So the bogo slope is +**coordinator state-size scaling that is independent of the +consensus backend**. + +For CRDB, those two phases have the same slope: +`transact_inner` +0.226, `prepare_state` +0.164 — within 13 % of +the bogo numbers. So the same in-memory state-scaling cost is +present on CRDB; CRDB just pays it *plus* the +per-CAS-latency-growth cost on top. + +### state_apply (persist) by shard_kind & stage + +`persist_state_apply_latency_by_shard_kind`, mean ms per +`State::apply_diff` invocation, all stages and shard kinds +combined: + +| shard_kind | stage | backend | N=10k ms | N=25k ms | growth | +|---|---|---|---:|---:|---:| +| `user_data` | `total` | CRDB | 0.0100 | 0.0112 | 1.12× | +| `user_data` | `total` | bogo | 0.0076 | 0.0100 | 1.31× | +| `catalog` | `total` | CRDB | 0.0109 | 0.0120 | 1.10× | +| `catalog` | `total` | bogo | 0.0105 | 0.0107 | 1.02× | + +Per-call state-apply latency is *not* a scaling bottleneck: total +mean stays under 12 µs/call on both backends across all scales. +Per-rep budget is dominated by call *volume*, not per-call cost: +~0.8 ms/rep on user_data, ~1 ms/rep on catalog. None of this +explains the 10 ms/rep bench-level slope between N=10k and N=25k. + +### Attribution: the split + +Of the CRDB create-slope of +1.296 ms/+1k tables: + +1. **~0.69 ms/+1k (53 %) is coordinator-side and reproduces with + bogo.** It lives in `transact_inner` (+0.26) and + `prepare_state` (+0.20) — in-memory work in + `Coordinator::sequence_*` and + `StorageCollectionsImpl::prepare_state` that scales with the + number of resident collections. Consensus backend does not + matter. +2. **~0.61 ms/+1k (47 %) is the additional cost CRDB carries + from per-CAS latency growth.** Mean per-call latency on + `consensus_cas` against CRDB grows from 24 ms to 119 ms on + `user_data` and 2.2 ms to 4.1 ms on `catalog`; the + on-critical-path CAS calls (1× catalog `tx_commit` plus + ~24 user_data CAS via `open_data_handles_concurrent`) inflate + each rep by ~0.6 ms / +1k. The dominant `user_data` 4.87× + growth is mostly *background* compaction traffic on the + pre-existing N-table set; per-rep, only a small number of + those CAS calls block the bench. + +### Decision + +**Split, with numbers.** Roughly half the slope is client-side +coordinator state-size scaling (reproduces under bogo); the other +half is CRDB-side per-CAS latency growth as the `consensus` table +grows. Phase 17's "structural to persist-on-CRDB" is half right. + +What this changes vs phase 17's conclusion: + +- Phase 17 said "stop chasing the slope at the adapter / + storage-collections layer." That's wrong: half the slope *is* + there, and bogo proves the lever exists independent of the + consensus backend. +- The two specific call sites on the coordinator that scale with + catalog size are `Catalog::transact` (in + `src/catalog/src/durable/transaction.rs`) and the prepare-state + path inside `StorageCollectionsImpl` we already instrumented + in phase 14. + +### Suggested next iteration + +Attack `transact_inner` and `prepare_state`. Concretely: + +1. **`transact_inner` (+0.258 ms / +1k bogo, +0.226 ms / +1k + CRDB).** Look for an O(N) over catalog entries inside the + inner transact loop. Likely culprits: dependency walks, name + resolution refreshing on every commit, or rebuilding + in-memory indexes from scratch on each transact. Worth tracing + at N=25k with `mz_catalog_transact_phase_seconds` extended to + sub-phases inside `transact_inner` to localize within that + ~5 ms budget. +2. **`prepare_state` (+0.200 ms / +1k bogo).** Phase 14 broke + this down into `validate_and_enrich`, `sort`, + `install_collection_states`, `open_data_handles_concurrent`, + `synchronize_finalized_shards`, + `open_persist_client`. On bogo only + `install_collection_states` has a measurable slope (+0.061 + ms/+1k). The remaining +0.14 ms/+1k inside `prepare_state` is + not currently attributed — add a finer breakdown around the + borrow/lock acquisition or the snapshot of existing state at + the top of `prepare_state`. + +Both are one-file probes, both are pure coordinator work, neither +needs CRDB at all. That's where the next iteration should land +before circling back to the persist-on-CRDB half of the slope. + +If iteration 5 confirms a one-spot O(N) in `transact_inner`, +fixing it removes half the slope and pushes N=25k create_p50 +from 70 ms back toward 60 ms even on CRDB. + +### Methodology artefacts + +- `bench_phase18.py`: harness (single envd run, bogo consensus, + metrics snapshots). +- `analyze_phase18.py`: side-by-side per-call latency table. +- `/tmp/phase18_critical.py`, `/tmp/phase18_extra.py`, + `/tmp/phase18_phases.py`, `/tmp/phase18_slopes.py`: per-rep + budgeting, state_apply breakdown, catalog-transact-phase + slopes, bootstrap slope CIs. (Auxiliary; outputs are + reproducible from the snapshots in `metrics_phase18_bogo/` and + `metrics_phase17/`.) +- `mz-bogo-consensus`: in-memory consensus shim used as the + backend for phase 18 only. + +### Teardown + +`pkill` the envd / clusterd / bogo processes after the bench. +Phase 18 used a single bench run; no rerun is necessary if the +snapshots in `metrics_phase18_bogo/` are preserved. + +## Phase 19 — CRDB consensus CAS scaling root-cause + +Phase 18 attributed 47 % of the +1.296 ms/+1k create-slope to +"CRDB-side per-CAS latency growth": mean +`persist_external_op_latency_by_shard_kind{op="consensus_cas",shard_kind="user_data"}` +grew 24.4 ms → 118.6 ms (4.87×) between N=10k and N=25k on CRDB +while bogo stayed flat (0.93 → 0.96 ms). Phase 19 asks why a +single CAS against CRDB is 4.87× slower with more shards in the +consensus table, and whether that growth is in CRDB or in the +persist client. + +### Methodology + +Two probes, both at N=10k against a CRDB-backed envd +(`results_phase19/`, `metrics_phase19/`): + +1. **Quiescent CRDB inspection.** Pad to N=10k, stop envd, run + `EXPLAIN ANALYZE (VERBOSE)` on the CAS query shape from + `src/persist/src/postgres.rs`. Output saved as + `results_phase19/probe_N10k.txt` (script: + `probe_consensus.sh`). The same SQL is used at every CRDB + scale; the bench just varies the shard the query is parameterized + to. +2. **Per-call attribution from phase-17 metrics.** Diff the + `mz_persist_postgres_connpool_*` counters between + `measure_start_N{N}.prom` and `after_N{N}.prom` from + `metrics_phase17/`. Compute mean acquire ms per call (script: + `cas_attribute_phase19.py`). +3. **Pool-size variation experiment.** Pad to N=10k, run the + measurement bench at default `persist_consensus_connection_pool_max_size = 50`, + then bump the dyncfg default to 500 (one-file patch to + `src/persist-client/src/cfg.rs`, rebuild, restart envd) and + re-bench at the same scale. Compare per-shard-kind + `consensus_cas` mean latency under the two pool sizes. Bench + harness: `bench_phase19.py` (30 warmup + 200 measure reps, + metrics snapshotted at `before` / `measure_start` / `after`). + +### SQL that `compare_and_set` issues + +The full CRDB code path is in `src/persist/src/postgres.rs:380`: + +```sql +INSERT INTO consensus (shard, sequence_number, data) +SELECT $1, $2, $3 +WHERE (SELECT sequence_number FROM consensus + WHERE shard = $1 + ORDER BY sequence_number DESC LIMIT 1) = $4; +``` + +The `consensus` table schema (`src/persist/src/postgres.rs:46`): + +```sql +CREATE TABLE consensus ( + shard text NOT NULL, + sequence_number bigint NOT NULL, + data bytea NOT NULL, + PRIMARY KEY (shard, sequence_number) +) WITH (sql_stats_automatic_collection_enabled = false); +ALTER TABLE consensus CONFIGURE ZONE USING gc.ttlseconds = 600; +``` + +Both the inner sub-SELECT (a reverse scan with `LIMIT 1` on the +PK prefix `(shard, ...)`) and the outer `INSERT` are auto-commit, +one-row, single-round-trip. + +### EXPLAIN ANALYZE at N=10k + +Run against the widest `user_data` shard in the populated +consensus table (13 689 rows) and the narrowest (2 rows). Same +plan, same numbers: + +| shape | KV pairs | KV time | KV bytes | MVCC step/seek | exec time | plan | +|---|---:|---:|---:|---:|---:|---| +| CAS on shard with 13 689 rows | 1 | 793 µs | 684 B | 2/2 | 2 ms | `revscan` on `consensus_pkey`, `auto commit` | +| `head` on shard with 13 689 rows | 1 | 721 µs | 75 B | 2/2 | 845 µs | `revscan` on `consensus_pkey` | +| CAS on shard with 2 rows | 1 | 722 µs | 223 B | 2/2 | 2 ms | identical plan | + +CRDB does **not** scan tombstones at the per-CAS level — MVCC +step/seek is 2/2 regardless of shard size, because the PK +`(shard ASC, sequence_number ASC)` with `ORDER BY sequence_number +DESC LIMIT 1` is a single backward seek to the latest live key. +At N=10k the consensus table has **224 939 rows across 10 093 +shards**, p50 = 16 rows/shard, p99 = 24 rows/shard, total +on-disk = 390 MiB. The widest two shards (the `txns` and +`catalog` shards) are outliers at 13 689 / 13 021 rows; the rest +are tightly clustered. None of those shapes makes the CAS itself +slow at the CRDB level. + +So at the SQL level, **a single CAS against CRDB takes ~2 ms, +flat with shard size and consensus-table size**. The 24 ms → 119 ms +per-call growth from phase 17 cannot live in CRDB's query path. + +### Where the per-CAS time actually goes + +The phase-17 metrics carry it. Diff +`mz_persist_postgres_connpool_acquire_seconds` / +`mz_persist_postgres_connpool_acquires` over the measurement +window: + +| N | acquires | acquire_seconds | mean acquire ms | cas mean ms | acquire / cas | +|---:|---:|---:|---:|---:|---:| +| 10 000 | 44 669 | 633.8 | **14.2** | 21.0 | **77 %** | +| 15 000 | 54 958 | 1 635.5 | **29.8** | 37.9 | **87 %** | +| 20 000 | 71 937 | 4 759.2 | **66.2** | 77.3 | **92 %** | +| 25 000 | 98 730 | 9 547.4 | **96.7** | 112.7 | **91 %** | + +Persist's CRDB connection pool has `max_size = 50` (default +`persist_consensus_connection_pool_max_size` in +`src/persist-client/src/cfg.rs:389`). At N=25k there are ~1 645 +concurrent CAS calls/sec arriving across background +heartbeat/compaction/GC tasks on the 25k user_data shards. +Mean per-call acquire time blows up from 14 → 97 ms — 7× +growth that swallows 91 % of the consensus_cas budget at N=25k. +The actual CRDB work is the ~6–16 ms residue. + +**The per-CAS growth on CRDB is not CRDB getting slower. It is +the persist client's CRDB connection pool saturating under +N-shard background load.** + +### Pool-size experiment (N=10k, post-recovery state) + +Same `audit_pad` schema at N=10k under two configurations: + +| config | user_data CAS mean ms | catalog CAS mean ms | acquires/measure | acq mean ms | +|---|---:|---:|---:|---:| +| pool=50 (default) | **16.4** | 1.82 | 17 168 | 8.93 | +| pool=500 (patched default + rebuild) | **2.0** | 1.78 | 4 906 (steady-state CAS subset) | n/a | + +The patched build registered the pool with `size=500, +available=499` and the per-call `user_data` CAS mean dropped 8× +on steady-state shards. Catalog and txns CAS were already fast +(< 2 ms) under both pools; they aren't pool-bound at N=10k. + +There is a transient "unknown" shard_kind bucket in the pool=500 +window with 10 193 calls at 103 ms each — those are CAS calls on +shards that the freshly-restarted envd has not yet reopened (so +`shard_kind_for_key()` returns `"unknown"`). They are +client-side recovery traffic, not on the DDL hot path. They do +not appear in the pool=50 row because pool=50 was measured after +envd had been up long enough for the recovery surge to finish. + +The DDL-level `create_p50` did not improve from 42.5 ms (pool=50) +to 46.1 ms (pool=500) at N=10k, because the user_data CAS +contribution on the DDL hot path is small — only +`open_data_handles_concurrent` issues a small number of +user_data CAS per CREATE (phase-17: ++0.54 ms/+1k tables out of the +1.296 ms/+1k total). The +on-path catalog `tx_commit` and txns CAS calls were never +pool-bound at this scale. + +### Answers to the iteration's questions + +1. **The SQL `compare_and_set` issues**: one + `INSERT INTO consensus ... SELECT $1,$2,$3 WHERE (SELECT + sequence_number FROM consensus WHERE shard = $1 ORDER BY ... + DESC LIMIT 1) = $4`, single round trip with auto-commit on + CRDB. +2. **EXPLAIN ANALYZE numbers**: 2 ms total execution at the + widest 13 689-row `user_data` shard, identical on a 2-row + shard. KV time 721–793 µs, 1 KV pair read, MVCC step/seek + 2/2 in all cases. The plan and the numbers are independent + of shard size at this scale. +3. **Consensus row counts**: 224 939 rows / 10 093 shards at + N=10k (mean 22, p50 16, p99 24, max 13 689). At N=25k the + total scales linearly to ~525k rows; the per-CAS work in the + CRDB plan stays at 2/2 MVCC seeks because the PK prefix is + `shard`, not `sequence_number`. +4. **Schema/index**: no issue. The PK + `(shard ASC, sequence_number ASC)` makes the CAS query a + single backward seek to the latest live key on a known prefix. + The 600 s GC TTL keeps tombstones bounded; we observe + live_count / key_count ratios of 0.45–0.65 per range, well + within healthy CRDB territory. +5. **Retention growth per shard**: ≤ 24 rows per shard at p99, + capped by the rollup threshold and GC. Not the cause. + +The bottleneck is **the persist-client CRDB connection pool** +(default `max_size=50`). At N=25k, ~1.6k CAS calls/sec from +background workers across 25k shards queue at the pool. Each +caller spends ~97 ms waiting for a connection before doing +~16 ms of actual CRDB work. + +### What this changes vs phase 18's conclusion + +Phase 18 split the slope 53 % / 47 % between "coordinator-side +state-size scaling" (reproduces under bogo) and "CRDB-side +per-CAS latency growth". Phase 19 refines the second half: that +47 % is **not** CRDB-side at all. It is the persist client's +connection pool saturating. The pool is part of the same +"client side" that phase 18 said held the other half — but it +shows up only on CRDB because bogo doesn't go through the +PostgresClient pool. So the cleaner restatement is: + +- **~53 % coordinator-side, in-memory** (`transact_inner`, + `prepare_state` — visible on both backends). +- **~47 % client-side, in the persist CRDB connection pool** + (only visible on CRDB; bogo's wire path doesn't queue here). +- **~0 % CRDB-side**. The CRDB query plan is flat with shard + count and table size at this range. + +### Fix + +Increasing `persist_consensus_connection_pool_max_size` (or +equivalently the dyncfg) is the obvious mitigation. The +experiment used 500 and dropped steady-state `user_data` CAS +from 16 → 2 ms at N=10k. We did **not** ship a default change in +this iteration because: + +- The DDL hot path's `create_p50` did not improve at N=10k under + pool=500 (the on-path CAS calls aren't pool-bound at this + scale). The fix would land at N=20k+ where + `open_data_handles_concurrent` (16 → 24 ms in phase-17) starts + to feel the pool queue. +- The dyncfg name (`persist_consensus_connection_pool_max_size`) + is read at pool-builder time. `ALTER SYSTEM SET` propagates + to envd's dyncfg, but the pool is built before the system-var + sync runs in `environmentd/main.rs:982`, so changing the dyncfg + at runtime needs an envd restart anyway. Worth aligning the + init order so this can be tuned without a redeploy. +- The "right" pool size depends on workload shape; 50 is a + defensible default for low-N envs and ~200–500 looks + appropriate at N=10k–25k. A targeted change should come + with a sweep across N=5k → 50k and across pool sizes + 50/200/500/1000 to find the knee. + +### Suggested next iteration + +Re-run the phase-17 4-scale sweep with `pool_max_size = 500` +baked in. If the +0.61 ms/+1k of "CRDB-side" slope flattens out, +ship the pool default bump. If the bench-level create_p50 slope +still tracks (because the on-path catalog/txns CAS — not +user_data — drives DDL p50), pivot the search inside +`Coordinator::sequence_create_table` to find the on-path +catalog/txns CAS calls and see what part of `tx_commit` walks +the pool. Either way, the per-CAS slope on `user_data` shards is +no longer the open question. + +### Methodology artefacts + +- `bench_phase19.py`: 30 warmup + 200 measure-rep harness, single + envd run, metrics snapshots `before` / `measure_start` / + `after`. +- `analyze_phase19.py`: per-pool-size table of create/drop + latencies and per-shard-kind consensus_cas means. +- `dump_pool_phase19.py`, + `cas_attribute_phase19.py`: per-rep / per-call attribution + scripts; reproducible from `metrics_phase19/` and + `metrics_phase17/`. +- `probe_consensus.sh`: schema + row-count + EXPLAIN ANALYZE + probe against a quiescent CRDB consensus table. +- `metrics_phase19/`: before/measure_start/after snapshots for + pool=50, pool=200 (false negative — dyncfg didn't propagate), + pool=500 (with patched build). +- `results_phase19/timings_pool{50,200,500}.csv`: per-rep + create/drop latencies. +- `results_phase19/probe_N10k.txt`: EXPLAIN ANALYZE output and + consensus-table stats at N=10k. + +### Teardown + +`pkill` the envd / clusterd processes after the experiment; revert +the one-line cfg.rs default bump used to verify the pool +hypothesis (we did, the rebuild is back to `max_size = 50`). No +state needs to be retained between iterations beyond +`metrics_phase19/` and `results_phase19/`. + +## Phase 20 — pool bump confirmation sweep + +Phase 19 measured a single point (N=10k, steady state) and +concluded that +`persist_consensus_connection_pool_max_size = 500` would flatten +the +0.61 ms/+1k "CRDB-side" half of the create-slope. Phase 20 +re-runs the full phase-17 sweep (N=10k → 25k, 30 warmup + 500 +measure reps per scale, single envd run, ascending pad) with the +pool default bumped to 500 in `src/persist-client/src/cfg.rs:389` +and a fresh `--profile=optimized` rebuild. Artefacts: +`results_phase20_crdb/`, `metrics_phase20_crdb/`. + +### Bench: N=10k → N=25k, pool=500 + +| N | create_p50 ms | 95% CI | drop_p50 ms | 95% CI | +|------:|--------------:|------------------:|------------:|------------------:| +| 10000 | 47.53 | [47.11, 47.96] | 33.25 | [32.87, 33.59] | +| 15000 | 50.69 | [50.02, 51.32] | 35.94 | [35.48, 36.73] | +| 20000 | 58.11 | [57.44, 58.88] | 42.91 | [42.16, 43.75] | +| 25000 | 69.85 | [68.38, 71.84] | 50.69 | [49.62, 51.94] | + +Least-squares slopes over the four medians: + +- `create_p50` slope: **+1.487 ms / +1k tables [95% CI +1.391, +1.624]** +- `drop_p50` slope: **+1.185 ms / +1k tables [95% CI +1.117, +1.257]** + +Compared to phase 17 (pool=50): + +| metric | phase17 slope | phase20 slope | Δ slope | +|---|---|---|---:| +| create_p50 | +1.296 [1.187, 1.434] | +1.487 [1.391, 1.624] | +0.19 | +| drop_p50 | +1.108 [1.028, 1.194] | +1.185 [1.117, 1.257] | +0.08 | + +The create_p50 slope confidence intervals do **not overlap**: +phase 20 is statistically steeper than phase 17. The level shifted +*down* at low/mid N (10k: 50.91→47.53, 15k: 52.46→50.69, 20k: +58.84→58.11) and *up* at N=25k (70.39→69.85 is a wash, within CI). +The pool bump made N=10k–N=20k faster and **did not improve N=25k +at all** — it just made the slope steeper between them. + +### Did the pool bump take effect? + +Yes, unambiguously: + +- `mz_persist_postgres_connpool_size` reaches **500** at every + snapshot from N=15k onward (it's at 500 by `before_N10000` too, + i.e. post-warmup). +- Mean per-call `connpool_acquire` at N=10k dropped from **14.2 ms + (pool=50, phase17)** to **2.4 ms (pool=500, phase20)**, an 5.9× + improvement that matches phase 19's steady-state microbench + ("16 → 2 ms per CAS"). +- At N=25k, mean per-call acquire dropped from **96.7 ms** to + **65.2 ms** (1.5× improvement). The pool is **still saturating** + at the high end, just less catastrophically. + +So the dyncfg-init-order worry from phase 19 was unfounded once we +patched the default and rebuilt — the pool registers at the new +size from process start. But pool=500 is *not* a sufficient +ceiling at N=25k. + +### Where the residual slope lives (per-call mean, measure window) + +Diffing `measure_start_N{N}.prom` against `after_N{N}.prom` from +`metrics_phase20_crdb/`: + +| metric | N=10k | N=15k | N=20k | N=25k | slope ms/+1k | +|---|---:|---:|---:|---:|---:| +| `consensus_cas` user_data | 25.31 | 53.85 | 110.11 | 133.07 | **+7.59** | +| `connpool_acquire` (overall) | 2.40 | 15.66 | 52.50 | 65.18 | **+4.50** | +| `consensus_truncate` catalog | 30.69 | 24.88 | 57.85 | 84.63 | +3.90 | +| `open_data_handles_concurrent` | 15.55 | 14.89 | 18.77 | 22.85 | +0.52 | +| `transact_inner` | 1.61 | 2.65 | 3.82 | 5.11 | +0.23 | +| `prepare_state` | 0.28 | 1.01 | 1.86 | 2.78 | +0.17 | +| `consensus_scan` txns | 1.67 | 3.25 | 1.69 | 3.98 | +0.11 | +| `consensus_cas` catalog | 1.89 | 2.17 | 2.58 | 2.99 | +0.07 | +| `tx_commit` | 2.83 | 2.85 | 3.72 | 3.68 | +0.07 | +| `consensus_cas` txns | 1.80 | 1.91 | 2.04 | 2.80 | +0.06 | + +`consensus_cas` on `user_data` is still by far the dominant +per-call slope owner. But its breakdown changed: + +| N | cas user_data | pool_acquire | non-pool (cas − acquire) | +|---:|---:|---:|---:| +| 10k phase17 (pool=50) | 21.0 | 14.2 | 6.8 | +| 10k phase20 (pool=500) | 25.3 | 2.4 | **22.9** | +| 25k phase17 (pool=50) | 112.7 | 96.7 | 16.0 | +| 25k phase20 (pool=500) | 133.1 | 65.2 | **67.9** | + +The pool went from owning 91 % of CAS at N=25k to owning 49 %. +The **non-pool portion of CAS** — i.e. the round trip and CRDB +work that sits behind the pool — **grew 3–4× when we lifted the +pool throttle**. At N=25k phase17 left only 16 ms outside the +pool; phase20 has 68 ms outside the pool. Total CAS got 18 % worse +at N=25k. + +This is the cleanest explanation for why DDL p50 didn't improve at +N=25k: pool=50 was acting as a *throttle* that limited the +concurrency hitting CRDB. Lifting it shifted the queue to whatever +sits behind the pool (CRDB connection slot count, CRDB SQL CPU, +`tokio_postgres` task scheduling, or — most likely — the same +finite CRDB-side concurrency for `consensus` writes that phase 19 +measured at 2 ms per CAS *in isolation* but did not exercise under +~1.6 k CAS/sec concurrent load). + +Phase 19's microbench was a steady-state, low-concurrency probe +("user_data CAS mean dropped 16 → 2 ms" measured after the DDL +burst had finished). It captured the *idle* per-CAS latency at +pool=500; it does not generalize to the actual DDL-burst regime +where 25k user_data shards' background tasks are all hitting the +pool at once. + +### Decision: do not ship the default bump + +The mandate's first criterion was "if `connpool_acquire` is now +flat with N, ship the bump even if the slope isn't flatter." It is +**not** flat: acquire mean grows 2.4 → 65 ms across the sweep. +The DDL-level evidence: + +- N=10k–20k creates are 3–10 ms faster (the level win is real). +- N=25k creates are unchanged (69.85 vs 70.39, within CI). +- N=25k slope is statistically steeper (+1.49 vs +1.30, CIs + non-overlapping). +- Total CAS at N=25k is 18 % *worse* under pool=500 because the + non-pool component grows 4×. + +Net for a real workload (which is N-dependent), pool=500 is a wash +at the high end and a modest improvement in the middle. It is not +unambiguously safe to ship as the new default: customers running +DDL bursts on populated environments (the case we actually care +about) wouldn't see a clear win, and the unmasked downstream +contention is unquantified. We **defer** the default bump and +**revert** the cfg.rs change. + +### What this changes vs phase 19's conclusion + +Phase 19's headline was "the pool is the bottleneck; bumping it +gives an 8× CAS speedup." Both halves of that are narrower than +phase 19 wrote them: + +- The pool *is* a bottleneck at high N — confirmed: bumping it + from 50 → 500 cut acquire mean from 97 → 65 ms at N=25k. +- But the **8× speedup was measured on idle steady-state CAS at + N=10k**, not on the DDL hot path under concurrent burst load. + Under DDL burst, the same change moves the queue to a different + bottleneck behind the pool, and the user-visible CAS mean + *worsens* by 18 % at N=25k. + +The slope-attribution split phase 19 reported ("~47 % +client-side, in the persist CRDB connection pool") was correct +about *where* the slope lives at pool=50 but wrong about whether +removing the pool throttle removes the slope. It doesn't — the +slope is split across the pool and whatever sits behind it, and +relaxing one reveals the other. + +The phase 19 NOTE that the on-path catalog/txns CAS aren't pool +bound *is* corroborated: in phase 20, `cas catalog` stays +1.9–3.0 ms and `cas txns` stays 1.8–2.8 ms across the entire +sweep. Those calls do not feel the pool size at all because they +go through different pool instances or are not pool-contended. + +### Bigger picture + +Across iterations 14–20 the DDL slope at N=5k → N=25k has been +attributed to multiple roughly-equal contributors, each owning a +fraction of the +1.3 ms/+1k create-slope: + +| phase | finding | landed | +|---|---|---| +| 15 | per-DDL O(N) txns-upper fanout in `BackgroundTask` | fix shipped (chunked + shared upper) | +| 18 | bogo backend: client-side `transact_inner` / `prepare_state` growth (+0.23 + +0.16 ms/+1k) | identified, deferred | +| 19 | persist CRDB connection-pool saturation | identified | +| 20 | phase-19's fix made the slope worse; the bottleneck-behind-the-pool is the new top contributor | deferred | + +The marginal return is decreasing. Phase 15 erased a +0.6 ms/+1k +contributor. Phase 18's transact_inner/prepare_state would buy us ++0.4 ms/+1k (combined) if fixed. Phase 19/20's pool / behind-pool +contributors are entangled and need a coordinated change (pool + +CRDB-side capacity, or batching to reduce the CAS rate). No +single big-bang fix remains. + +### Suggested next iteration + +The pool slope at pool=500 is +4.50 ms/+1k call. Don't spend +another iteration on it as a single knob — increase the pool +further and the slope just shifts to behind-pool. The cleaner +question is **why is the per-CAS rate so high in the first place**: +phase 17 measured ~176 user_data CAS per CREATE+DROP rep. Most of +that is background compaction/heartbeat traffic on the existing +N shards, not DDL-driven. If a future iteration wants to flatten +the slope at the persist layer, the highest-leverage change is +reducing the per-shard background CAS rate (e.g. coalescing +heartbeat ticks across shards, raising +`persist_consensus_connection_pool_max_age`, batching compaction +state writes) so the foreground DDL CAS doesn't queue. + +Alternatively, pivot to the bogo-visible contributors +(`transact_inner` + `prepare_state`, +0.4 ms/+1k combined): those +are visible without CRDB in the picture and are clean targets in +code we own. + +### Methodology artefacts + +- `bench_phase20.py` (in `/home/ubuntu/envd-ddl-investigation/`): + same 30 warmup + 500 measure harness as phase 17, single envd + run, ascending pad, before/measure_start/after metrics snapshots + at every scale. +- `analyze_phase20.py`: per-scale median + 95 % bootstrap CI on + the measure window, least-squares slope ms/+1k with bootstrap + CI, and the per-call attribution table reproduced above. +- `compare_phase17_phase20.py`: side-by-side per-call attribution + for the two pool sizes. (Slow on 100 MB+ prom files; use + `analyze_phase20.py` for the headline numbers.) +- `metrics_phase20_crdb/`: before/measure_start/after snapshots + per scale; `results_phase20_crdb/timings_N{N}.csv`: per-rep + create/drop latencies (filter `phase=="measure"`). + +### Teardown + +Revert the cfg.rs default bump (back to `max_size = 50`); pkill +envd + clusterd. The phase 20 build was a one-line patch over +phase 17's code; no source change ships in this iteration. diff --git a/test/envd-ddl-scalability/README.md b/test/envd-ddl-scalability/README.md new file mode 100644 index 0000000000000..0dc0e7ae9edbe --- /dev/null +++ b/test/envd-ddl-scalability/README.md @@ -0,0 +1,102 @@ +# envd DDL scalability audit + +A tight, standalone reproducer for measuring how DDL and catalog-transaction +latency scale with the number of objects in the catalog. Intended for +**profiling**, not for CI tracking — the canonical numbers live in +`test/cluster-spec-sheet`. + +The harness connects to a **running** `environmentd` instance via psycopg. +It does **not** start its own envd. That makes the iteration loop tight: +start envd once (optionally with tracing or attached to a profiler) and run +the harness against it as many times as you like. + +## Quick start + +### 1. Start environmentd + +For tracing (Tempo / Grafana on `localhost:3000`): + +```bash +bin/mzcompose --find monitoring run default +bin/environmentd --optimized --monitoring --reset +``` + +Or for plain fast iteration (no tracing): + +```bash +bin/environmentd --reset +``` + +### 2. Run the audit + +```bash +python3 test/envd-ddl-scalability/audit.py \ + --padding tables \ + --scale 0,1000,5000 \ + --ops create_table,drop_table,alter_table_add_col,rename_table \ + --reps 10 +``` + +Replace `--padding tables` with `mvs`, `views`, or `indexes` to vary the +catalog along a different axis. `--ops` accepts a comma-separated subset of +the operations listed below. + +## What it measures + +For each `(padding, scale_point, op)` cell, the harness runs `--reps` +repetitions of `op`, timing only the measured DDL statement (each rep has +its own per-rep `setup_each` / `after_each` that is **not** timed). It +reports p50 / p95 / max latency in milliseconds. + +### Padding axes (`--padding`) + +| value | what fills the catalog | +| --- | --- | +| `tables` | N empty tables (no dataflows; pure catalog) | +| `views` | N views with distinct projections (no dataflows; pure catalog + plan cache) | +| `mvs` | N materialized views, sharded across pad clusters | +| `indexes` | N indexes on a single base table, sharded across pad clusters | + +### Measured ops (`--ops`) + +| op | what it times | +| --- | --- | +| `create_table` | `CREATE TABLE m_tmp (a int)` | +| `drop_table` | `DROP TABLE m_tmp` (pre-created in setup_each) | +| `alter_table_add_col` | `ALTER TABLE m_tmp ADD COLUMN m_col_ text` | +| `rename_table` | `ALTER TABLE m_tmp RENAME TO m_tmp_renamed` | +| `create_view` | `CREATE VIEW m_v AS SELECT ::int AS x` | +| `drop_view` | `DROP VIEW m_v` (pre-created in setup_each) | +| `create_mv` | `CREATE MATERIALIZED VIEW m_mv IN CLUSTER m_c AS SELECT a + FROM m_base` | +| `drop_mv` | `DROP MATERIALIZED VIEW m_mv` (pre-created in setup_each) | +| `create_index` | `CREATE INDEX m_idx IN CLUSTER m_c ON m_base ((a + ))` | +| `drop_index` | `DROP INDEX m_idx` (pre-created in setup_each) | + +## Profiling + +### Tracing (Tempo) + +Pass `--trace-out /tmp/audit-traces.csv` to record a trace ID per measured +statement (requires envd started with `--monitoring`; the harness enables +`emit_trace_id_notice` automatically). Look up trace IDs at +`http://localhost:3200/api/traces/` or in Grafana, and analyse them with +`.claude/skills/mz-query-tracing/trace_tree.py`. + +The `--system-parameter-default 'opentelemetry_filter=debug'` flag on envd +is useful for deeper spans. + +### CPU profile (samply) + +While envd is running, attach `samply`: + +```bash +samply record -p $(pgrep -f 'target/release/environmentd') & +python3 test/envd-ddl-scalability/audit.py --padding tables --scale 5000 --ops create_table --reps 100 +# Ctrl-C samply; open the flame graph +``` + +### Targeted instrumentation + +Once tracing/CPU has narrowed the surface, add scoped +`#[tracing::instrument]` or `Instant::now()` timers in the suspected code +paths and re-run. Trace IDs from the harness map directly to the new spans. diff --git a/test/envd-ddl-scalability/audit.py b/test/envd-ddl-scalability/audit.py new file mode 100755 index 0000000000000..9493779824c52 --- /dev/null +++ b/test/envd-ddl-scalability/audit.py @@ -0,0 +1,620 @@ +#!/usr/bin/env python3 +# Copyright Materialize, Inc. and contributors. All rights reserved. +# +# Use of this software is governed by the Business Source License +# included in the LICENSE file at the root of this repository. +# +# 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. + +"""DDL/catalog scaling audit harness. + +See README.md for usage. Connects to a running environmentd, pads the +catalog with N objects of a chosen type, and times CREATE/DROP/ALTER/RENAME +of various object types at each scale point. +""" + +import argparse +import statistics +import sys +import time +from collections.abc import Callable +from dataclasses import dataclass + +import psycopg + +# Object name prefixes / schemas / clusters used by the harness. The +# `audit_` prefix lets `--reset` and ad-hoc cleanup target them safely. +PAD_SCHEMA = "audit_pad" +MEAS_SCHEMA = "audit_meas" +MEAS_CLUSTER = "audit_meas_c" +PAD_CLUSTER_PREFIX = "audit_pad_c_" + +# Pad MVs/indexes per cluster. Keeps dataflow count per replica modest so +# small replica sizes can host the pad load. +PAD_DATAFLOWS_PER_CLUSTER = 400 + + +# --------------------------------------------------------------------------- +# Connection helpers +# --------------------------------------------------------------------------- + + +class TraceCollector: + """Collects `trace id: ...` NOTICE lines from a Materialize connection. + + Materialize emits one such NOTICE per statement when + `emit_trace_id_notice` is enabled. We track them in a list and the + measurement loop pops the most recent one after each timed statement. + """ + + def __init__(self) -> None: + self.trace_ids: list[str] = [] + + def handle(self, diag: psycopg.errors.Diagnostic) -> None: + msg = diag.message_primary or "" + if msg.startswith("trace id: "): + self.trace_ids.append(msg[len("trace id: ") :].strip()) + + +def connect(url: str, trace_collector: TraceCollector) -> psycopg.Connection: + conn = psycopg.connect(url, autocommit=True) + conn.add_notice_handler(trace_collector.handle) + return conn + + +def run(conn: psycopg.Connection, sql: str) -> None: + with conn.cursor() as cur: + cur.execute(sql.encode()) + + +def time_one( + conn: psycopg.Connection, sql: str, traces: TraceCollector +) -> tuple[float, str | None]: + """Time a single DDL statement; return (ms, trace_id_or_None).""" + before = len(traces.trace_ids) + start = time.perf_counter() + with conn.cursor() as cur: + cur.execute(sql.encode()) + elapsed_ms = (time.perf_counter() - start) * 1000.0 + trace_id = traces.trace_ids[-1] if len(traces.trace_ids) > before else None + return elapsed_ms, trace_id + + +# --------------------------------------------------------------------------- +# Padding strategies +# --------------------------------------------------------------------------- + + +class PadStrategy: + """Fills the catalog with N objects. Designed to ramp incrementally so + a scale ladder (e.g. 0,1000,5000) doesn't redo work between points.""" + + name: str + + def init(self, conn: psycopg.Connection) -> None: + raise NotImplementedError + + def add_n(self, conn: psycopg.Connection, current_n: int, target_n: int) -> None: + raise NotImplementedError + + def teardown(self, conn: psycopg.Connection) -> None: + raise NotImplementedError + + +def _drop_pad_clusters(conn: psycopg.Connection) -> None: + cur = conn.execute( + f"SELECT name FROM mz_clusters WHERE name LIKE '{PAD_CLUSTER_PREFIX}%'" + ) + for (name,) in cur.fetchall(): + run(conn, f"DROP CLUSTER IF EXISTS {name} CASCADE") + + +def _reset_pad_schema(conn: psycopg.Connection) -> None: + run(conn, f"DROP SCHEMA IF EXISTS {PAD_SCHEMA} CASCADE") + run(conn, f"CREATE SCHEMA {PAD_SCHEMA}") + + +class TablesPadding(PadStrategy): + name = "tables" + + def init(self, conn: psycopg.Connection) -> None: + _reset_pad_schema(conn) + + def add_n(self, conn: psycopg.Connection, current_n: int, target_n: int) -> None: + for i in range(current_n + 1, target_n + 1): + run( + conn, + f"CREATE TABLE IF NOT EXISTS {PAD_SCHEMA}.pad_t_{i} " + f"(a int, b text)", + ) + + def teardown(self, conn: psycopg.Connection) -> None: + run(conn, f"DROP SCHEMA IF EXISTS {PAD_SCHEMA} CASCADE") + + +class ViewsPadding(PadStrategy): + name = "views" + + def init(self, conn: psycopg.Connection) -> None: + _reset_pad_schema(conn) + + def add_n(self, conn: psycopg.Connection, current_n: int, target_n: int) -> None: + for i in range(current_n + 1, target_n + 1): + # Parameterise on i so views are structurally distinct. + run( + conn, + f"CREATE VIEW {PAD_SCHEMA}.pad_v_{i} AS " + f"SELECT {i}::int AS x, '{i}'::text AS y", + ) + + def teardown(self, conn: psycopg.Connection) -> None: + run(conn, f"DROP SCHEMA IF EXISTS {PAD_SCHEMA} CASCADE") + + +class MvsPadding(PadStrategy): + name = "mvs" + + def __init__(self, pad_cluster_size: str) -> None: + self.pad_cluster_size = pad_cluster_size + self._pad_clusters = 0 + + def init(self, conn: psycopg.Connection) -> None: + _drop_pad_clusters(conn) + _reset_pad_schema(conn) + run(conn, f"CREATE TABLE {PAD_SCHEMA}.pad_base (id int, val text)") + run(conn, f"INSERT INTO {PAD_SCHEMA}.pad_base VALUES (1, 'x')") + + def _ensure_cluster(self, conn: psycopg.Connection, idx: int) -> None: + while self._pad_clusters <= idx: + k = self._pad_clusters + # CREATE CLUSTER has no IF NOT EXISTS form; drop-then-create. + run(conn, f"DROP CLUSTER IF EXISTS {PAD_CLUSTER_PREFIX}{k} CASCADE") + run( + conn, + f"CREATE CLUSTER {PAD_CLUSTER_PREFIX}{k} " + f"SIZE '{self.pad_cluster_size}'", + ) + self._pad_clusters += 1 + + def add_n(self, conn: psycopg.Connection, current_n: int, target_n: int) -> None: + i = current_n + 1 + while i <= target_n: + cluster_idx = (i - 1) // PAD_DATAFLOWS_PER_CLUSTER + self._ensure_cluster(conn, cluster_idx) + cluster_end = min(target_n, (cluster_idx + 1) * PAD_DATAFLOWS_PER_CLUSTER) + for j in range(i, cluster_end + 1): + run( + conn, + f"CREATE MATERIALIZED VIEW IF NOT EXISTS " + f"{PAD_SCHEMA}.pad_mv_{j} " + f"IN CLUSTER {PAD_CLUSTER_PREFIX}{cluster_idx} " + f"AS SELECT id, val FROM {PAD_SCHEMA}.pad_base " + f"WHERE id < {j}", + ) + i = cluster_end + 1 + + def teardown(self, conn: psycopg.Connection) -> None: + run(conn, f"DROP SCHEMA IF EXISTS {PAD_SCHEMA} CASCADE") + _drop_pad_clusters(conn) + self._pad_clusters = 0 + + +class IndexesPadding(PadStrategy): + name = "indexes" + + def __init__(self, pad_cluster_size: str) -> None: + self.pad_cluster_size = pad_cluster_size + self._pad_clusters = 0 + + def init(self, conn: psycopg.Connection) -> None: + _drop_pad_clusters(conn) + _reset_pad_schema(conn) + run(conn, f"CREATE TABLE {PAD_SCHEMA}.pad_base (a int, b int, c int)") + run(conn, f"INSERT INTO {PAD_SCHEMA}.pad_base VALUES (1, 2, 3)") + + def _ensure_cluster(self, conn: psycopg.Connection, idx: int) -> None: + while self._pad_clusters <= idx: + k = self._pad_clusters + # CREATE CLUSTER has no IF NOT EXISTS form; drop-then-create. + run(conn, f"DROP CLUSTER IF EXISTS {PAD_CLUSTER_PREFIX}{k} CASCADE") + run( + conn, + f"CREATE CLUSTER {PAD_CLUSTER_PREFIX}{k} " + f"SIZE '{self.pad_cluster_size}'", + ) + self._pad_clusters += 1 + + def add_n(self, conn: psycopg.Connection, current_n: int, target_n: int) -> None: + i = current_n + 1 + while i <= target_n: + cluster_idx = (i - 1) // PAD_DATAFLOWS_PER_CLUSTER + self._ensure_cluster(conn, cluster_idx) + cluster_end = min(target_n, (cluster_idx + 1) * PAD_DATAFLOWS_PER_CLUSTER) + for j in range(i, cluster_end + 1): + # Distinct expression per index so we don't dedupe. + run( + conn, + f"CREATE INDEX IF NOT EXISTS pad_idx_{j} " + f"IN CLUSTER {PAD_CLUSTER_PREFIX}{cluster_idx} " + f"ON {PAD_SCHEMA}.pad_base ((a + {j}))", + ) + i = cluster_end + 1 + + def teardown(self, conn: psycopg.Connection) -> None: + run(conn, f"DROP SCHEMA IF EXISTS {PAD_SCHEMA} CASCADE") + _drop_pad_clusters(conn) + self._pad_clusters = 0 + + +PAD_STRATEGIES: dict[str, Callable[[argparse.Namespace], PadStrategy]] = { + "tables": lambda args: TablesPadding(), + "views": lambda args: ViewsPadding(), + "mvs": lambda args: MvsPadding(args.pad_cluster_size), + "indexes": lambda args: IndexesPadding(args.pad_cluster_size), +} + + +# --------------------------------------------------------------------------- +# Measured operations +# --------------------------------------------------------------------------- + + +@dataclass +class MeasuredOp: + name: str + # SQL run once before all reps of this op at a given scale point. + setup_once: Callable[[], list[str]] = lambda: [] + # SQL run before each rep (timed work excluded). Receives rep index. + setup_each: Callable[[int], list[str]] = lambda rep: [] + # The single SQL statement we time. Receives rep index. + query: Callable[[int], str] = lambda rep: "" + # SQL run after each rep (timed work excluded). Receives rep index. + after_each: Callable[[int], list[str]] = lambda rep: [] + + +def _ensure_meas_table(conn: psycopg.Connection) -> None: + run(conn, f"CREATE TABLE IF NOT EXISTS {MEAS_SCHEMA}.m_base (a int)") + # idempotent — insert only if empty + cur = conn.execute(f"SELECT count(*) FROM {MEAS_SCHEMA}.m_base") + (n,) = cur.fetchone() or (0,) + if n == 0: + run(conn, f"INSERT INTO {MEAS_SCHEMA}.m_base VALUES (1)") + + +# Op definitions. Each closure binds nothing — schema/cluster come from +# module-level constants. Per-rep setup keeps measurement-side state +# clean between reps without re-timing. + +MEASURED_OPS: dict[str, MeasuredOp] = { + "create_table": MeasuredOp( + name="create_table", + setup_each=lambda rep: [f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp CASCADE"], + query=lambda rep: f"CREATE TABLE {MEAS_SCHEMA}.m_tmp (a int)", + after_each=lambda rep: [f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp CASCADE"], + ), + "drop_table": MeasuredOp( + name="drop_table", + setup_each=lambda rep: [ + f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp CASCADE", + f"CREATE TABLE {MEAS_SCHEMA}.m_tmp (a int)", + ], + query=lambda rep: f"DROP TABLE {MEAS_SCHEMA}.m_tmp", + ), + "alter_table_add_col": MeasuredOp( + name="alter_table_add_col", + setup_once=lambda: [ + f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp CASCADE", + f"CREATE TABLE {MEAS_SCHEMA}.m_tmp (a int)", + ], + # Columns accumulate across reps — that's fine; the cost we care + # about is the catalog transaction, not the resulting schema width. + query=lambda rep: ( + f"ALTER TABLE {MEAS_SCHEMA}.m_tmp " f"ADD COLUMN m_col_{rep} text" + ), + ), + "rename_table": MeasuredOp( + name="rename_table", + setup_each=lambda rep: [ + f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp CASCADE", + f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp_renamed CASCADE", + f"CREATE TABLE {MEAS_SCHEMA}.m_tmp (a int)", + ], + query=lambda rep: ( + f"ALTER TABLE {MEAS_SCHEMA}.m_tmp " f"RENAME TO m_tmp_renamed" + ), + after_each=lambda rep: [ + f"DROP TABLE IF EXISTS {MEAS_SCHEMA}.m_tmp_renamed CASCADE", + ], + ), + "create_view": MeasuredOp( + name="create_view", + setup_each=lambda rep: [f"DROP VIEW IF EXISTS {MEAS_SCHEMA}.m_v CASCADE"], + query=lambda rep: (f"CREATE VIEW {MEAS_SCHEMA}.m_v AS SELECT {rep}::int AS x"), + after_each=lambda rep: [f"DROP VIEW IF EXISTS {MEAS_SCHEMA}.m_v CASCADE"], + ), + "drop_view": MeasuredOp( + name="drop_view", + setup_each=lambda rep: [ + f"DROP VIEW IF EXISTS {MEAS_SCHEMA}.m_v CASCADE", + f"CREATE VIEW {MEAS_SCHEMA}.m_v AS SELECT {rep}::int AS x", + ], + query=lambda rep: f"DROP VIEW {MEAS_SCHEMA}.m_v", + ), + "create_mv": MeasuredOp( + name="create_mv", + setup_each=lambda rep: [ + f"DROP MATERIALIZED VIEW IF EXISTS {MEAS_SCHEMA}.m_mv CASCADE", + ], + query=lambda rep: ( + f"CREATE MATERIALIZED VIEW {MEAS_SCHEMA}.m_mv " + f"IN CLUSTER {MEAS_CLUSTER} " + f"AS SELECT a + {rep} AS x FROM {MEAS_SCHEMA}.m_base" + ), + after_each=lambda rep: [ + f"DROP MATERIALIZED VIEW IF EXISTS {MEAS_SCHEMA}.m_mv CASCADE", + ], + ), + "drop_mv": MeasuredOp( + name="drop_mv", + setup_each=lambda rep: [ + f"DROP MATERIALIZED VIEW IF EXISTS {MEAS_SCHEMA}.m_mv CASCADE", + f"CREATE MATERIALIZED VIEW {MEAS_SCHEMA}.m_mv " + f"IN CLUSTER {MEAS_CLUSTER} " + f"AS SELECT a + {rep} AS x FROM {MEAS_SCHEMA}.m_base", + ], + query=lambda rep: f"DROP MATERIALIZED VIEW {MEAS_SCHEMA}.m_mv", + ), + "create_index": MeasuredOp( + name="create_index", + setup_each=lambda rep: [ + f"DROP INDEX IF EXISTS {MEAS_SCHEMA}.m_idx CASCADE", + ], + query=lambda rep: ( + f"CREATE INDEX m_idx IN CLUSTER {MEAS_CLUSTER} " + f"ON {MEAS_SCHEMA}.m_base ((a + {rep}))" + ), + after_each=lambda rep: [f"DROP INDEX IF EXISTS {MEAS_SCHEMA}.m_idx CASCADE"], + ), + "drop_index": MeasuredOp( + name="drop_index", + setup_each=lambda rep: [ + f"DROP INDEX IF EXISTS {MEAS_SCHEMA}.m_idx CASCADE", + f"CREATE INDEX m_idx IN CLUSTER {MEAS_CLUSTER} " + f"ON {MEAS_SCHEMA}.m_base ((a + {rep}))", + ], + query=lambda rep: f"DROP INDEX {MEAS_SCHEMA}.m_idx", + ), +} + + +# --------------------------------------------------------------------------- +# Measurement loop +# --------------------------------------------------------------------------- + + +@dataclass +class Sample: + padding: str + scale: int + op: str + rep: int + ms: float + trace_id: str | None + + +def measure_op( + conn: psycopg.Connection, + traces: TraceCollector, + op: MeasuredOp, + padding: str, + scale: int, + reps: int, +) -> list[Sample]: + for sql in op.setup_once(): + run(conn, sql) + samples: list[Sample] = [] + for rep in range(reps): + for sql in op.setup_each(rep): + run(conn, sql) + ms, trace_id = time_one(conn, op.query(rep), traces) + for sql in op.after_each(rep): + run(conn, sql) + samples.append(Sample(padding, scale, op.name, rep, ms, trace_id)) + return samples + + +def setup_measurement_state(conn: psycopg.Connection, meas_cluster_size: str) -> None: + """Idempotent setup of the schema/cluster used by measured ops.""" + run(conn, f"CREATE SCHEMA IF NOT EXISTS {MEAS_SCHEMA}") + # CREATE CLUSTER doesn't support IF NOT EXISTS in Materialize; drop-then-create. + run(conn, f"DROP CLUSTER IF EXISTS {MEAS_CLUSTER} CASCADE") + run(conn, f"CREATE CLUSTER {MEAS_CLUSTER} SIZE '{meas_cluster_size}'") + _ensure_meas_table(conn) + + +def teardown_measurement_state(conn: psycopg.Connection) -> None: + run(conn, f"DROP SCHEMA IF EXISTS {MEAS_SCHEMA} CASCADE") + run(conn, f"DROP CLUSTER IF EXISTS {MEAS_CLUSTER} CASCADE") + + +# --------------------------------------------------------------------------- +# Reporting +# --------------------------------------------------------------------------- + + +def _percentile(values: list[float], pct: float) -> float: + s = sorted(values) + if not s: + return 0.0 + k = max(0, min(len(s) - 1, int(round(pct / 100.0 * (len(s) - 1))))) + return s[k] + + +def print_summary(samples: list[Sample]) -> None: + """Aggregate samples by (padding, scale, op) and print a table.""" + by_key: dict[tuple[str, int, str], list[float]] = {} + for s in samples: + by_key.setdefault((s.padding, s.scale, s.op), []).append(s.ms) + header = ( + f"{'padding':<10} {'scale':>8} {'op':<22} " + f"{'n':>3} {'p50_ms':>8} {'p95_ms':>8} " + f"{'max_ms':>8} {'mean_ms':>9}" + ) + print(header) + print("-" * len(header)) + for (padding, scale, op), ms_list in sorted(by_key.items()): + n = len(ms_list) + p50 = _percentile(ms_list, 50) + p95 = _percentile(ms_list, 95) + mx = max(ms_list) + mean = statistics.fmean(ms_list) + print( + f"{padding:<10} {scale:>8d} {op:<22} {n:>3d} " + f"{p50:>8.1f} {p95:>8.1f} {mx:>8.1f} {mean:>9.1f}" + ) + + +def write_trace_csv(path: str, samples: list[Sample]) -> None: + """Write one row per sample: padding, scale, op, rep, ms, trace_id.""" + with open(path, "w") as f: + f.write("padding,scale,op,rep,ms,trace_id\n") + for s in samples: + f.write( + f"{s.padding},{s.scale},{s.op},{s.rep}," + f"{s.ms:.3f},{s.trace_id or ''}\n" + ) + + +# --------------------------------------------------------------------------- +# CLI +# --------------------------------------------------------------------------- + + +def parse_args() -> argparse.Namespace: + p = argparse.ArgumentParser( + description="DDL/catalog scaling audit harness", + formatter_class=argparse.ArgumentDefaultsHelpFormatter, + ) + p.add_argument( + "--url", + default="postgres://materialize@localhost:6875/materialize", + help="psycopg connection URL to a running environmentd", + ) + p.add_argument( + "--padding", + required=True, + choices=sorted(PAD_STRATEGIES.keys()), + help="which type of object fills the catalog", + ) + p.add_argument( + "--scale", + default="0,1000,5000", + help="comma-separated N values; pad ramps incrementally", + ) + p.add_argument( + "--ops", + default=( + "create_table,drop_table,alter_table_add_col,rename_table," + "create_view,drop_view,create_mv,drop_mv," + "create_index,drop_index" + ), + help="comma-separated measured ops", + ) + p.add_argument("--reps", type=int, default=10, help="reps per (scale, op)") + p.add_argument( + "--pad-cluster-size", + default="scale=1,workers=1", + help="SIZE for pad clusters (mvs / indexes padding)", + ) + p.add_argument( + "--meas-cluster-size", + default="scale=1,workers=1", + help="SIZE for the measurement cluster", + ) + p.add_argument( + "--trace-out", + default=None, + help="optional CSV path to write per-sample timings + trace ids", + ) + p.add_argument( + "--keep-state", + action="store_true", + help="skip teardown of pad+meas state at the end", + ) + return p.parse_args() + + +def main() -> int: + args = parse_args() + + scale_ladder = sorted({int(s) for s in args.scale.split(",")}) + op_names = [o.strip() for o in args.ops.split(",") if o.strip()] + bad = [o for o in op_names if o not in MEASURED_OPS] + if bad: + print(f"unknown ops: {bad}", file=sys.stderr) + print(f"valid ops: {sorted(MEASURED_OPS.keys())}", file=sys.stderr) + return 2 + ops = [MEASURED_OPS[o] for o in op_names] + padding = PAD_STRATEGIES[args.padding](args) + + traces = TraceCollector() + conn = connect(args.url, traces) + run(conn, "SET emit_trace_id_notice = true") + + print( + f"# padding={padding.name} scale={scale_ladder} " + f"ops={op_names} reps={args.reps}" + ) + + samples: list[Sample] = [] + try: + setup_measurement_state(conn, args.meas_cluster_size) + padding.init(conn) + + current_n = 0 + for target_n in scale_ladder: + if target_n > current_n: + t0 = time.time() + padding.add_n(conn, current_n, target_n) + print( + f"# padded {padding.name} " + f"{current_n} -> {target_n} " + f"({time.time() - t0:.1f}s)" + ) + current_n = target_n + for op in ops: + op_samples = measure_op( + conn, traces, op, padding.name, target_n, args.reps + ) + samples.extend(op_samples) + ms = [s.ms for s in op_samples] + print( + f" scale={target_n:>6d} {op.name:<22} " + f"p50={_percentile(ms, 50):>7.1f}ms " + f"p95={_percentile(ms, 95):>7.1f}ms " + f"max={max(ms):>7.1f}ms" + ) + finally: + if not args.keep_state: + try: + padding.teardown(conn) + except Exception as e: + print(f"warning: padding teardown failed: {e}", file=sys.stderr) + try: + teardown_measurement_state(conn) + except Exception as e: + print(f"warning: meas teardown failed: {e}", file=sys.stderr) + + print() + print_summary(samples) + if args.trace_out: + write_trace_csv(args.trace_out, samples) + print(f"\nwrote {len(samples)} samples to {args.trace_out}") + + return 0 + + +if __name__ == "__main__": + sys.exit(main()) diff --git a/test/envd-ddl-scalability/summarize_traces.py b/test/envd-ddl-scalability/summarize_traces.py new file mode 100755 index 0000000000000..d782d4bc54233 --- /dev/null +++ b/test/envd-ddl-scalability/summarize_traces.py @@ -0,0 +1,172 @@ +#!/usr/bin/env python3 +# Copyright Materialize, Inc. and contributors. All rights reserved. +# +# Use of this software is governed by the Business Source License +# included in the LICENSE file at the root of this repository. +# +# 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. + +"""Bulk-fetch traces from Tempo and summarize self-time per span name. + +Reads a CSV produced by audit.py (--trace-out), picks N traces per +(padding, scale, op) cell, fetches each from Tempo, and prints a table +of self-time per span name aggregated across the sampled traces. + +The point is to see *how a span's cost scales with N* without having to +eyeball individual trace trees: if `apply_updates` self-time grows +linearly with `scale`, that's the smoking gun. + +Usage: + python3 summarize_traces.py audit-tables.csv \\ + [--tempo http://localhost:3200] \\ + [--per-cell 2] \\ + [--top 10] +""" + +import argparse +import base64 +import csv +import json +import sys +import time +import urllib.error +import urllib.request +from collections import defaultdict + + +def fetch_trace(tempo: str, trace_id: str, tries: int = 5) -> dict | None: + """Fetch a trace by ID with retry; returns parsed JSON or None.""" + url = f"{tempo}/api/traces/{trace_id}" + for i in range(tries): + try: + with urllib.request.urlopen(url, timeout=10) as r: + data = json.loads(r.read()) + if data.get("batches"): + return data + except urllib.error.HTTPError as e: + if e.code != 404: + print(f" http {e.code} for {trace_id}", file=sys.stderr) + except Exception as e: + print(f" fetch error {trace_id}: {e}", file=sys.stderr) + time.sleep(2 + i) + return None + + +def parse_spans(trace: dict) -> list[dict]: + """Extract flat span list with name + duration.""" + spans = [] + for batch in trace.get("batches", []): + for scope in batch.get("scopeSpans", []): + for s in scope.get("spans", []): + start = int(s["startTimeUnixNano"]) + end = int(s["endTimeUnixNano"]) + span_id = base64.b64decode(s.get("spanId", "")).hex() + parent_id = base64.b64decode(s.get("parentSpanId", "") or "").hex() + spans.append( + { + "name": s["name"], + "id": span_id, + "parent": parent_id, + "dur_ns": end - start, + } + ) + return spans + + +def self_times(spans: list[dict]) -> dict[str, int]: + """Sum self-time ns per span name across all spans in this trace.""" + children: dict[str, list[dict]] = defaultdict(list) + by_id = {s["id"]: s for s in spans} + for s in spans: + if s["parent"] and s["parent"] in by_id: + children[s["parent"]].append(s) + out: dict[str, int] = defaultdict(int) + for s in spans: + child_ns = sum(c["dur_ns"] for c in children.get(s["id"], [])) + st = max(0, s["dur_ns"] - child_ns) + out[s["name"]] += st + return out + + +def fmt_us(ns: int) -> str: + if ns >= 1_000_000: + return f"{ns/1_000_000:.1f}ms" + if ns >= 1_000: + return f"{ns/1_000:.0f}μs" + return f"{ns}ns" + + +def main() -> int: + ap = argparse.ArgumentParser(description=__doc__) + ap.add_argument("csv", help="audit.py --trace-out CSV") + ap.add_argument("--tempo", default="http://localhost:3200") + ap.add_argument( + "--per-cell", + type=int, + default=2, + help="how many trace samples to pull per (padding,scale,op)", + ) + ap.add_argument( + "--top", + type=int, + default=12, + help="top N spans by self-time to display per cell", + ) + ap.add_argument( + "--min-ms", + type=float, + default=0.5, + help="only show spans with avg self-time >= this many ms", + ) + args = ap.parse_args() + + # Group rows by (padding, scale, op); pick up to per-cell trace ids each. + by_cell: dict[tuple[str, int, str], list[str]] = defaultdict(list) + with open(args.csv) as f: + for row in csv.DictReader(f): + if not row["trace_id"]: + continue + cell = (row["padding"], int(row["scale"]), row["op"]) + if len(by_cell[cell]) < args.per_cell: + by_cell[cell].append(row["trace_id"]) + + # Fetch traces and aggregate self-time by span name per cell. + print(f"# fetching traces from {args.tempo}") + for cell in sorted(by_cell.keys()): + padding, scale, op = cell + per_name_ns: dict[str, list[int]] = defaultdict(list) + for tid in by_cell[cell]: + tr = fetch_trace(args.tempo, tid) + if tr is None: + print(f" miss {tid} ({padding} scale={scale} {op})", file=sys.stderr) + continue + for name, ns in self_times(parse_spans(tr)).items(): + per_name_ns[name].append(ns) + if not per_name_ns: + continue + + # Average self-time per span name across the sampled traces. + avg = sorted( + [ + (n, sum(v) / len(v)) + for n, v in per_name_ns.items() + if sum(v) / len(v) / 1_000_000 >= args.min_ms + ], + key=lambda x: -x[1], + ) + print() + print( + f"=== padding={padding} scale={scale} op={op} " + f"(n_traces={len(by_cell[cell])}) ===" + ) + print(f"{'avg_self':>10} span") + for name, ns in avg[: args.top]: + print(f"{fmt_us(int(ns)):>10} {name}") + + return 0 + + +if __name__ == "__main__": + sys.exit(main())