From 41d44079c410e498ec4e3b1fbe8ceaffdf411e80 Mon Sep 17 00:00:00 2001 From: Moritz Hoffmann Date: Thu, 26 Feb 2026 14:17:13 +0100 Subject: [PATCH] Add persist-backed compute introspection sources MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds per-replica introspection sources backed by persist shards, enabling queryable introspection data that survives replica restarts. Each replica gets a dedicated schema (mz_introspection_{cluster}_{replica}) containing sources and views derived from compute log variants. Sources live in a separate durable collection while views use the regular items collection. Cleanup is handled via the dependency system through cluster_replica_dependents → schema_dependents. Co-Authored-By: Claude Opus 4.6 --- PLAN.md | 55 + ci/test/pipeline.template.yml | 11 + .../20250225_persist_backed_introspection.md | 369 ++ misc/python/materialize/mzcompose/__init__.py | 1 + .../materialize/parallel_workload/action.py | 1 + src/adapter/src/catalog/apply.rs | 163 +- src/adapter/src/catalog/open.rs | 2 + src/adapter/src/catalog/state.rs | 43 + src/adapter/src/catalog/transact.rs | 233 +- src/adapter/src/coord.rs | 4 + src/adapter/src/coord/catalog_implications.rs | 1 + .../parsed_state_updates.rs | 41 + src/adapter/src/coord/ddl.rs | 5 + src/adapter/src/coord/sequencer/inner.rs | 1 + .../src/coord/sequencer/inner/cluster.rs | 78 +- src/adapter/src/optimize/dataflows.rs | 4 +- src/catalog-debug/src/main.rs | 21 +- src/catalog-protos/objects_hashes.json | 12 +- src/catalog-protos/src/lib.rs | 3 +- src/catalog-protos/src/objects.rs | 84 + src/catalog-protos/src/objects_v81.rs | 3015 +++++++++++++++++ src/catalog-protos/src/serialization.rs | 12 + src/catalog/src/builtin.rs | 3 + src/catalog/src/durable.rs | 8 +- src/catalog/src/durable/debug.rs | 13 + src/catalog/src/durable/initialize.rs | 1 + src/catalog/src/durable/objects.rs | 131 + .../src/durable/objects/serialization.rs | 77 +- .../src/durable/objects/state_update.rs | 25 + src/catalog/src/durable/persist.rs | 12 + src/catalog/src/durable/transaction.rs | 238 +- src/catalog/src/durable/upgrade.rs | 12 +- .../durable/upgrade/snapshots/objects_v81.txt | 0 src/catalog/src/durable/upgrade/v80_to_v81.rs | 21 + src/catalog/src/memory/objects.rs | 38 +- src/catalog/tests/debug.rs | 5 + src/catalog/tests/open.rs | 5 + .../tests/snapshots/debug__opened_trace.snap | 5 +- .../snapshots/open__initial_snapshot.snap | 3 +- src/compute-client/src/controller.rs | 16 +- src/compute-client/src/controller/instance.rs | 17 +- src/compute-client/src/logging.rs | 4 + src/compute-client/src/protocol/command.rs | 8 + src/compute-types/src/dyncfgs.rs | 12 + src/compute/src/compute_state.rs | 59 +- src/compute/src/logging.rs | 1 + src/compute/src/logging/compute.rs | 21 +- src/compute/src/logging/differential.rs | 21 +- src/compute/src/logging/initialize.rs | 33 +- src/compute/src/logging/persist.rs | 162 + src/compute/src/logging/reachability.rs | 25 +- src/compute/src/logging/timely.rs | 31 +- src/compute/src/metrics.rs | 4 +- src/compute/src/sink.rs | 2 +- src/compute/src/sink/materialized_view.rs | 2 +- src/controller/src/clusters.rs | 9 + src/repr/src/catalog_item_id.proto | 1 + src/repr/src/catalog_item_id.rs | 16 +- src/repr/src/global_id.rs | 12 +- src/sql-lexer/src/keywords.txt | 1 + src/sql-parser/src/ast/defs/statement.rs | 4 + src/sql-parser/src/parser.rs | 5 + src/sql-parser/tests/testdata/ddl | 4 +- src/sql/src/ast/transform.rs | 109 + src/sql/src/plan.rs | 3 + src/sql/src/plan/statement/ddl.rs | 20 + test/persisted-introspection/mzcompose | 14 + test/persisted-introspection/mzcompose.py | 556 +++ test/sqllogictest/cluster.slt | 2 +- test/sqllogictest/persisted_introspection.slt | 31 + 70 files changed, 5881 insertions(+), 80 deletions(-) create mode 100644 PLAN.md create mode 100644 doc/developer/design/20250225_persist_backed_introspection.md create mode 100644 src/catalog-protos/src/objects_v81.rs create mode 100644 src/catalog/src/durable/upgrade/snapshots/objects_v81.txt create mode 100644 src/catalog/src/durable/upgrade/v80_to_v81.rs create mode 100644 src/compute/src/logging/persist.rs create mode 100755 test/persisted-introspection/mzcompose create mode 100644 test/persisted-introspection/mzcompose.py create mode 100644 test/sqllogictest/persisted_introspection.slt diff --git a/PLAN.md b/PLAN.md new file mode 100644 index 0000000000000..534c6c2a47d3b --- /dev/null +++ b/PLAN.md @@ -0,0 +1,55 @@ +# Plan: 0dt read-only support for logging persist sinks + +## Problem + +Logging persist sinks are not tracked as compute collections by the controller, so they cannot use the per-collection `AllowWrites` mechanism. +During 0dt deploys, a read-only replica should not write to persist shards until promoted. + +An earlier attempt piggybacked on the first `AllowWrites` for any collection, but this broke clusters with no MVs (no `AllowWrites` was ever sent). + +## Approach: add `read_only` to `InstanceConfig` + +The compute controller already knows its `read_only` state (instance-level `read_only: bool`). +`CreateInstance(InstanceConfig)` is sent to all replicas and tracked in the command history. +Adding a `read_only: bool` field to `InstanceConfig` lets the controller communicate its read-only state to all replicas at instance creation time. + +On the replica, `handle_create_instance` sets a `watch::Sender` on `ComputeState` before initializing logging, so logging persist sinks pick up the correct value. +The signal also flips to writable on the first `AllowWrites` command for any collection, providing a fallback for 0dt promotion. + +This handles all cases: +* Read-only controller (0dt catchup): sends `read_only: true`, sinks stay read-only. +* Writable controller: sends `read_only: false`, sinks start writable. +* Writable controller with no MVs: still sends `read_only: false`, sinks start writable. +* Command history replay: the `CreateInstance` includes the `read_only` field, so reconnecting replicas get the right state. +* 0dt promotion: `AllowWrites` for any collection flips the signal from `true` to `false`. + +## Implementation (done) + +### 1. Add `read_only` field to `InstanceConfig` + +**File:** `src/compute-client/src/protocol/command.rs` + +* Added `pub read_only: bool` to `InstanceConfig`. +* Updated `compatible_with()` destructures to include `read_only: _` (transitions handled via `AllowWrites`, no compatibility check needed). + +### 2. Set `read_only` from the compute controller + +**File:** `src/compute-client/src/controller/instance.rs` + +* In `Instance::run()`, set `read_only: self.read_only` when constructing `InstanceConfig`. + +### 3. Add replica-level read-only signal to `ComputeState` + +**File:** `src/compute/src/compute_state.rs` + +* Added `read_only_tx: watch::Sender` and `pub read_only_rx: watch::Receiver` fields. +* Initialized as `watch::channel(true)` (read-only by default, safe default for 0dt). +* In `handle_create_instance`, call `send_replace(config.read_only)` before `initialize_logging`. +* In `handle_allow_writes`, flip `read_only_tx` to `false` via `send_if_modified`. + +### 4. Use the replica-level signal in logging persist sinks + +**File:** `src/compute/src/logging/persist.rs` + +* Replaced `let (_tx, read_only_rx) = watch::channel(false)` with `compute_state.read_only_rx.clone()`. +* Removed `use tokio::sync::watch` import. diff --git a/ci/test/pipeline.template.yml b/ci/test/pipeline.template.yml index 9b2944883e43c..01c74c58c6dc7 100644 --- a/ci/test/pipeline.template.yml +++ b/ci/test/pipeline.template.yml @@ -669,6 +669,17 @@ steps: agents: queue: hetzner-aarch64-4cpu-8gb + - id: persisted-introspection + label: "Persisted Introspection" + depends_on: build-aarch64 + timeout_in_minutes: 30 + inputs: [test/persisted-introspection] + plugins: + - ./ci/plugins/mzcompose: + composition: persisted-introspection + agents: + queue: hetzner-aarch64-4cpu-8gb + - id: secrets-logging label: "Secrets Logging" depends_on: build-aarch64 diff --git a/doc/developer/design/20250225_persist_backed_introspection.md b/doc/developer/design/20250225_persist_backed_introspection.md new file mode 100644 index 0000000000000..73ba85a321eaf --- /dev/null +++ b/doc/developer/design/20250225_persist_backed_introspection.md @@ -0,0 +1,369 @@ +# Persist-backed compute introspection + +* Associated: `persist_introspection` branch + +## The problem + +Compute introspection data (dataflow operators, arrangement sizes, scheduling histograms, etc.) is ephemeral. +When a replica restarts, all introspection data is lost and must be rebuilt from scratch. +Introspection indexes are only accessible through subscribes scoped to the active cluster, which requires the replica to serve the subscribe. +When a replica is struggling (out of CPU, thrashing), it cannot serve subscribes, making introspection data inaccessible precisely when it is most needed. + +Additionally, introspection data is only accessible through subscribes scoped to the active cluster. +There is no way to query historical introspection data from a different cluster, join it with catalog metadata, or build materialized views over it. + +## Success criteria + +* Introspection data survives replica restarts through persist-backed storage. +* Stale data from a previous replica incarnation is retracted via self-correction. +* Introspection sources appear as queryable catalog items scoped to a specific replica. +* A global kill switch disables persist writes at runtime without requiring catalog changes. +* The feature is opt-in per cluster via `PERSIST INTROSPECTION = true`. + +## Out of scope + +* **Cross-replica aggregation.** + Combining introspection data from multiple replicas into a single view. +* **Retention policies.** + Automatic compaction or time-based retention for persisted introspection data. + +## Solution proposal + +### Overview + +Each compute replica with persist introspection enabled gets: + +1. A **per-replica schema** in the `materialize` database that holds the replica's introspection sources and views. +2. A set of **persisted introspection sources** (one per log variant, 32 total) stored as `CatalogItem::Source` items with `DataSourceDesc::PersistedIntrospection`. +3. **Per-replica views** that mirror the existing `mz_introspection` views, generated via SQL-to-SQL rewriting. +4. **MV persist sinks** on the compute side that write introspection data to persist shards with self-correction enabled. + +The catalog stores persisted introspection sources in a **dedicated durable collection** (separate from the general `items` collection), following the same pattern as `IntrospectionSourceIndex`. + +### Architecture + +``` +┌─────────────────────────────────────────────────────────────────┐ +│ Catalog │ +│ │ +│ ┌──────────────────────────────┐ │ +│ │ Durable collection: │ │ +│ │ persisted_introspection_ │ (cluster_id, replica_id, │ +│ │ sources │ name) → (item_id, │ +│ │ │ global_id, oid) │ +│ └──────────────────────────────┘ │ +│ │ +│ Per-replica schema: materialize.mz_introspection_{id}_{id} │ +│ ├── mz_dataflow_operators_per_worker (Source, persisted) │ +│ ├── mz_scheduling_elapsed_raw (Source, persisted) │ +│ ├── mz_compute_error_counts_raw (Source, persisted) │ +│ ├── ... (32 log variants total) │ +│ ├── mz_dataflow_operators (View, rewritten) │ +│ ├── mz_arrangement_sizes (View, rewritten) │ +│ └── ... (views mirroring mz_introspection) │ +│ │ +│ System schema: mz_introspection │ +│ ├── mz_dataflow_operators_{cluster_id}_primary_idx (Index) │ +│ └── ... (existing IntrospectionSourceIndex items) │ +│ │ +└─────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ Compute replica │ +│ │ +│ For each log variant with a registered persist shard: │ +│ introspection stream ──► MV persist sink (self-correcting) │ +│ │ │ +│ ▼ │ +│ Persist shard │ +└─────────────────────────────────────────────────────────────────┘ +``` + +### Catalog representation + +#### Dedicated durable collection + +Persisted introspection sources use their own durable collection rather than the general `items` collection. +This avoids the sentinel `create_sql` values (`""`) that previously required guards in six places: three migration functions, the consistency checker, the topological sort, and `item_type()`. + +The collection follows the `IntrospectionSourceIndex` pattern exactly: + +* **Key**: `(cluster_id, replica_id, name)` — uniquely identifies a source. +* **Value**: `(catalog_item_id, global_id, oid, schema_id)` — the catalog metadata including the per-replica schema. +* **StateUpdateKind variant**: `PersistedIntrospectionSource` at the proto, durable, and memory levels. +* **Apply logic**: `apply_persisted_introspection_source_update()` constructs `CatalogItem::Source` with `DataSourceDesc::PersistedIntrospection(log_variant)` and a generated `CREATE MATERIALIZED VIEW` SQL string (see below). + +#### ID allocation + +Both `CatalogItemId` and `GlobalId` are deterministically encoded in a single `u64`: + +``` +Bits 56-63: replica variant (3 = System, 4 = User) +Bits 8-55: replica inner ID +Bits 0-7: log variant (0-31) +``` + +This encoding avoids collisions with `IntrospectionSourceIndex` IDs, which use cluster variant values 1 (Managed) or 2 (Unmanaged) in the same bit positions. +The encoding is deterministic: given a replica ID and log variant, both the `CatalogItemId` and `GlobalId` can be reconstructed without consulting an allocator. + +#### Per-replica schemas + +Each replica gets a schema created in the `materialize` database, named `mz_introspection_{cluster_id}_{replica_id}`. +The `SchemaId` is stored directly on the `PersistedIntrospectionSource` durable object, eliminating any name-based lookup during catalog apply. + +Schemas and their contents get `rbac::default_builtin_object_privilege` grants (USAGE for schemas, SELECT for sources and views) so that any user with the PUBLIC role can query them. + +#### `create_sql` generation + +Persisted introspection sources use `create_sql` containing a `CREATE MATERIALIZED VIEW` statement: + +```sql +CREATE MATERIALIZED VIEW mz_introspection_u2_u2.mz_arrangement_batches_raw + AS SELECT worker_id, operator_id, batches + FROM mz_introspection.mz_arrangement_batches_raw +``` + +The column list is derived from the log variant's `RelationDesc`. +The body references the corresponding `mz_introspection` source by name. +This satisfies the catalog consistency checker, which parses `create_sql` for all items. + +#### Catalog implications + +When `PersistedIntrospectionSource` state updates flow through `parse_state_update`, they are converted into `ParsedStateUpdateKind::Item` by synthesizing a durable `Item`. +This reuses the existing catalog implications path, generating a `CatalogImplication::Source(Added(...))` that triggers `handle_create_source`. +The storage controller creates a collection with `DataSource::Other`, which is required for reads via `PeekTarget::Persist` — the adapter needs the in-memory `CollectionMetadata` (containing the shard ID) and read holds to serve peek queries. + +### Per-replica views via SQL-to-SQL rewriting + +The existing `mz_introspection` schema contains views (e.g., `mz_arrangement_sizes`, `mz_dataflow_operators`) that aggregate raw introspection data into user-friendly formats. +These views reference raw `BuiltinLog` sources via fully-qualified names like `mz_introspection.mz_arrangement_batches_raw`. + +Per-replica schemas need equivalent views that reference the replica's persisted sources instead. +Rather than duplicating view definitions, we generate them by rewriting the existing `BuiltinView` SQL. + +#### Rewrite approach + +Each `BuiltinView` in `MZ_INTROSPECTION_SCHEMA` has a `sql` field containing a `SELECT` statement with `mz_introspection.`-qualified source references. +The rewrite produces a `CREATE VIEW` in the per-replica schema by: + +1. Taking the `BuiltinView.sql` and constructing `CREATE VIEW {per_replica_schema}.{name} AS {sql}`. +2. Parsing the statement with the existing SQL parser. +3. Walking the AST with a `VisitMut` implementation that replaces `mz_introspection` schema qualifiers in `UnresolvedItemName` nodes with the per-replica schema name. +4. Unparsing back to SQL via `to_ast_string_stable()`. + +This rewrite is selective: only `mz_introspection.*` references are replaced. +References to other schemas (`mz_catalog.*`, `mz_internal.*`) remain unchanged, since those system tables are shared and schema-independent. + +#### Why this works + +* The `BuiltinView` definitions are the single source of truth. + When upstream view SQL changes, per-replica views automatically pick up the new definitions on next catalog open. +* The parse/transform/unparse pipeline is the same pattern used by catalog migrations (`ast_rewrite_*` functions in `migrate.rs`). +* All SQL constructs (CTEs, subqueries, joins) are handled uniformly by the AST visitor. +* The per-replica sources have the same names and schemas as the `BuiltinLog` sources they mirror, so the rewritten SQL is valid. + +#### Lifecycle + +Views are created and dropped alongside sources in the same transaction: + +* **Creation:** After inserting persisted introspection sources, iterate over `BuiltinView` definitions in `MZ_INTROSPECTION_SCHEMA`, rewrite each, and insert into the catalog via `tx.insert_item()`. +* **Drop:** When dropping a replica's per-replica schema, all views in that schema are dropped along with the sources. + +#### Which views to include + +Only `BuiltinView` definitions where `schema == MZ_INTROSPECTION_SCHEMA` and whose SQL references only `mz_introspection.*` sources are candidates. +Views that reference system tables outside `mz_introspection` (e.g., joins with `mz_catalog.mz_clusters`) work without modification since those references are left untouched by the rewrite. + +### Compute-side persistence + +Each log variant's introspection stream is connected to an MV persist sink: + +* **Self-correction enabled:** On replica restart, the sink diffs the current introspection state against persisted state and emits corrections. +* **Starts from `Timestamp::MIN`:** The as_of is the minimum timestamp, so the sink captures the full history. +* **No controller frontier tracking:** The compute controller skips frontier updates for `GlobalId::PersistedIntrospectionSource` IDs because these sinks are entirely managed by the replica. +* **Replica-level read-only signal:** Logging persist sinks are not tracked as compute collections and cannot receive per-collection `AllowWrites` commands. + Instead, they use a replica-level `read_only_rx` signal on `ComputeState`, initialized from `InstanceConfig.read_only` (which propagates the controller's instance-level `read_only` flag). + The signal flips to writable on the first `AllowWrites` command for any collection, providing a fallback for 0dt promotion. + This ensures correct behavior in all cases: non-0dt clusters start writable, 0dt clusters start read-only, and new replicas joining an already-promoted cluster receive the correct state via command history replay. + +The "Persist flatten" operator (converting arranged key-value pairs back to flat rows) is implemented once in `persist::render_arranged` and shared by all four logging dataflow files (`compute.rs`, `differential.rs`, `timely.rs`, `reachability.rs`). + +The `ENABLE_PERSIST_INTROSPECTION` dyncfg (`enable_persist_introspection`, default `true`) acts as a global kill switch. +When disabled, compute replicas skip writing introspection data to persist even if the cluster has `PERSIST INTROSPECTION = true`. + +### Lifecycle + +**Cluster creation** (`PERSIST INTROSPECTION = true`): + +1. For each log variant, allocate a deterministic persisted introspection source ID. +2. Register persist shards for each source in storage via `storage_collections_to_register`. +3. Create a per-replica schema in the `materialize` database with default privileges. +4. Insert `PersistedIntrospectionSource` durable objects (with `schema_id`) into the catalog. +5. Create per-replica views via SQL-to-SQL rewriting of `mz_introspection` builtin views. +6. On catalog apply, `parse_state_update` converts the source updates into `ParsedStateUpdateKind::Item`, triggering storage collection creation. + +**Replica drop:** + +1. Look up the per-replica schema by ID. +2. Drop all views and persisted introspection source items. +3. Mark persist shards for cleanup. +4. Drop the per-replica schema. + +**ALTER CLUSTER SET (PERSIST INTROSPECTION = true/false):** +Toggling the option on an existing cluster creates or drops sources for all existing replicas. + +### Files modified + +| Area | Files | +|------|-------| +| Proto types | `src/catalog-protos/src/objects.rs` | +| Durable objects | `src/catalog/src/durable/objects.rs`, `serialization.rs` | +| Collection infra | `src/catalog/src/durable/debug.rs` | +| State updates | `src/catalog/src/durable/objects/state_update.rs`, `persist.rs` | +| Transactions | `src/catalog/src/durable/transaction.rs` | +| Memory objects | `src/catalog/src/memory/objects.rs` | +| Apply logic | `src/adapter/src/catalog/apply.rs` | +| Bootstrap | `src/adapter/src/catalog/open.rs` | +| Transact | `src/adapter/src/catalog/transact.rs` | +| Catalog implications | `src/adapter/src/coord/catalog_implications/parsed_state_updates.rs` | +| Migrations/consistency | `src/adapter/src/catalog/migrate.rs`, `consistency.rs` | +| Debug tooling | `src/catalog-debug/src/main.rs` | +| Re-exports | `src/catalog/src/durable.rs` | +| Compute persistence | `src/compute/src/logging/persist.rs`, `compute.rs`, `differential.rs`, `timely.rs`, `reachability.rs` | +| Compute state | `src/compute/src/compute_state.rs` | +| Test infra | `src/catalog/tests/debug.rs`, `open.rs` | +| Lint config | `misc/python/materialize/parallel_workload/action.py`, `mzcompose/__init__.py` | + +## Minimal viable prototype + +The prototype is implemented on the `persist_introspection` branch and validated by the `test/persisted-introspection/mzcompose.py` test suite, which covers: + +* SQL syntax for `CREATE/ALTER CLUSTER ... PERSIST INTROSPECTION` +* Dyncfg kill switch behavior +* Cluster lifecycle (create, use, drop, recreate) +* Replica restart with self-correction (stale data retraction) +* Catalog changes (add/remove clusters) +* Multiple coexisting clusters with persist introspection +* Environmentd restart with data consistency + +## Alternatives + +### Store persisted introspection sources in the `items` collection + +The initial implementation stored persisted introspection sources in the general `items` collection with a sentinel `create_sql` value of `""`. +This required guards in six places to skip or special-case items with invalid SQL: three migration functions (`ast_rewrite_create_sql_*`), the consistency checker, the topological sort (empty dependencies), and `item_type()`. + +Rejected because every new migration or consistency check must remember to add a guard, and the sentinel SQL values break any code that parses `create_sql`. + +### Reuse `IntrospectionSourceIndex` collection + +Rather than creating a new durable collection, persisted introspection sources could share the `ClusterIntrospectionSourceIndex` collection. +The key difference is scope: introspection source indexes are per-cluster, while persisted introspection sources are per-replica. +The key type would need a `replica_id` field, changing the existing collection's semantics. + +Rejected because it conflates two distinct concepts and would require migrating existing introspection source index data. + +### Generate valid SQL for `create_sql` + +Instead of a dedicated collection, generate a valid `CREATE SOURCE` statement (similar to `index_sql()` for indexes) and continue using the `items` collection. +The generated SQL would have no item dependencies (e.g., `SELECT 1`), making topological sort work naturally. + +Rejected because items in the `items` collection go through migration functions that rewrite SQL ASTs. +Persisted introspection sources don't need SQL migration, and the fake SQL would need to be maintained across AST changes. +The dedicated collection approach is cleaner. + +### Use `create_sql: None` + +The current implementation uses `create_sql: None` on the `CatalogItem::Source`, which is also used by other builtin sources. +This works for the in-memory representation but doesn't address the durable storage question — the items would still need to be stored somewhere with sentinel markers unless a separate collection is used. + +### Use regular materialized views instead of custom persist sinks + +Instead of rendering persist sinks inside the logging dataflows (via `logging/persist.rs`), create actual materialized views over the introspection sources. +The MV definitions would be identical to the current `create_sql` (e.g., `CREATE MATERIALIZED VIEW ... AS SELECT * FROM mz_introspection.mz_arrangement_batches_raw`), but they would be real catalog items rendered through the standard dataflow pipeline rather than custom operator graphs wired into logging initialization. + +#### How it would work + +The per-replica schema would contain materialized views instead of sources + views: + +``` +materialize.mz_introspection_u2_u2 + ├── mz_dataflow_operators_per_worker (Materialized View) + ├── mz_scheduling_elapsed_raw (Materialized View) + ├── ... (32 raw log variant MVs, each SELECT * FROM mz_introspection.) + ├── mz_dataflow_operators (Materialized View over the raw MVs) + └── mz_arrangement_sizes (Materialized View over the raw MVs) +``` + +Each raw MV selects from the existing `mz_introspection` subscribe source (the `IntrospectionSourceIndex` that arranges logging data in memory). +The standard MV rendering pipeline handles persist sink creation, self-correction, frontier tracking, and read-only semantics. + +Creation happens during `Op::CreateClusterReplica` by generating `CREATE MATERIALIZED VIEW` catalog ops. +Dropping happens automatically through the dependency system (already implemented via `cluster_replica_dependents` → `schema_dependents`). + +#### Advantages + +* **No compute changes.** + Removes `logging/persist.rs` entirely. + No custom persist sink rendering, no "Persist flatten" operator, no replica-level `read_only_rx` signal. + Logging dataflows remain unchanged — they only produce arranged in-memory collections as before. +* **Runtime creation and removal.** + MVs can be created and dropped at runtime via catalog ops without restarting the replica. + This enables enabling/disabling persist introspection on existing clusters without recreating replicas. +* **Standard lifecycle management.** + Per-collection frontier tracking, `AllowWrites` commands, controller visibility, and 0dt promotion all work out of the box. + No special-casing needed for logging sinks. +* **No dedicated durable collection.** + MVs are regular items in the `items` collection with valid `create_sql`. + Removes the `PersistedIntrospectionSource` durable collection, the `CatalogItemId::PersistedIntrospectionSource` variant, deterministic ID encoding, dedicated apply logic, and the partition in the item drop path. + +#### Challenges + +* **Cluster binding.** + Regular MVs are bound to a cluster — they need a cluster to render the dataflow on. + The introspection MVs must be rendered on the same cluster and replica whose logging they capture. + A `SELECT * FROM mz_introspection.mz_arrangement_batches_raw` on cluster X captures cluster X's data, so the MV must also run on cluster X. + This is the natural behavior, since MVs are bound to the cluster they're created on. + However, the MV dataflow runs on all replicas of the cluster, not just the target replica. + On non-target replicas, the MV reads the local `mz_introspection` source (which contains that replica's data, not the target's), so it captures the wrong replica's data. + This is a fundamental mismatch: per-replica introspection requires per-replica dataflows, but MVs render on all replicas. +* **`as_of` semantics.** + The current implementation starts persist sinks from `Timestamp::MIN` to capture the full history. + Regular MVs use the `as_of` from the dataflow, which is typically the current timestamp. + Missing the initial data window means losing introspection data from before the MV was created. + This may be acceptable since the MV is created immediately with the replica. +* **Source availability.** + The `mz_introspection` sources (subscribe-backed `IntrospectionSourceIndex` items) must be readable as of the MV's `as_of`. + These sources are arranged in memory and may not be ready at the exact moment the MV dataflow starts. + The standard dataflow rendering already handles this via `StartSignal` / suspension tokens, but it needs to be verified that introspection indexes are treated as available inputs. +* **Arranged-to-flat conversion.** + The current implementation explicitly converts arranged key-value pairs back to flat rows via `render_arranged`. + With MVs, this conversion happens implicitly: the MV reads from the introspection source index (which is arranged), and the peek/subscribe path handles flattening. + This should work transparently since the MV's `SELECT *` produces flat rows. + +#### Assessment + +The cluster binding challenge is the main blocker. +Regular MVs render on all replicas of a cluster, but per-replica introspection requires per-replica dataflows. +An MV created for replica R1 would also render on replica R2, where it captures R2's data instead of R1's — silently writing the wrong data to R1's persist shard. + +Possible mitigations: +* **Replica-pinned MVs:** A new MV variant that renders only on a specific replica. + This does not exist today and would require changes to the compute controller's dataflow scheduling. +* **Per-replica sources:** If the `mz_introspection` sources were per-replica (each replica has its own `GlobalId`), the MV could reference the correct source. + This would require changes to introspection source index management. +* **Accept the limitation:** Only support single-replica clusters for persist introspection. + This sidesteps the multi-replica issue but limits the feature's applicability. + +Given the cluster binding issue, the custom persist sink approach avoids a significant gap in the MV infrastructure. +The custom approach hooks directly into the per-replica logging dataflow, guaranteeing it captures that replica's data. +If replica-pinned dataflows become available in the future, migrating to regular MVs would simplify the implementation. + +## Open questions + +### Querying from `mz_introspection` + +The existing `mz_introspection` schema works within the context of a `SET CLUSTER` session variable. +Persisted introspection sources live in per-replica schemas, which means they are queryable by fully qualifying the schema name. +Whether persisted sources should also be accessible through the standard `mz_introspection` schema (perhaps with a `SET CLUSTER REPLICA` variable) is an open UX question. diff --git a/misc/python/materialize/mzcompose/__init__.py b/misc/python/materialize/mzcompose/__init__.py index 22c322247039d..293ceddfbfb8d 100644 --- a/misc/python/materialize/mzcompose/__init__.py +++ b/misc/python/materialize/mzcompose/__init__.py @@ -610,6 +610,7 @@ def get_default_system_parameters( "kafka_reconnect_backoff_max", "oidc_issuer", "oidc_audience", + "enable_persist_introspection", ] diff --git a/misc/python/materialize/parallel_workload/action.py b/misc/python/materialize/parallel_workload/action.py index f9790a653af3d..b308ad674b19e 100644 --- a/misc/python/materialize/parallel_workload/action.py +++ b/misc/python/materialize/parallel_workload/action.py @@ -1520,6 +1520,7 @@ def __init__( "false", ] self.flags_with_values["enable_cast_elimination"] = BOOLEAN_FLAG_VALUES + self.flags_with_values["enable_persist_introspection"] = BOOLEAN_FLAG_VALUES # If you are adding a new config flag in Materialize, consider using it # here instead of just marking it as uninteresting to silence the diff --git a/src/adapter/src/catalog/apply.rs b/src/adapter/src/catalog/apply.rs index aed664bae1a29..14cd0aab9529e 100644 --- a/src/adapter/src/catalog/apply.rs +++ b/src/adapter/src/catalog/apply.rs @@ -45,15 +45,17 @@ use mz_ore::{assert_none, instrument, soft_assert_no_log}; use mz_pgrepr::oid::INVALID_OID; use mz_repr::adt::mz_acl_item::{MzAclItem, PrivilegeMap}; use mz_repr::role_id::RoleId; -use mz_repr::{CatalogItemId, Diff, GlobalId, RelationVersion, Timestamp, VersionedRelationDesc}; +use mz_repr::{ + CatalogItemId, Diff, GlobalId, RelationDesc, RelationVersion, Timestamp, VersionedRelationDesc, +}; use mz_sql::catalog::CatalogError as SqlCatalogError; use mz_sql::catalog::{CatalogItem as SqlCatalogItem, CatalogItemType, CatalogSchema, CatalogType}; use mz_sql::names::{ FullItemName, ItemQualifiers, QualifiedItemName, RawDatabaseSpecifier, - ResolvedDatabaseSpecifier, ResolvedIds, SchemaSpecifier, + ResolvedDatabaseSpecifier, ResolvedIds, SchemaId, SchemaSpecifier, }; use mz_sql::session::user::MZ_SYSTEM_ROLE_ID; -use mz_sql::session::vars::{VarError, VarInput}; +use mz_sql::session::vars::{DEFAULT_DATABASE_NAME, VarError, VarInput}; use mz_sql::{plan, rbac}; use mz_sql_parser::ast::Expr; use mz_storage_types::sources::Timeline; @@ -87,6 +89,7 @@ struct InProgressRetractions { items: BTreeMap, temp_items: BTreeMap, introspection_source_indexes: BTreeMap, + persisted_introspection_sources: BTreeMap, system_object_mappings: BTreeMap, } @@ -307,6 +310,9 @@ impl CatalogState { retractions, ); } + StateUpdateKind::PersistedIntrospectionSource(source) => { + self.apply_persisted_introspection_source_update(source, diff, retractions); + } StateUpdateKind::ClusterReplica(cluster_replica) => { self.apply_cluster_replica_update(cluster_replica, diff, retractions); } @@ -588,6 +594,49 @@ impl CatalogState { } } + #[instrument(level = "debug")] + fn apply_persisted_introspection_source_update( + &mut self, + source: mz_catalog::durable::PersistedIntrospectionSource, + diff: StateDiff, + retractions: &mut InProgressRetractions, + ) { + match diff { + StateDiff::Addition => { + if let Some(mut entry) = retractions + .persisted_introspection_sources + .remove(&source.item_id) + { + // This should only happen during startup as a result of builtin migrations. + let (name, catalog_item) = self.create_persisted_introspection_source( + &source.name, + source.schema_id, + source.global_id, + ); + assert_eq!(entry.id, source.item_id); + assert_eq!(entry.oid, source.oid); + assert_eq!(entry.name, name); + entry.item = catalog_item; + self.insert_entry(entry); + } else { + self.insert_persisted_introspection_source( + &source.name, + source.schema_id, + source.item_id, + source.global_id, + source.oid, + ); + } + } + StateDiff::Retraction => { + let entry = self.drop_item(source.item_id); + retractions + .persisted_introspection_sources + .insert(entry.id, entry); + } + } + } + #[instrument(level = "debug")] fn apply_cluster_replica_update( &mut self, @@ -1367,6 +1416,9 @@ impl CatalogState { StateUpdateKind::IntrospectionSourceIndex(introspection_source_index) => { self.pack_item_update(introspection_source_index.item_id, diff) } + StateUpdateKind::PersistedIntrospectionSource(source) => { + self.pack_item_update(source.item_id, diff) + } StateUpdateKind::ClusterReplica(cluster_replica) => self.pack_cluster_replica_update( cluster_replica.cluster_id, &cluster_replica.name, @@ -1905,6 +1957,77 @@ impl CatalogState { (index_name, index) } + fn insert_persisted_introspection_source( + &mut self, + log_name: &str, + schema_id: SchemaId, + item_id: CatalogItemId, + global_id: GlobalId, + oid: u32, + ) { + let (name, catalog_item) = + self.create_persisted_introspection_source(log_name, schema_id, global_id); + self.insert_item( + item_id, + oid, + name, + catalog_item, + MZ_SYSTEM_ROLE_ID, + PrivilegeMap::from_mz_acl_items(vec![ + rbac::default_builtin_object_privilege(mz_sql::catalog::ObjectType::Source), + rbac::owner_privilege(mz_sql::catalog::ObjectType::Source, MZ_SYSTEM_ROLE_ID), + ]), + ); + } + + fn create_persisted_introspection_source( + &self, + log_name: &str, + schema_id: SchemaId, + global_id: GlobalId, + ) -> (QualifiedItemName, CatalogItem) { + let log = BUILTIN_LOG_LOOKUP + .get(log_name) + .expect("missing builtin log"); + let log_variant = log.variant; + + let database_id = *self + .database_by_name + .get(DEFAULT_DATABASE_NAME) + .expect("materialize database must exist"); + let database = &self.database_by_id[&database_id]; + let schema = &database.schemas_by_id[&schema_id]; + + let name = QualifiedItemName { + qualifiers: ItemQualifiers { + database_spec: ResolvedDatabaseSpecifier::Id(database_id), + schema_spec: SchemaSpecifier::Id(schema_id), + }, + item: log_name.to_string(), + }; + + let full_name = FullItemName { + database: RawDatabaseSpecifier::Name(database.name.clone()), + schema: schema.name.schema.clone(), + item: log_name.to_string(), + }; + let create_sql = + persisted_introspection_source_sql(full_name, log_name, &log_variant.desc()); + + let catalog_item = CatalogItem::Source(Source { + create_sql: Some(create_sql), + global_id, + data_source: DataSourceDesc::PersistedIntrospection(log_variant), + desc: log_variant.desc(), + timeline: Timeline::EpochMilliseconds, + resolved_ids: ResolvedIds::empty(), + custom_logical_compaction_window: None, + is_retained_metrics_object: false, + }); + + (name, catalog_item) + } + /// Insert system configuration `name` with `value`. /// /// Return a `bool` value indicating whether the configuration was modified @@ -1922,6 +2045,31 @@ impl CatalogState { } } +/// Generate a valid `CREATE MATERIALIZED VIEW` SQL string for a persisted +/// introspection source. The SQL is used for the `create_sql` field so +/// the catalog consistency checker can verify the item name. +fn persisted_introspection_source_sql( + name: FullItemName, + log_name: &str, + desc: &RelationDesc, +) -> String { + use mz_sql_parser::ast::Ident; + use mz_sql_parser::ast::display::AstDisplay; + + let unresolved = mz_sql::normalize::unresolve(name); + let columns = desc + .iter_names() + .map(|col| Ident::new_unchecked(col.as_str()).to_ast_string_stable()) + .collect::>() + .join(", "); + format!( + "CREATE MATERIALIZED VIEW {} AS SELECT {} FROM mz_introspection.{}", + unresolved.to_ast_string_stable(), + columns, + Ident::new_unchecked(log_name).to_ast_string_stable(), + ) +} + /// Sort [`StateUpdate`]s in dependency order. /// /// # Panics @@ -1984,6 +2132,7 @@ fn sort_updates(updates: Vec) -> Vec { ), StateUpdateKind::Cluster(_) | StateUpdateKind::IntrospectionSourceIndex(_) + | StateUpdateKind::PersistedIntrospectionSource(_) | StateUpdateKind::ClusterReplica(_) => push_update( update, diff, @@ -2386,9 +2535,11 @@ impl ApplyState { Self::BuiltinViewAdditions(vec![view_addition]) } - IntrospectionSourceIndex(_) | SystemObjectMapping(_) | TemporaryItem(_) | Item(_) => { - Self::Items(vec![update]) - } + IntrospectionSourceIndex(_) + | PersistedIntrospectionSource(_) + | SystemObjectMapping(_) + | TemporaryItem(_) + | Item(_) => Self::Items(vec![update]), Role(_) | RoleAuth(_) diff --git a/src/adapter/src/catalog/open.rs b/src/adapter/src/catalog/open.rs index 5dab659a1ec25..726f915349096 100644 --- a/src/adapter/src/catalog/open.rs +++ b/src/adapter/src/catalog/open.rs @@ -284,6 +284,7 @@ impl Catalog { }) } BootstrapStateUpdateKind::IntrospectionSourceIndex(_) + | BootstrapStateUpdateKind::PersistedIntrospectionSource(_) | BootstrapStateUpdateKind::SystemObjectMapping(_) => { system_item_updates.push(StateUpdate { kind: kind.into(), @@ -981,6 +982,7 @@ fn add_new_remove_old_builtin_clusters_migration( logging: default_logging_config(), optimizer_feature_overrides: Default::default(), schedule: Default::default(), + persist_introspection: false, }), workload_class: None, }, diff --git a/src/adapter/src/catalog/state.rs b/src/adapter/src/catalog/state.rs index 086505e2ce406..b8f327f35ce0e 100644 --- a/src/adapter/src/catalog/state.rs +++ b/src/adapter/src/catalog/state.rs @@ -538,10 +538,34 @@ impl CatalogState { replica_id: ReplicaId, seen: &mut BTreeSet, ) -> Vec { + use mz_catalog::memory::objects::ClusterVariant; + let mut dependents = Vec::new(); let object_id = ObjectId::ClusterReplica((cluster_id, replica_id)); if !seen.contains(&object_id) { seen.insert(object_id.clone()); + + // If persist introspection is enabled, include the per-replica + // schema and its items as dependents so the generic drop machinery + // handles them. + let cluster = self.get_cluster(cluster_id); + let persist_introspection = match &cluster.config.variant { + ClusterVariant::Managed(managed) => managed.persist_introspection, + ClusterVariant::Unmanaged => false, + }; + if persist_introspection { + let (_schema_name, database_id, schema_id) = + self.get_persisted_introspection_schema(cluster_id, replica_id); + if let Some(schema_id) = schema_id { + dependents.extend_from_slice(&self.schema_dependents( + ResolvedDatabaseSpecifier::Id(database_id), + SchemaSpecifier::Id(schema_id), + &SYSTEM_CONN_ID, + seen, + )); + } + } + dependents.push(object_id); } dependents @@ -1660,6 +1684,24 @@ impl CatalogState { .expect("schema must exist") } + /// Look up the per-replica introspection schema for the given cluster and + /// replica. Returns the schema name, database ID, and optionally the + /// schema ID if the schema already exists. + pub(super) fn get_persisted_introspection_schema( + &self, + cluster_id: ClusterId, + replica_id: ReplicaId, + ) -> (String, DatabaseId, Option) { + let schema_name = format!("mz_introspection_{}_{}", cluster_id, replica_id); + let database_id = *self + .database_by_name + .get(DEFAULT_DATABASE_NAME) + .expect("materialize database must exist"); + let database = &self.database_by_id[&database_id]; + let schema_id = database.schemas_by_name.get(&schema_name).copied(); + (schema_name, database_id, schema_id) + } + pub(super) fn find_non_temp_schema(&self, schema_id: &SchemaId) -> &Schema { self.database_by_id .values() @@ -2604,6 +2646,7 @@ impl CatalogState { cws.entry(source_cw).or_default().insert(item_id); } DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) | DataSourceDesc::Progress | DataSourceDesc::Webhook { .. } => { cws.entry(source_cw).or_default().insert(item_id); diff --git a/src/adapter/src/catalog/transact.rs b/src/adapter/src/catalog/transact.rs index 111454e0b637c..c74243661ac6f 100644 --- a/src/adapter/src/catalog/transact.rs +++ b/src/adapter/src/catalog/transact.rs @@ -26,7 +26,7 @@ use mz_audit_log::{ ObjectType, SchedulingDecisionsWithReasonsV2, VersionedEvent, VersionedStorageUsage, }; use mz_catalog::SYSTEM_CONN_ID; -use mz_catalog::builtin::BuiltinLog; +use mz_catalog::builtin::{BUILTINS, BuiltinLog, BuiltinView}; use mz_catalog::durable::{NetworkPolicy, Transaction}; use mz_catalog::expr_cache::LocalExpressions; use mz_catalog::memory::error::{AmbiguousRename, Error, ErrorKind}; @@ -121,6 +121,9 @@ pub enum Op { database_id: ResolvedDatabaseSpecifier, schema_name: String, owner_id: RoleId, + /// If true, skip the reserved name check. Used for system-created + /// schemas (e.g., per-replica introspection schemas). + system: bool, }, CreateRole { name: String, @@ -1051,8 +1054,9 @@ impl Catalog { database_id, schema_name, owner_id, + system, } => { - if is_reserved_name(&schema_name) { + if !system && is_reserved_name(&schema_name) { return Err(AdapterError::Catalog(Error::new( ErrorKind::ReservedSchemaName(schema_name), ))); @@ -1216,6 +1220,89 @@ impl Catalog { let cluster = state.get_cluster(cluster_id); let id = tx.insert_cluster_replica(cluster_id, &name, config.clone().into(), owner_id)?; + + // If persist introspection is enabled for this cluster, + // register shard IDs, create a per-replica schema, and + // create source catalog items for each log variant. + { + use mz_catalog::builtin::BUILTIN_LOG_BY_VARIANT; + use mz_catalog::memory::objects::ClusterVariant; + use mz_compute_types::dyncfgs::ENABLE_PERSIST_INTROSPECTION; + + let persist_introspection = match &cluster.config.variant { + ClusterVariant::Managed(managed) => managed.persist_introspection, + ClusterVariant::Unmanaged => false, + }; + if persist_introspection + && ENABLE_PERSIST_INTROSPECTION.get(state.system_configuration.dyncfgs()) + { + // Register shard IDs so they survive restarts. + for variant in cluster.log_indexes.keys() { + let (_, global_id) = + Transaction::allocate_persisted_introspection_source_id( + &id, *variant, + ); + storage_collections_to_register.insert(global_id, ShardId::new()); + } + + // Create a per-replica schema in the materialize database. + let (schema_name, database_id, _) = + state.get_persisted_introspection_schema(cluster_id, id); + let temporary_oids = state.get_temporary_oids().collect(); + let (schema_id, _schema_oid) = tx.insert_user_schema( + database_id, + &schema_name, + MZ_SYSTEM_ROLE_ID, + vec![ + rbac::default_builtin_object_privilege( + mz_sql::catalog::ObjectType::Schema, + ), + rbac::owner_privilege( + mz_sql::catalog::ObjectType::Schema, + MZ_SYSTEM_ROLE_ID, + ), + ], + &temporary_oids, + )?; + + // Create a source catalog item for each log variant. + let sources: Vec<_> = cluster + .log_indexes + .keys() + .map(|variant| { + let builtin_log = BUILTIN_LOG_BY_VARIANT + .get(variant) + .expect("every log variant has a builtin log"); + let (item_id, global_id) = + Transaction::allocate_persisted_introspection_source_id( + &id, *variant, + ); + ( + cluster_id, + id, + builtin_log.name.to_string(), + item_id, + global_id, + schema_id, + ) + }) + .collect(); + tx.insert_persisted_introspection_sources(sources, &temporary_oids)?; + + // Create per-replica views by rewriting builtin + // introspection views to reference items in the + // per-replica schema. + create_persisted_introspection_views( + tx, + cluster, + &id, + schema_id, + &schema_name, + &temporary_oids, + )?; + } + } + if let ReplicaLocation::Managed(ManagedReplicaLocation { size, billed_as, @@ -1264,7 +1351,14 @@ impl Catalog { storage_collections_to_create.extend(gids); } CatalogItem::Source(source) => { - storage_collections_to_create.insert(source.global_id()); + // Don't add to storage_collections_to_create if the + // shard was already pre-registered (e.g., persist + // introspection sources registered during + // Op::CreateClusterReplica). + let gid = source.global_id(); + if !storage_collections_to_register.contains_key(&gid) { + storage_collections_to_create.insert(gid); + } } CatalogItem::MaterializedView(mv) => { let mv_gid = mv.global_id_writes(); @@ -1602,7 +1696,15 @@ impl Catalog { .iter() .map(|id| id) .partition(|id| !state.get_entry(*id).item().is_temporary()); - tx.remove_items(&durable_items_to_drop)?; + // Persisted introspection sources live in a dedicated durable + // collection, not the regular items collection. Partition them + // out so they are removed separately via + // `remove_persisted_introspection_sources`. + let (regular_items, _persisted_introspection_items): (BTreeSet<_>, BTreeSet<_>) = + durable_items_to_drop.into_iter().partition(|id| { + !matches!(id, CatalogItemId::PersistedIntrospectionSource(_)) + }); + tx.remove_items(®ular_items)?; temporary_item_updates.extend(temporary_items_to_drop.into_iter().map(|id| { let entry = state.get_entry(&id); (entry.clone().into(), StateDiff::Retraction) @@ -1766,6 +1868,39 @@ impl Catalog { let replicas = delta.replicas.keys().copied().collect(); tx.remove_cluster_replicas(&replicas)?; + // Remove persisted introspection sources from their + // dedicated durable collection. Schemas, views, and storage + // collections are already handled by the dependency system + // via `cluster_replica_dependents` → `schema_dependents`. + { + use mz_catalog::builtin::BUILTIN_LOG_BY_VARIANT; + use mz_catalog::memory::objects::ClusterVariant; + + let mut sources_to_drop = BTreeSet::new(); + for (replica_id, (cluster_id, _reason)) in &delta.replicas { + let cluster = state.get_cluster(*cluster_id); + let persist_introspection = match &cluster.config.variant { + ClusterVariant::Managed(managed) => managed.persist_introspection, + ClusterVariant::Unmanaged => false, + }; + if persist_introspection { + let (_schema_name, _database_id, schema_id) = + state.get_persisted_introspection_schema(*cluster_id, *replica_id); + if schema_id.is_some() { + sources_to_drop.extend(cluster.log_indexes.keys().map(|variant| { + let builtin_log = BUILTIN_LOG_BY_VARIANT + .get(variant) + .expect("every log variant has a builtin log"); + (cluster.id, *replica_id, builtin_log.name.to_string()) + })); + } + } + } + if !sources_to_drop.is_empty() { + tx.remove_persisted_introspection_sources(sources_to_drop)?; + } + } + for (replica_id, (cluster_id, reason)) in delta.replicas { let cluster = state.get_cluster(cluster_id); let replica = cluster.replica(replica_id).expect("Must exist"); @@ -2895,7 +3030,11 @@ impl ObjectsToDrop { } DropObjectInfo::Item(item_id) => { let entry = state.get_entry(&item_id); - if item_id.is_system() { + // Persisted introspection sources are system-managed but + // droppable when their parent replica is dropped. + if item_id.is_system() + && !matches!(item_id, CatalogItemId::PersistedIntrospectionSource(_)) + { let name = entry.name(); let full_name = state.resolve_full_name(name, session.map(|session| session.conn_id())); @@ -2923,6 +3062,90 @@ impl ObjectsToDrop { } } +/// Create per-replica views by rewriting each builtin `mz_introspection` view +/// to reference items in the per-replica schema, using ID-based references so +/// the catalog apply pipeline's topological sort can order them correctly. +fn create_persisted_introspection_views( + tx: &mut Transaction<'_>, + cluster: &mz_catalog::memory::objects::Cluster, + replica_id: &ReplicaId, + schema_id: SchemaId, + schema_name: &str, + temporary_oids: &HashSet, +) -> Result<(), AdapterError> { + use itertools::Itertools; + use mz_catalog::builtin::BUILTIN_LOG_BY_VARIANT; + use mz_ore::cast::CastFrom; + use mz_repr::namespaces::MZ_INTROSPECTION_SCHEMA; + + // Build name → CatalogItemId map for all items in the per-replica schema. + let mut name_to_id: BTreeMap = BTreeMap::new(); + + // Add persisted introspection sources. + for variant in cluster.log_indexes.keys() { + let builtin_log = BUILTIN_LOG_BY_VARIANT + .get(variant) + .expect("every log variant has a builtin log"); + let (item_id, _global_id) = + Transaction::allocate_persisted_introspection_source_id(replica_id, *variant); + name_to_id.insert(builtin_log.name.to_string(), item_id); + } + + // Collect mz_introspection views. + let introspection_views: Vec<&BuiltinView> = BUILTINS::views() + .filter(|v| v.schema == MZ_INTROSPECTION_SCHEMA) + .collect(); + + // Allocate IDs for all views upfront so we can use them in cross-references. + let view_ids = tx.allocate_user_item_ids(u64::cast_from(introspection_views.len()))?; + for ((item_id, _global_id), view) in view_ids.iter().zip_eq(introspection_views.iter()) { + name_to_id.insert(view.name.to_string(), *item_id); + } + + // Create and insert each view. + for ((item_id, global_id), view) in view_ids.into_iter().zip_eq(introspection_views.iter()) { + let create_sql = rewrite_introspection_view_sql(view, schema_name, &name_to_id); + tx.insert_user_item( + item_id, + global_id, + schema_id, + view.name, + create_sql, + MZ_SYSTEM_ROLE_ID, + vec![ + rbac::default_builtin_object_privilege(mz_sql::catalog::ObjectType::View), + rbac::owner_privilege(mz_sql::catalog::ObjectType::View, MZ_SYSTEM_ROLE_ID), + ], + temporary_oids, + BTreeMap::new(), + )?; + } + + Ok(()) +} + +/// Produce CREATE VIEW SQL for a per-replica introspection view by rewriting +/// schema references from `mz_introspection` to `target_schema` and converting +/// item references to ID-based refs. +fn rewrite_introspection_view_sql( + view: &BuiltinView, + target_schema: &str, + name_to_id: &BTreeMap, +) -> String { + use mz_ore::collections::CollectionExt; + use mz_sql::ast::transform::rewrite_introspection_schema_refs; + use mz_sql_parser::ast::display::AstDisplay; + + let create_sql = format!( + "CREATE VIEW {}.{} AS {}", + target_schema, view.name, view.sql + ); + let stmts = mz_sql::parse::parse(&create_sql).expect("builtin view SQL must parse"); + let mut stmt = stmts.into_element().ast; + rewrite_introspection_schema_refs(&mut stmt, target_schema, Some(name_to_id)); + stmt.to_ast_string_stable() +} + #[cfg(test)] mod tests { use mz_repr::adt::mz_acl_item::{AclMode, MzAclItem, PrivilegeMap}; diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index 5e4b6cc2f3441..0d63e98deae42 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -2001,6 +2001,7 @@ impl Coordinator { )?; for replica in instance.replicas() { let role = instance.role(); + let sink_logs = self.build_log_sink_configs(instance.id, &replica.replica_id); self.controller.create_replica( instance.id, replica.replica_id, @@ -2009,6 +2010,7 @@ impl Coordinator { role, replica.config.clone(), enable_worker_core_affinity, + sink_logs, )?; } } @@ -2466,6 +2468,7 @@ impl Coordinator { CatalogItemId::System(id) => *id >= next_system_item_id, CatalogItemId::User(id) => *id >= next_user_item_id, CatalogItemId::IntrospectionSourceIndex(_) + | CatalogItemId::PersistedIntrospectionSource(_) | CatalogItemId::Transient(_) => false, }; if id_too_large { @@ -2813,6 +2816,7 @@ impl Coordinator { DataSourceDesc::Introspection(introspection) => { DataSource::Introspection(introspection) } + DataSourceDesc::PersistedIntrospection(_) => DataSource::Other, }; CollectionDescription { desc: desc.clone(), diff --git a/src/adapter/src/coord/catalog_implications.rs b/src/adapter/src/coord/catalog_implications.rs index 43a2180218f7f..48232c984ee65 100644 --- a/src/adapter/src/coord/catalog_implications.rs +++ b/src/adapter/src/coord/catalog_implications.rs @@ -1282,6 +1282,7 @@ impl Coordinator { DataSourceDesc::Introspection(_) => { unreachable!("cannot create sources with introspection data sources") } + DataSourceDesc::PersistedIntrospection(_) => DataSource::Other, }; storage_collections_to_create.insert( diff --git a/src/adapter/src/coord/catalog_implications/parsed_state_updates.rs b/src/adapter/src/coord/catalog_implications/parsed_state_updates.rs index 97a74c6e7b1bb..af84ded90f87b 100644 --- a/src/adapter/src/coord/catalog_implications/parsed_state_updates.rs +++ b/src/adapter/src/coord/catalog_implications/parsed_state_updates.rs @@ -87,6 +87,9 @@ pub fn parse_state_update( StateUpdateKind::ClusterReplica(replica) => { Some(parse_cluster_replica_update(catalog, replica)) } + StateUpdateKind::PersistedIntrospectionSource(source) => { + Some(parse_persisted_introspection_source_update(catalog, source)) + } _ => { // The controllers are currently not interested in other kinds of // changes to the catalog. @@ -131,6 +134,44 @@ fn parse_temporary_item_update( } } +/// Convert a `PersistedIntrospectionSource` update into a [`ParsedStateUpdateKind::Item`]. +/// +/// Persisted introspection sources are not stored in the `items` durable collection, +/// so we synthesize a durable `Item` to reuse the existing Item implications path. +/// This ensures a storage collection is created for the source, which is required +/// for reads via `PeekTarget::Persist` (to provide `CollectionMetadata` and read holds). +fn parse_persisted_introspection_source_update( + catalog: &CatalogState, + source: durable::objects::PersistedIntrospectionSource, +) -> ParsedStateUpdateKind { + use mz_sql::session::user::MZ_SYSTEM_ROLE_ID; + use std::collections::BTreeMap; + + let (parsed_item, connection, parsed_full_name) = + parse_item_update_common(catalog, &source.item_id); + + // Synthesize a durable Item so we can reuse the existing + // ParsedStateUpdateKind::Item path. + let durable_item = durable::objects::Item { + id: source.item_id, + oid: source.oid, + global_id: source.global_id, + schema_id: source.schema_id, + name: source.name, + create_sql: String::new(), + owner_id: MZ_SYSTEM_ROLE_ID, + privileges: vec![], + extra_versions: BTreeMap::new(), + }; + + ParsedStateUpdateKind::Item { + durable_item, + parsed_item, + connection, + parsed_full_name, + } +} + // Shared between temporary items and durable items. fn parse_item_update_common( catalog: &CatalogState, diff --git a/src/adapter/src/coord/ddl.rs b/src/adapter/src/coord/ddl.rs index 480d123fe826e..053abc9a8ebad 100644 --- a/src/adapter/src/coord/ddl.rs +++ b/src/adapter/src/coord/ddl.rs @@ -578,6 +578,11 @@ impl Coordinator { self.controller .drop_replica(cluster_id, replica_id) .expect("dropping replica must not fail"); + + // Shard finalization for persist-backed introspection is now handled + // by the catalog transaction: `Op::DropObjects` adds the GlobalIds to + // `storage_collections_to_drop`, and `prepare_state` moves the shards + // to `unfinalized_shards` for background finalization. } /// A convenience method for dropping sources. diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index e9ffae1341165..940dc53a3af3f 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -845,6 +845,7 @@ impl Coordinator { database_id: plan.database_spec, schema_name: plan.schema_name.clone(), owner_id: *session.current_role_id(), + system: false, }; match self.catalog_transact(Some(session), vec![op]).await { Ok(_) => Ok(ExecuteResponse::CreatedSchema), diff --git a/src/adapter/src/coord/sequencer/inner/cluster.rs b/src/adapter/src/coord/sequencer/inner/cluster.rs index 60fb7ff3fc50f..7a68b1196fdcf 100644 --- a/src/adapter/src/coord/sequencer/inner/cluster.rs +++ b/src/adapter/src/coord/sequencer/inner/cluster.rs @@ -7,7 +7,7 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use std::collections::BTreeSet; +use std::collections::{BTreeMap, BTreeSet}; use std::time::{Duration, Instant}; use itertools::Itertools; @@ -17,6 +17,7 @@ use mz_catalog::builtin::BUILTINS; use mz_catalog::memory::objects::{ ClusterConfig, ClusterReplica, ClusterVariant, ClusterVariantManaged, }; +use mz_compute_client::logging::LogVariant; use mz_compute_types::config::ComputeReplicaConfig; use mz_controller::clusters::{ ManagedReplicaAvailabilityZones, ManagedReplicaLocation, ReplicaConfig, ReplicaLocation, @@ -25,6 +26,7 @@ use mz_controller::clusters::{ use mz_controller_types::{ClusterId, DEFAULT_REPLICA_LOGGING_INTERVAL, ReplicaId}; use mz_ore::cast::CastFrom; use mz_ore::instrument; +use mz_repr::GlobalId; use mz_repr::role_id::RoleId; use mz_sql::ast::{Ident, QualifiedReplica}; use mz_sql::catalog::{CatalogCluster, CatalogClusterReplica, ObjectType}; @@ -37,6 +39,7 @@ use mz_sql::plan::{ use mz_sql::plan::{AlterClusterPlan, OnTimeoutAction}; use mz_sql::session::metadata::SessionMetadata; use mz_sql::session::vars::{MAX_REPLICAS_PER_CLUSTER, SystemVars, Var}; +use mz_storage_types::controller::CollectionMetadata; use tracing::{Instrument, Span, debug}; use super::return_if_err; @@ -182,6 +185,7 @@ impl Coordinator { replication_factor: 1, optimizer_feature_overrides: Default::default(), schedule: Default::default(), + persist_introspection: false, }); } } @@ -194,6 +198,7 @@ impl Coordinator { replication_factor, optimizer_feature_overrides: _, schedule, + persist_introspection, }) => { match &options.size { Set(s) => size.clone_from(s), @@ -232,6 +237,11 @@ impl Coordinator { Reset => *schedule = Default::default(), Unchanged => {} } + match &options.persist_introspection { + Set(pi) => *persist_introspection = *pi, + Reset => *persist_introspection = false, + Unchanged => {} + } if !matches!(options.replicas, Unchanged) { coord_bail!("Cannot change REPLICAS of managed clusters"); } @@ -625,6 +635,7 @@ impl Coordinator { replication_factor: plan.replication_factor, optimizer_feature_overrides: plan.optimizer_feature_overrides.clone(), schedule: plan.schedule.clone(), + persist_introspection: plan.persist_introspection, }) } CreateClusterVariant::Unmanaged(_) => ClusterVariant::Unmanaged, @@ -664,6 +675,7 @@ impl Coordinator { size, optimizer_feature_overrides: _, schedule: _, + persist_introspection: _, }: CreateClusterManagedPlan, cluster_id: ClusterId, mut ops: Vec, @@ -1073,6 +1085,8 @@ impl Coordinator { let enable_worker_core_affinity = self.catalog().system_config().enable_worker_core_affinity(); + let sink_logs = self.build_log_sink_configs(cluster_id, &replica_id); + self.controller .create_replica( cluster_id, @@ -1082,6 +1096,7 @@ impl Coordinator { role, replica_config, enable_worker_core_affinity, + sink_logs, ) .expect("creating replicas must not fail"); @@ -1089,6 +1104,64 @@ impl Coordinator { .await; } + /// Build the `sink_logs` map for a replica, if persist-backed introspection is enabled + /// for the cluster. + /// + /// Returns an empty map if the cluster does not have `PERSIST INTROSPECTION = true` + /// or the global dyncfg kill switch is disabled. + /// + /// Shard IDs are looked up from `StorageMetadata` where they were registered + /// during the catalog transaction (`Op::CreateClusterReplica`). + pub(crate) fn build_log_sink_configs( + &self, + cluster_id: ClusterId, + replica_id: &ReplicaId, + ) -> BTreeMap { + use mz_catalog::durable::Transaction; + use mz_compute_types::dyncfgs::ENABLE_PERSIST_INTROSPECTION; + + let cluster = self.catalog().get_cluster(cluster_id); + + // Check if persist introspection is enabled for this cluster. + let persist_introspection_enabled = match &cluster.config.variant { + ClusterVariant::Managed(managed) => managed.persist_introspection, + ClusterVariant::Unmanaged => false, + }; + if !persist_introspection_enabled { + return BTreeMap::new(); + } + + // Check the global dyncfg kill switch. + if !ENABLE_PERSIST_INTROSPECTION.get(self.controller.compute.dyncfg()) { + return BTreeMap::new(); + } + + let persist_location = self.controller.compute.persist_location().clone(); + let storage_metadata = self.catalog().state().storage_metadata(); + + // Create a sink entry for each log variant that has an introspection index. + // Shard IDs were registered in `StorageCollectionMetadata` during the + // catalog transaction and survive restarts. + cluster + .log_indexes + .keys() + .map(|variant| { + let (_, global_id) = + Transaction::allocate_persisted_introspection_source_id(replica_id, *variant); + let data_shard = storage_metadata + .get_collection_shard::(global_id) + .expect("shard registered during catalog transaction"); + let metadata = CollectionMetadata { + persist_location: persist_location.clone(), + data_shard, + relation_desc: variant.desc(), + txns_shard: None, + }; + (*variant, (global_id, metadata)) + }) + .collect() + } + /// When this is called by the automated cluster scheduling, `scheduling_decision_reason` should /// contain information on why is a cluster being turned On/Off. It will be forwarded to the /// `details` field of the audit log event that records creating or dropping replicas. @@ -1120,6 +1193,7 @@ impl Coordinator { replication_factor, optimizer_feature_overrides: _, schedule: _, + persist_introspection: _, }) = &cluster.config.variant else { panic!("expected existing managed cluster config"); @@ -1131,6 +1205,7 @@ impl Coordinator { logging: new_logging, optimizer_feature_overrides: _, schedule: _, + persist_introspection: _, }) = &new_config.variant else { panic!("expected new managed cluster config"); @@ -1309,6 +1384,7 @@ impl Coordinator { logging: _, optimizer_feature_overrides: _, schedule: _, + persist_introspection: _, }) = &mut new_config.variant else { panic!("expected new managed cluster config"); diff --git a/src/adapter/src/optimize/dataflows.rs b/src/adapter/src/optimize/dataflows.rs index 1c8d5bb4cc891..fb4f514c17f1d 100644 --- a/src/adapter/src/optimize/dataflows.rs +++ b/src/adapter/src/optimize/dataflows.rs @@ -462,7 +462,9 @@ impl<'a> DataflowBuilder<'a> { .expect("ingestion export must reference a source"); data_config.monotonic(&source_desc.connection) } - DataSourceDesc::Introspection(_) | DataSourceDesc::Progress => false, + DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) + | DataSourceDesc::Progress => false, } } diff --git a/src/catalog-debug/src/main.rs b/src/catalog-debug/src/main.rs index c995228c26886..869df98acb5fe 100644 --- a/src/catalog-debug/src/main.rs +++ b/src/catalog-debug/src/main.rs @@ -34,11 +34,11 @@ use mz_catalog::durable::debug::{ AuditLogCollection, ClusterCollection, ClusterIntrospectionSourceIndexCollection, ClusterReplicaCollection, Collection, CollectionTrace, CollectionType, CommentCollection, ConfigCollection, DatabaseCollection, DebugCatalogState, DefaultPrivilegeCollection, - IdAllocatorCollection, ItemCollection, NetworkPolicyCollection, RoleAuthCollection, - RoleCollection, SchemaCollection, SettingCollection, SourceReferencesCollection, - StorageCollectionMetadataCollection, SystemConfigurationCollection, - SystemItemMappingCollection, SystemPrivilegeCollection, Trace, TxnWalShardCollection, - UnfinalizedShardsCollection, + IdAllocatorCollection, ItemCollection, NetworkPolicyCollection, + PersistedIntrospectionSourceCollection, RoleAuthCollection, RoleCollection, SchemaCollection, + SettingCollection, SourceReferencesCollection, StorageCollectionMetadataCollection, + SystemConfigurationCollection, SystemItemMappingCollection, SystemPrivilegeCollection, Trace, + TxnWalShardCollection, UnfinalizedShardsCollection, }; use mz_catalog::durable::{ BootstrapArgs, OpenableDurableCatalogState, persist_backed_catalog_state, @@ -315,6 +315,9 @@ macro_rules! for_collection { $fn::($($arg),*).await? } CollectionType::TxnWalShard => $fn::($($arg),*).await?, + CollectionType::PersistedIntrospectionSource => { + $fn::($($arg),*).await? + } } }; } @@ -450,6 +453,7 @@ async fn dump( id_allocator, items, network_policies, + persisted_introspection_sources, roles, role_auth, schemas, @@ -551,6 +555,13 @@ async fn dump( consolidate, ); dump_col(&mut data, txn_wal_shard, &ignore, stats_only, consolidate); + dump_col( + &mut data, + persisted_introspection_sources, + &ignore, + stats_only, + consolidate, + ); writeln!(&mut target, "{data:#?}")?; Ok(()) diff --git a/src/catalog-protos/objects_hashes.json b/src/catalog-protos/objects_hashes.json index ee8616fb3c7b5..2926a30b3bd36 100644 --- a/src/catalog-protos/objects_hashes.json +++ b/src/catalog-protos/objects_hashes.json @@ -1,8 +1,4 @@ [ - { - "name": "objects.rs", - "md5": "4a133ca169b83cd4fd762aba9b556dc4" - }, { "name": "objects_v74.rs", "md5": "e7b522bf1c6d86e6045bbe3e9a3d4726" @@ -30,5 +26,13 @@ { "name": "objects_v80.rs", "md5": "4a133ca169b83cd4fd762aba9b556dc4" + }, + { + "name": "objects_v81.rs", + "md5": "12ea97403053fa3a1f21c84a9eedd9b1" + }, + { + "name": "objects.rs", + "md5": "12ea97403053fa3a1f21c84a9eedd9b1" } ] diff --git a/src/catalog-protos/src/lib.rs b/src/catalog-protos/src/lib.rs index d7dd0c4fae9be..34aade71816a1 100644 --- a/src/catalog-protos/src/lib.rs +++ b/src/catalog-protos/src/lib.rs @@ -18,6 +18,7 @@ pub mod objects_v77; pub mod objects_v78; pub mod objects_v79; pub mod objects_v80; +pub mod objects_v81; pub mod serialization; /// The current version of the `Catalog`. @@ -25,7 +26,7 @@ pub mod serialization; /// We will initialize new `Catalog`s with this version, and migrate existing `Catalog`s to this /// version. Whenever the `Catalog` changes, e.g. the types we serialize in the `Catalog` /// change, we need to bump this version. -pub const CATALOG_VERSION: u64 = 80; +pub const CATALOG_VERSION: u64 = 81; /// The minimum `Catalog` version number that we support migrating from. /// diff --git a/src/catalog-protos/src/objects.rs b/src/catalog-protos/src/objects.rs index 9ca4eca585a42..f08d88b7c3552 100644 --- a/src/catalog-protos/src/objects.rs +++ b/src/catalog-protos/src/objects.rs @@ -205,6 +205,42 @@ pub struct ClusterIntrospectionSourceIndexValue { pub oid: u32, } +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceKey { + pub cluster_id: ClusterId, + pub replica_id: ReplicaId, + pub name: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceValue { + pub catalog_id: PersistedIntrospectionSourceCatalogItemId, + pub global_id: PersistedIntrospectionSourceGlobalId, + pub oid: u32, + pub schema_id: SchemaId, +} + #[derive( Clone, Copy, @@ -840,6 +876,7 @@ pub enum CatalogItemId { User(u64), Transient(u64), IntrospectionSourceIndex(u64), + PersistedIntrospectionSource(u64), } #[derive( @@ -870,6 +907,20 @@ pub struct SystemCatalogItemId(pub u64); )] pub struct IntrospectionSourceIndexCatalogItemId(pub u64); +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceCatalogItemId(pub u64); + #[derive( Clone, Copy, @@ -888,6 +939,7 @@ pub enum GlobalId { Transient(u64), Explain, IntrospectionSourceIndex(u64), + PersistedIntrospectionSource(u64), } #[derive( @@ -918,6 +970,20 @@ pub struct SystemGlobalId(pub u64); )] pub struct IntrospectionSourceIndexGlobalId(pub u64); +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceGlobalId(pub u64); + #[derive( Clone, Copy, @@ -1187,6 +1253,7 @@ pub struct ManagedCluster { pub logging: ReplicaLogging, pub optimizer_feature_overrides: Vec, pub schedule: ClusterSchedule, + pub persist_introspection: bool, } #[derive( @@ -2486,6 +2553,7 @@ pub enum StateUpdateKind { IdAlloc(IdAlloc), Item(Item), NetworkPolicy(NetworkPolicy), + PersistedIntrospectionSource(PersistedIntrospectionSource), Role(Role), RoleAuth(RoleAuth), Schema(Schema), @@ -2659,6 +2727,22 @@ pub struct ClusterIntrospectionSourceIndex { pub value: ClusterIntrospectionSourceIndexValue, } +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSource { + pub key: PersistedIntrospectionSourceKey, + pub value: PersistedIntrospectionSourceValue, +} + #[derive( Clone, Debug, diff --git a/src/catalog-protos/src/objects_v81.rs b/src/catalog-protos/src/objects_v81.rs new file mode 100644 index 0000000000000..f08d88b7c3552 --- /dev/null +++ b/src/catalog-protos/src/objects_v81.rs @@ -0,0 +1,3015 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use proptest_derive::Arbitrary; +use serde::{Deserialize, Serialize}; +use serde_repr::{Deserialize_repr, Serialize_repr}; + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ConfigKey { + pub key: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ConfigValue { + pub value: u64, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SettingKey { + pub name: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SettingValue { + pub value: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct IdAllocKey { + pub name: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct IdAllocValue { + pub next_id: u64, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct GidMappingKey { + pub schema_name: String, + pub object_type: CatalogItemType, + pub object_name: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct GidMappingValue { + pub catalog_id: SystemCatalogItemId, + pub global_id: SystemGlobalId, + pub fingerprint: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterKey { + pub id: ClusterId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterValue { + pub name: String, + pub owner_id: RoleId, + pub privileges: Vec, + pub config: ClusterConfig, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterIntrospectionSourceIndexKey { + pub cluster_id: ClusterId, + pub name: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterIntrospectionSourceIndexValue { + pub catalog_id: IntrospectionSourceIndexCatalogItemId, + pub global_id: IntrospectionSourceIndexGlobalId, + pub oid: u32, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceKey { + pub cluster_id: ClusterId, + pub replica_id: ReplicaId, + pub name: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceValue { + pub catalog_id: PersistedIntrospectionSourceCatalogItemId, + pub global_id: PersistedIntrospectionSourceGlobalId, + pub oid: u32, + pub schema_id: SchemaId, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterReplicaKey { + pub id: ReplicaId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterReplicaValue { + pub cluster_id: ClusterId, + pub name: String, + pub config: ReplicaConfig, + pub owner_id: RoleId, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct DatabaseKey { + pub id: DatabaseId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct DatabaseValue { + pub name: String, + pub owner_id: RoleId, + pub privileges: Vec, + pub oid: u32, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SchemaKey { + pub id: SchemaId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SchemaValue { + pub database_id: Option, + pub name: String, + pub owner_id: RoleId, + pub privileges: Vec, + pub oid: u32, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ItemKey { + pub gid: CatalogItemId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ItemValue { + pub schema_id: SchemaId, + pub name: String, + pub definition: CatalogItem, + pub owner_id: RoleId, + pub privileges: Vec, + pub oid: u32, + pub global_id: GlobalId, + pub extra_versions: Vec, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ItemVersion { + pub global_id: GlobalId, + pub version: Version, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleKey { + pub id: RoleId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleValue { + pub name: String, + pub attributes: RoleAttributes, + pub membership: RoleMembership, + pub vars: RoleVars, + pub oid: u32, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleAuthKey { + pub id: RoleId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleAuthValue { + pub password_hash: Option, + pub updated_at: EpochMillis, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct NetworkPolicyKey { + pub id: NetworkPolicyId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct NetworkPolicyValue { + pub name: String, + pub rules: Vec, + pub owner_id: RoleId, + pub privileges: Vec, + pub oid: u32, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ServerConfigurationKey { + pub name: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ServerConfigurationValue { + pub value: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct AuditLogKey { + pub event: AuditLogEvent, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum AuditLogEvent { + V1(AuditLogEventV1), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct CommentKey { + pub object: CommentObject, + pub sub_component: Option, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum CommentObject { + Table(CatalogItemId), + View(CatalogItemId), + MaterializedView(CatalogItemId), + Source(CatalogItemId), + Sink(CatalogItemId), + Index(CatalogItemId), + Func(CatalogItemId), + Connection(CatalogItemId), + Type(CatalogItemId), + Secret(CatalogItemId), + ContinualTask(CatalogItemId), + Role(RoleId), + Database(DatabaseId), + Schema(ResolvedSchema), + Cluster(ClusterId), + ClusterReplica(ClusterReplicaId), + NetworkPolicy(NetworkPolicyId), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum CommentSubComponent { + ColumnPos(u64), +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct CommentValue { + pub comment: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SourceReferencesKey { + pub source: CatalogItemId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SourceReferencesValue { + pub references: Vec, + pub updated_at: EpochMillis, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SourceReference { + pub name: String, + pub namespace: Option, + pub columns: Vec, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct StorageCollectionMetadataKey { + pub id: GlobalId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct StorageCollectionMetadataValue { + pub shard: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct UnfinalizedShardKey { + pub shard: String, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct TxnWalShardValue { + pub shard: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Empty {} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct StringWrapper { + pub inner: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Duration { + pub secs: u64, + pub nanos: u32, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct EpochMillis { + pub millis: u64, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Version { + pub value: u64, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum CatalogItem { + V1(CatalogItemV1), +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct CatalogItemV1 { + pub create_sql: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum CatalogItemId { + System(u64), + User(u64), + Transient(u64), + IntrospectionSourceIndex(u64), + PersistedIntrospectionSource(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SystemCatalogItemId(pub u64); + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct IntrospectionSourceIndexCatalogItemId(pub u64); + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceCatalogItemId(pub u64); + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum GlobalId { + System(u64), + User(u64), + Transient(u64), + Explain, + IntrospectionSourceIndex(u64), + PersistedIntrospectionSource(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SystemGlobalId(pub u64); + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct IntrospectionSourceIndexGlobalId(pub u64); + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSourceGlobalId(pub u64); + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum ClusterId { + System(u64), + User(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum DatabaseId { + System(u64), + User(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum ResolvedDatabaseSpecifier { + Ambient, + Id(DatabaseId), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum SchemaId { + System(u64), + User(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum SchemaSpecifier { + Temporary, + Id(SchemaId), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ResolvedSchema { + pub database: ResolvedDatabaseSpecifier, + pub schema: SchemaSpecifier, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum ReplicaId { + System(u64), + User(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterReplicaId { + pub cluster_id: ClusterId, + pub replica_id: ReplicaId, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum NetworkPolicyId { + System(u64), + User(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ReplicaLogging { + pub log_logging: bool, + pub interval: Option, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct OptimizerFeatureOverride { + pub name: String, + pub value: String, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterScheduleRefreshOptions { + pub rehydration_time_estimate: Duration, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum ClusterSchedule { + Manual, + Refresh(ClusterScheduleRefreshOptions), +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterConfig { + pub workload_class: Option, + pub variant: ClusterVariant, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum ClusterVariant { + Unmanaged, + Managed(ManagedCluster), +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ManagedCluster { + pub size: String, + pub replication_factor: u32, + pub availability_zones: Vec, + pub logging: ReplicaLogging, + pub optimizer_feature_overrides: Vec, + pub schedule: ClusterSchedule, + pub persist_introspection: bool, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ReplicaConfig { + pub logging: ReplicaLogging, + pub location: ReplicaLocation, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct UnmanagedLocation { + pub storagectl_addrs: Vec, + pub computectl_addrs: Vec, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ManagedLocation { + pub size: String, + pub availability_zone: Option, + pub internal: bool, + pub billed_as: Option, + pub pending: bool, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum ReplicaLocation { + Unmanaged(UnmanagedLocation), + Managed(ManagedLocation), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum RoleId { + System(u64), + User(u64), + Public, + Predefined(u64), +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleAttributes { + pub inherit: bool, + pub superuser: Option, + pub login: Option, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleMembership { + pub map: Vec, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleMembershipEntry { + pub key: RoleId, + pub value: RoleId, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleVars { + pub entries: Vec, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleVarsEntry { + pub key: String, + pub val: RoleVar, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum RoleVar { + Flat(String), + SqlSet(Vec), +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct NetworkPolicyRule { + pub name: String, + pub address: String, + pub action: NetworkPolicyRuleAction, + pub direction: NetworkPolicyRuleDirection, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum NetworkPolicyRuleAction { + Allow, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub enum NetworkPolicyRuleDirection { + Ingress, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct AclMode { + pub bitflags: u64, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct MzAclItem { + pub grantee: RoleId, + pub grantor: RoleId, + pub acl_mode: AclMode, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct DefaultPrivilegesKey { + pub role_id: RoleId, + pub database_id: Option, + pub schema_id: Option, + pub object_type: ObjectType, + pub grantee: RoleId, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct DefaultPrivilegesValue { + pub privileges: AclMode, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SystemPrivilegesKey { + pub grantee: RoleId, + pub grantor: RoleId, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SystemPrivilegesValue { + pub acl_mode: AclMode, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct AuditLogEventV1 { + pub id: u64, + pub event_type: audit_log_event_v1::EventType, + pub object_type: audit_log_event_v1::ObjectType, + pub user: Option, + pub occurred_at: EpochMillis, + pub details: audit_log_event_v1::Details, +} + +pub mod audit_log_event_v1 { + use super::*; + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct IdFullNameV1 { + pub id: String, + pub name: FullNameV1, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct FullNameV1 { + pub database: String, + pub schema: String, + pub item: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct IdNameV1 { + pub id: String, + pub name: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RenameClusterV1 { + pub id: String, + pub old_name: String, + pub new_name: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RenameClusterReplicaV1 { + pub cluster_id: String, + pub replica_id: String, + pub old_name: String, + pub new_name: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RenameItemV1 { + pub id: String, + pub old_name: FullNameV1, + pub new_name: FullNameV1, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateClusterReplicaV1 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + pub logical_size: String, + pub disk: bool, + pub billed_as: Option, + pub internal: bool, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateClusterReplicaV2 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + pub logical_size: String, + pub disk: bool, + pub billed_as: Option, + pub internal: bool, + pub reason: CreateOrDropClusterReplicaReasonV1, + pub scheduling_policies: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateClusterReplicaV3 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + pub logical_size: String, + pub disk: bool, + pub billed_as: Option, + pub internal: bool, + pub reason: CreateOrDropClusterReplicaReasonV1, + pub scheduling_policies: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateClusterReplicaV4 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + pub logical_size: String, + pub billed_as: Option, + pub internal: bool, + pub reason: CreateOrDropClusterReplicaReasonV1, + pub scheduling_policies: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct DropClusterReplicaV1 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct DropClusterReplicaV2 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + pub reason: CreateOrDropClusterReplicaReasonV1, + pub scheduling_policies: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct DropClusterReplicaV3 { + pub cluster_id: String, + pub cluster_name: String, + pub replica_id: Option, + pub replica_name: String, + pub reason: CreateOrDropClusterReplicaReasonV1, + pub scheduling_policies: Option, + } + + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateOrDropClusterReplicaReasonV1 { + pub reason: CreateOrDropClusterReplicaReasonV1Reason, + } + + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub enum CreateOrDropClusterReplicaReasonV1Reason { + Manual(Empty), + Schedule(Empty), + System(Empty), + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct SchedulingDecisionsWithReasonsV1 { + pub on_refresh: RefreshDecisionWithReasonV1, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct SchedulingDecisionsWithReasonsV2 { + pub on_refresh: RefreshDecisionWithReasonV2, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub enum RefreshDecision { + On(Empty), + Off(Empty), + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RefreshDecisionWithReasonV1 { + pub objects_needing_refresh: Vec, + pub rehydration_time_estimate: String, + pub decision: RefreshDecision, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RefreshDecisionWithReasonV2 { + pub objects_needing_refresh: Vec, + pub objects_needing_compaction: Vec, + pub rehydration_time_estimate: String, + pub decision: RefreshDecision, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateSourceSinkV1 { + pub id: String, + pub name: FullNameV1, + pub size: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateSourceSinkV2 { + pub id: String, + pub name: FullNameV1, + pub size: Option, + pub external_type: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateSourceSinkV3 { + pub id: String, + pub name: FullNameV1, + pub external_type: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateSourceSinkV4 { + pub id: String, + pub cluster_id: Option, + pub name: FullNameV1, + pub external_type: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateIndexV1 { + pub id: String, + pub cluster_id: String, + pub name: FullNameV1, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct CreateMaterializedViewV1 { + pub id: String, + pub cluster_id: String, + pub name: FullNameV1, + pub replacement_target_id: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct AlterApplyReplacementV1 { + pub target: IdFullNameV1, + pub replacement: IdFullNameV1, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct AlterSourceSinkV1 { + pub id: String, + pub name: FullNameV1, + pub old_size: Option, + pub new_size: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct AlterSetClusterV1 { + pub id: String, + pub name: FullNameV1, + pub old_cluster_id: String, + pub new_cluster_id: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct GrantRoleV1 { + pub role_id: String, + pub member_id: String, + pub grantor_id: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct GrantRoleV2 { + pub role_id: String, + pub member_id: String, + pub grantor_id: String, + pub executed_by: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RevokeRoleV1 { + pub role_id: String, + pub member_id: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RevokeRoleV2 { + pub role_id: String, + pub member_id: String, + pub grantor_id: String, + pub executed_by: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct UpdatePrivilegeV1 { + pub object_id: String, + pub grantee_id: String, + pub grantor_id: String, + pub privileges: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct AlterDefaultPrivilegeV1 { + pub role_id: String, + pub database_id: Option, + pub schema_id: Option, + pub grantee_id: String, + pub privileges: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct UpdateOwnerV1 { + pub object_id: String, + pub old_owner_id: String, + pub new_owner_id: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct SchemaV1 { + pub id: String, + pub name: String, + pub database_name: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct SchemaV2 { + pub id: String, + pub name: String, + pub database_name: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RenameSchemaV1 { + pub id: String, + pub database_name: Option, + pub old_name: String, + pub new_name: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct UpdateItemV1 { + pub id: String, + pub name: FullNameV1, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct AlterRetainHistoryV1 { + pub id: String, + pub old_history: Option, + pub new_history: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct ToNewIdV1 { + pub id: String, + pub new_id: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct FromPreviousIdV1 { + pub id: String, + pub previous_id: String, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct SetV1 { + pub name: String, + pub value: Option, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub struct RotateKeysV1 { + pub id: String, + pub name: String, + } + + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + Serialize_repr, + Deserialize_repr, + Arbitrary + )] + #[repr(u8)] + pub enum EventType { + Unknown = 0, + Create = 1, + Drop = 2, + Alter = 3, + Grant = 4, + Revoke = 5, + Comment = 6, + } + + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + Serialize_repr, + Deserialize_repr, + Arbitrary + )] + #[repr(u8)] + pub enum ObjectType { + Unknown = 0, + Cluster = 1, + ClusterReplica = 2, + Connection = 3, + Database = 4, + Func = 5, + Index = 6, + MaterializedView = 7, + Role = 8, + Secret = 9, + Schema = 10, + Sink = 11, + Source = 12, + Table = 13, + Type = 14, + View = 15, + System = 16, + ContinualTask = 17, + NetworkPolicy = 18, + } + + #[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary + )] + pub enum Details { + CreateClusterReplicaV1(CreateClusterReplicaV1), + CreateClusterReplicaV2(CreateClusterReplicaV2), + CreateClusterReplicaV3(CreateClusterReplicaV3), + CreateClusterReplicaV4(CreateClusterReplicaV4), + DropClusterReplicaV1(DropClusterReplicaV1), + DropClusterReplicaV2(DropClusterReplicaV2), + DropClusterReplicaV3(DropClusterReplicaV3), + CreateSourceSinkV1(CreateSourceSinkV1), + CreateSourceSinkV2(CreateSourceSinkV2), + AlterSourceSinkV1(AlterSourceSinkV1), + AlterSetClusterV1(AlterSetClusterV1), + GrantRoleV1(GrantRoleV1), + GrantRoleV2(GrantRoleV2), + RevokeRoleV1(RevokeRoleV1), + RevokeRoleV2(RevokeRoleV2), + UpdatePrivilegeV1(UpdatePrivilegeV1), + AlterDefaultPrivilegeV1(AlterDefaultPrivilegeV1), + UpdateOwnerV1(UpdateOwnerV1), + IdFullNameV1(IdFullNameV1), + RenameClusterV1(RenameClusterV1), + RenameClusterReplicaV1(RenameClusterReplicaV1), + RenameItemV1(RenameItemV1), + IdNameV1(IdNameV1), + SchemaV1(SchemaV1), + SchemaV2(SchemaV2), + RenameSchemaV1(RenameSchemaV1), + UpdateItemV1(UpdateItemV1), + CreateSourceSinkV3(CreateSourceSinkV3), + AlterRetainHistoryV1(AlterRetainHistoryV1), + ToNewIdV1(ToNewIdV1), + FromPreviousIdV1(FromPreviousIdV1), + SetV1(SetV1), + ResetAllV1(Empty), + RotateKeysV1(RotateKeysV1), + CreateSourceSinkV4(CreateSourceSinkV4), + CreateIndexV1(CreateIndexV1), + CreateMaterializedViewV1(CreateMaterializedViewV1), + AlterApplyReplacementV1(AlterApplyReplacementV1), + } +} + +/// The contents of a single state update. +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +// Serialize the top-level enum in the persist-backed catalog as internally tagged to set up +// persist pushdown statistics for success. +#[serde(tag = "kind")] +pub enum StateUpdateKind { + AuditLog(AuditLog), + Cluster(Cluster), + ClusterIntrospectionSourceIndex(ClusterIntrospectionSourceIndex), + ClusterReplica(ClusterReplica), + Comment(Comment), + Config(Config), + Database(Database), + DefaultPrivileges(DefaultPrivileges), + FenceToken(FenceToken), + GidMapping(GidMapping), + IdAlloc(IdAlloc), + Item(Item), + NetworkPolicy(NetworkPolicy), + PersistedIntrospectionSource(PersistedIntrospectionSource), + Role(Role), + RoleAuth(RoleAuth), + Schema(Schema), + ServerConfiguration(ServerConfiguration), + Setting(Setting), + SourceReferences(SourceReferences), + StorageCollectionMetadata(StorageCollectionMetadata), + SystemPrivileges(SystemPrivileges), + TxnWalShard(TxnWalShard), + UnfinalizedShard(UnfinalizedShard), +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct AuditLog { + pub key: AuditLogKey, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Cluster { + pub key: ClusterKey, + pub value: ClusterValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterReplica { + pub key: ClusterReplicaKey, + pub value: ClusterReplicaValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Comment { + pub key: CommentKey, + pub value: CommentValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Config { + pub key: ConfigKey, + pub value: ConfigValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Database { + pub key: DatabaseKey, + pub value: DatabaseValue, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct DefaultPrivileges { + pub key: DefaultPrivilegesKey, + pub value: DefaultPrivilegesValue, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct FenceToken { + pub deploy_generation: u64, + pub epoch: i64, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct IdAlloc { + pub key: IdAllocKey, + pub value: IdAllocValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ClusterIntrospectionSourceIndex { + pub key: ClusterIntrospectionSourceIndexKey, + pub value: ClusterIntrospectionSourceIndexValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct PersistedIntrospectionSource { + pub key: PersistedIntrospectionSourceKey, + pub value: PersistedIntrospectionSourceValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Item { + pub key: ItemKey, + pub value: ItemValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Role { + pub key: RoleKey, + pub value: RoleValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct RoleAuth { + pub key: RoleAuthKey, + pub value: RoleAuthValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct NetworkPolicy { + pub key: NetworkPolicyKey, + pub value: NetworkPolicyValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Schema { + pub key: SchemaKey, + pub value: SchemaValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct Setting { + pub key: SettingKey, + pub value: SettingValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct ServerConfiguration { + pub key: ServerConfigurationKey, + pub value: ServerConfigurationValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SourceReferences { + pub key: SourceReferencesKey, + pub value: SourceReferencesValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct GidMapping { + pub key: GidMappingKey, + pub value: GidMappingValue, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct SystemPrivileges { + pub key: SystemPrivilegesKey, + pub value: SystemPrivilegesValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct StorageCollectionMetadata { + pub key: StorageCollectionMetadataKey, + pub value: StorageCollectionMetadataValue, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct UnfinalizedShard { + pub key: UnfinalizedShardKey, +} + +#[derive( + Clone, + Debug, + PartialEq, + Eq, + PartialOrd, + Ord, + Serialize, + Deserialize, + Arbitrary +)] +pub struct TxnWalShard { + pub value: TxnWalShardValue, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + Serialize_repr, + Deserialize_repr, + Arbitrary +)] +#[repr(u8)] +pub enum CatalogItemType { + Unknown = 0, + Table = 1, + Source = 2, + Sink = 3, + View = 4, + MaterializedView = 5, + Index = 6, + Type = 7, + Func = 8, + Secret = 9, + Connection = 10, + ContinualTask = 11, +} + +#[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + Serialize_repr, + Deserialize_repr, + Arbitrary +)] +#[repr(u8)] +pub enum ObjectType { + Unknown = 0, + Table = 1, + View = 2, + MaterializedView = 3, + Source = 4, + Sink = 5, + Index = 6, + Type = 7, + Role = 8, + Cluster = 9, + ClusterReplica = 10, + Secret = 11, + Connection = 12, + Database = 13, + Schema = 14, + Func = 15, + ContinualTask = 16, + NetworkPolicy = 17, +} diff --git a/src/catalog-protos/src/serialization.rs b/src/catalog-protos/src/serialization.rs index e605d7ad703ed..8fd214c67e996 100644 --- a/src/catalog-protos/src/serialization.rs +++ b/src/catalog-protos/src/serialization.rs @@ -529,6 +529,9 @@ impl RustType for CatalogItemId { CatalogItemId::IntrospectionSourceIndex(x) => { crate::objects::CatalogItemId::IntrospectionSourceIndex(*x) } + CatalogItemId::PersistedIntrospectionSource(x) => { + crate::objects::CatalogItemId::PersistedIntrospectionSource(*x) + } CatalogItemId::User(x) => crate::objects::CatalogItemId::User(*x), CatalogItemId::Transient(x) => crate::objects::CatalogItemId::Transient(*x), } @@ -540,6 +543,9 @@ impl RustType for CatalogItemId { crate::objects::CatalogItemId::IntrospectionSourceIndex(x) => { Ok(CatalogItemId::IntrospectionSourceIndex(x)) } + crate::objects::CatalogItemId::PersistedIntrospectionSource(x) => { + Ok(CatalogItemId::PersistedIntrospectionSource(x)) + } crate::objects::CatalogItemId::User(x) => Ok(CatalogItemId::User(x)), crate::objects::CatalogItemId::Transient(x) => Ok(CatalogItemId::Transient(x)), } @@ -553,6 +559,9 @@ impl RustType for GlobalId { GlobalId::IntrospectionSourceIndex(x) => { crate::objects::GlobalId::IntrospectionSourceIndex(*x) } + GlobalId::PersistedIntrospectionSource(x) => { + crate::objects::GlobalId::PersistedIntrospectionSource(*x) + } GlobalId::User(x) => crate::objects::GlobalId::User(*x), GlobalId::Transient(x) => crate::objects::GlobalId::Transient(*x), GlobalId::Explain => crate::objects::GlobalId::Explain, @@ -565,6 +574,9 @@ impl RustType for GlobalId { crate::objects::GlobalId::IntrospectionSourceIndex(x) => { Ok(GlobalId::IntrospectionSourceIndex(x)) } + crate::objects::GlobalId::PersistedIntrospectionSource(x) => { + Ok(GlobalId::PersistedIntrospectionSource(x)) + } crate::objects::GlobalId::User(x) => Ok(GlobalId::User(x)), crate::objects::GlobalId::Transient(x) => Ok(GlobalId::Transient(x)), crate::objects::GlobalId::Explain => Ok(GlobalId::Explain), diff --git a/src/catalog/src/builtin.rs b/src/catalog/src/builtin.rs index e3bea6326ef6a..e99d9ee9a7a8f 100644 --- a/src/catalog/src/builtin.rs +++ b/src/catalog/src/builtin.rs @@ -14237,6 +14237,9 @@ pub mod BUILTINS { pub static BUILTIN_LOG_LOOKUP: LazyLock> = LazyLock::new(|| BUILTINS::logs().map(|log| (log.name, log)).collect()); +/// Reverse lookup: LogVariant → BuiltinLog. +pub static BUILTIN_LOG_BY_VARIANT: LazyLock> = + LazyLock::new(|| BUILTINS::logs().map(|log| (log.variant, log)).collect()); /// Keys are builtin object description, values are the builtin index when sorted by dependency and /// the builtin itself. pub static BUILTIN_LOOKUP: LazyLock< diff --git a/src/catalog/src/durable.rs b/src/catalog/src/durable.rs index aed7c5b728e66..c9278280125b4 100644 --- a/src/catalog/src/durable.rs +++ b/src/catalog/src/durable.rs @@ -34,10 +34,10 @@ use crate::durable::objects::state_update::{StateUpdateKindJson, TryIntoStateUpd use crate::durable::objects::{AuditLog, Snapshot}; pub use crate::durable::objects::{ Cluster, ClusterConfig, ClusterReplica, ClusterVariant, ClusterVariantManaged, Comment, - Database, DefaultPrivilege, IntrospectionSourceIndex, Item, NetworkPolicy, ReplicaConfig, - ReplicaLocation, Role, RoleAuth, Schema, SourceReference, SourceReferences, - StorageCollectionMetadata, SystemConfiguration, SystemObjectDescription, SystemObjectMapping, - UnfinalizedShard, + Database, DefaultPrivilege, IntrospectionSourceIndex, Item, NetworkPolicy, + PersistedIntrospectionSource, ReplicaConfig, ReplicaLocation, Role, RoleAuth, Schema, + SourceReference, SourceReferences, StorageCollectionMetadata, SystemConfiguration, + SystemObjectDescription, SystemObjectMapping, UnfinalizedShard, }; pub use crate::durable::persist::shard_id; use crate::durable::persist::{Timestamp, UnopenedPersistCatalogState}; diff --git a/src/catalog/src/durable/debug.rs b/src/catalog/src/durable/debug.rs index 5586053e8bc69..c32d686a2fc8f 100644 --- a/src/catalog/src/durable/debug.rs +++ b/src/catalog/src/durable/debug.rs @@ -64,6 +64,7 @@ pub enum CollectionType { IdAlloc, Item, NetworkPolicy, + PersistedIntrospectionSource, Role, RoleAuth, Schema, @@ -208,6 +209,14 @@ collection_impl!({ trace_field: network_policies, update: StateUpdateKind::NetworkPolicy, }); +collection_impl!({ + name: PersistedIntrospectionSourceCollection, + key: proto::PersistedIntrospectionSourceKey, + value: proto::PersistedIntrospectionSourceValue, + collection_type: CollectionType::PersistedIntrospectionSource, + trace_field: persisted_introspection_sources, + update: StateUpdateKind::PersistedIntrospectionSource, +}); collection_impl!({ name: RoleCollection, key: proto::RoleKey, @@ -339,6 +348,7 @@ pub struct Trace { pub id_allocator: CollectionTrace, pub items: CollectionTrace, pub network_policies: CollectionTrace, + pub persisted_introspection_sources: CollectionTrace, pub roles: CollectionTrace, pub role_auth: CollectionTrace, pub schemas: CollectionTrace, @@ -366,6 +376,7 @@ impl Trace { id_allocator: CollectionTrace::new(), items: CollectionTrace::new(), network_policies: CollectionTrace::new(), + persisted_introspection_sources: CollectionTrace::new(), roles: CollectionTrace::new(), role_auth: CollectionTrace::new(), schemas: CollectionTrace::new(), @@ -393,6 +404,7 @@ impl Trace { id_allocator, items, network_policies, + persisted_introspection_sources, roles, role_auth, schemas, @@ -416,6 +428,7 @@ impl Trace { id_allocator.sort(); items.sort(); network_policies.sort(); + persisted_introspection_sources.sort(); roles.sort(); role_auth.sort(); schemas.sort(); diff --git a/src/catalog/src/durable/initialize.rs b/src/catalog/src/durable/initialize.rs index 4256bc99a9cf5..3631fb4018d21 100644 --- a/src/catalog/src/durable/initialize.rs +++ b/src/catalog/src/durable/initialize.rs @@ -793,6 +793,7 @@ fn default_cluster_config(args: &BootstrapArgs) -> Result, pub logging: ReplicaLogging, + pub persist_introspection: bool, pub replication_factor: u32, pub optimizer_feature_overrides: BTreeMap, pub schedule: ClusterSchedule, @@ -403,6 +404,62 @@ impl DurableType for IntrospectionSourceIndex { } } +#[derive(Clone, Debug, Ord, PartialOrd, PartialEq, Eq)] +pub struct PersistedIntrospectionSource { + pub cluster_id: ClusterId, + pub replica_id: ReplicaId, + pub name: String, + pub item_id: CatalogItemId, + pub global_id: GlobalId, + pub oid: u32, + pub schema_id: SchemaId, +} + +impl DurableType for PersistedIntrospectionSource { + type Key = PersistedIntrospectionSourceKey; + type Value = PersistedIntrospectionSourceValue; + + fn into_key_value(self) -> (Self::Key, Self::Value) { + ( + PersistedIntrospectionSourceKey { + cluster_id: self.cluster_id, + replica_id: self.replica_id, + name: self.name, + }, + PersistedIntrospectionSourceValue { + catalog_id: self.item_id.try_into().expect( + "persisted introspection source must be a PersistedIntrospectionSource ID", + ), + global_id: self.global_id.try_into().expect( + "persisted introspection source must be a PersistedIntrospectionSource ID", + ), + oid: self.oid, + schema_id: self.schema_id, + }, + ) + } + + fn from_key_value(key: Self::Key, value: Self::Value) -> Self { + Self { + cluster_id: key.cluster_id, + replica_id: key.replica_id, + name: key.name, + item_id: value.catalog_id.into(), + global_id: value.global_id.into(), + oid: value.oid, + schema_id: value.schema_id, + } + } + + fn key(&self) -> Self::Key { + PersistedIntrospectionSourceKey { + cluster_id: self.cluster_id, + replica_id: self.replica_id, + name: self.name.clone(), + } + } +} + #[derive(Debug, Clone, Ord, PartialOrd, PartialEq, Eq)] pub struct ClusterReplica { pub cluster_id: ClusterId, @@ -635,6 +692,7 @@ impl TryFrom for SystemCatalogItemId { match val { CatalogItemId::System(x) => Ok(SystemCatalogItemId(x)), CatalogItemId::IntrospectionSourceIndex(_) => Err("introspection_source_index"), + CatalogItemId::PersistedIntrospectionSource(_) => Err("persisted_introspection_source"), CatalogItemId::User(_) => Err("user"), CatalogItemId::Transient(_) => Err("transient"), } @@ -660,6 +718,7 @@ impl TryFrom for IntrospectionSourceIndexCatalogItemId { CatalogItemId::IntrospectionSourceIndex(x) => { Ok(IntrospectionSourceIndexCatalogItemId(x)) } + CatalogItemId::PersistedIntrospectionSource(_) => Err("persisted_introspection_source"), CatalogItemId::User(_) => Err("user"), CatalogItemId::Transient(_) => Err("transient"), } @@ -683,6 +742,7 @@ impl TryFrom for SystemGlobalId { match val { GlobalId::System(x) => Ok(SystemGlobalId(x)), GlobalId::IntrospectionSourceIndex(_) => Err("introspection_source_index"), + GlobalId::PersistedIntrospectionSource(_) => Err("persisted_introspection_source"), GlobalId::User(_) => Err("user"), GlobalId::Transient(_) => Err("transient"), GlobalId::Explain => Err("explain"), @@ -707,6 +767,7 @@ impl TryFrom for IntrospectionSourceIndexGlobalId { match val { GlobalId::System(_) => Err("system"), GlobalId::IntrospectionSourceIndex(x) => Ok(IntrospectionSourceIndexGlobalId(x)), + GlobalId::PersistedIntrospectionSource(_) => Err("persisted_introspection_source"), GlobalId::User(_) => Err("user"), GlobalId::Transient(_) => Err("transient"), GlobalId::Explain => Err("explain"), @@ -720,6 +781,59 @@ impl From for GlobalId { } } +/// A newtype wrapper for [`CatalogItemId`] that is only for the "persisted introspection source" namespace. +#[derive(Debug, Copy, Clone, Ord, PartialOrd, PartialEq, Eq)] +pub struct PersistedIntrospectionSourceCatalogItemId(u64); + +impl TryFrom for PersistedIntrospectionSourceCatalogItemId { + type Error = &'static str; + + fn try_from(val: CatalogItemId) -> Result { + match val { + CatalogItemId::System(_) => Err("system"), + CatalogItemId::IntrospectionSourceIndex(_) => Err("introspection_source_index"), + CatalogItemId::PersistedIntrospectionSource(x) => { + Ok(PersistedIntrospectionSourceCatalogItemId(x)) + } + CatalogItemId::User(_) => Err("user"), + CatalogItemId::Transient(_) => Err("transient"), + } + } +} + +impl From for CatalogItemId { + fn from(val: PersistedIntrospectionSourceCatalogItemId) -> Self { + CatalogItemId::PersistedIntrospectionSource(val.0) + } +} + +/// A newtype wrapper for [`GlobalId`] that is only for the "persisted introspection source" namespace. +#[derive(Debug, Copy, Clone, Ord, PartialOrd, PartialEq, Eq)] +pub struct PersistedIntrospectionSourceGlobalId(u64); + +impl TryFrom for PersistedIntrospectionSourceGlobalId { + type Error = &'static str; + + fn try_from(val: GlobalId) -> Result { + match val { + GlobalId::System(_) => Err("system"), + GlobalId::IntrospectionSourceIndex(_) => Err("introspection_source_index"), + GlobalId::PersistedIntrospectionSource(x) => { + Ok(PersistedIntrospectionSourceGlobalId(x)) + } + GlobalId::User(_) => Err("user"), + GlobalId::Transient(_) => Err("transient"), + GlobalId::Explain => Err("explain"), + } + } +} + +impl From for GlobalId { + fn from(val: PersistedIntrospectionSourceGlobalId) -> Self { + GlobalId::PersistedIntrospectionSource(val.0) + } +} + #[derive(Debug, Clone, PartialOrd, Ord, PartialEq, Eq, Hash)] pub struct SystemObjectDescription { pub schema_name: String, @@ -1147,6 +1261,8 @@ pub struct Snapshot { proto::ClusterIntrospectionSourceIndexKey, proto::ClusterIntrospectionSourceIndexValue, >, + pub persisted_introspection_sources: + BTreeMap, pub id_allocator: BTreeMap, pub configs: BTreeMap, pub settings: BTreeMap, @@ -1252,6 +1368,21 @@ pub struct ClusterIntrospectionSourceIndexValue { pub(crate) oid: u32, } +#[derive(Debug, Clone, PartialOrd, PartialEq, Eq, Ord, Hash)] +pub struct PersistedIntrospectionSourceKey { + pub(crate) cluster_id: ClusterId, + pub(crate) replica_id: ReplicaId, + pub(crate) name: String, +} + +#[derive(Debug, Clone, PartialOrd, PartialEq, Eq, Ord)] +pub struct PersistedIntrospectionSourceValue { + pub(crate) catalog_id: PersistedIntrospectionSourceCatalogItemId, + pub(crate) global_id: PersistedIntrospectionSourceGlobalId, + pub(crate) oid: u32, + pub(crate) schema_id: SchemaId, +} + #[derive(Debug, Clone, PartialOrd, PartialEq, Eq, Ord, Hash)] pub struct ClusterReplicaKey { pub(crate) id: ReplicaId, diff --git a/src/catalog/src/durable/objects/serialization.rs b/src/catalog/src/durable/objects/serialization.rs index 28a164509d6d3..2f0f4b1d4777b 100644 --- a/src/catalog/src/durable/objects/serialization.rs +++ b/src/catalog/src/durable/objects/serialization.rs @@ -19,7 +19,9 @@ use crate::durable::objects::{ ConfigKey, ConfigValue, DatabaseKey, DatabaseValue, DefaultPrivilegesKey, DefaultPrivilegesValue, GidMappingKey, GidMappingValue, IdAllocKey, IdAllocValue, IntrospectionSourceIndexCatalogItemId, IntrospectionSourceIndexGlobalId, ItemKey, ItemValue, - NetworkPolicyKey, NetworkPolicyValue, RoleKey, RoleValue, SchemaKey, SchemaValue, + NetworkPolicyKey, NetworkPolicyValue, PersistedIntrospectionSourceCatalogItemId, + PersistedIntrospectionSourceGlobalId, PersistedIntrospectionSourceKey, + PersistedIntrospectionSourceValue, RoleKey, RoleValue, SchemaKey, SchemaValue, ServerConfigurationKey, ServerConfigurationValue, SettingKey, SettingValue, SourceReference, SourceReferencesKey, SourceReferencesValue, StorageCollectionMetadataKey, StorageCollectionMetadataValue, SystemCatalogItemId, SystemGlobalId, SystemPrivilegesKey, @@ -75,6 +77,7 @@ impl RustType for ClusterVariant { replication_factor, optimizer_feature_overrides, schedule, + persist_introspection, }) => proto::ClusterVariant::Managed(proto::ManagedCluster { size: size.to_string(), availability_zones: availability_zones.clone(), @@ -82,6 +85,7 @@ impl RustType for ClusterVariant { replication_factor: *replication_factor, optimizer_feature_overrides: optimizer_feature_overrides.into_proto(), schedule: schedule.into_proto(), + persist_introspection: *persist_introspection, }), ClusterVariant::Unmanaged => proto::ClusterVariant::Unmanaged, } @@ -97,6 +101,7 @@ impl RustType for ClusterVariant { replication_factor: managed.replication_factor, optimizer_feature_overrides: managed.optimizer_feature_overrides.into_rust()?, schedule: managed.schedule.into_rust()?, + persist_introspection: managed.persist_introspection, })), } } @@ -916,6 +921,76 @@ impl RustType for IntrospectionSourceIn } } +impl RustType + for PersistedIntrospectionSourceCatalogItemId +{ + fn into_proto(&self) -> proto::PersistedIntrospectionSourceCatalogItemId { + proto::PersistedIntrospectionSourceCatalogItemId(self.0) + } + + fn from_proto( + proto: proto::PersistedIntrospectionSourceCatalogItemId, + ) -> Result { + Ok(PersistedIntrospectionSourceCatalogItemId(proto.0)) + } +} + +impl RustType + for PersistedIntrospectionSourceGlobalId +{ + fn into_proto(&self) -> proto::PersistedIntrospectionSourceGlobalId { + proto::PersistedIntrospectionSourceGlobalId(self.0) + } + + fn from_proto( + proto: proto::PersistedIntrospectionSourceGlobalId, + ) -> Result { + Ok(PersistedIntrospectionSourceGlobalId(proto.0)) + } +} + +impl RustType for PersistedIntrospectionSourceKey { + fn into_proto(&self) -> proto::PersistedIntrospectionSourceKey { + proto::PersistedIntrospectionSourceKey { + cluster_id: self.cluster_id.into_proto(), + replica_id: self.replica_id.into_proto(), + name: self.name.to_string(), + } + } + + fn from_proto( + proto: proto::PersistedIntrospectionSourceKey, + ) -> Result { + Ok(PersistedIntrospectionSourceKey { + cluster_id: proto.cluster_id.into_rust()?, + replica_id: proto.replica_id.into_rust()?, + name: proto.name, + }) + } +} + +impl RustType for PersistedIntrospectionSourceValue { + fn into_proto(&self) -> proto::PersistedIntrospectionSourceValue { + proto::PersistedIntrospectionSourceValue { + catalog_id: self.catalog_id.into_proto(), + global_id: self.global_id.into_proto(), + oid: self.oid, + schema_id: self.schema_id.into_proto(), + } + } + + fn from_proto( + proto: proto::PersistedIntrospectionSourceValue, + ) -> Result { + Ok(PersistedIntrospectionSourceValue { + catalog_id: proto.catalog_id.into_rust()?, + global_id: proto.global_id.into_rust()?, + oid: proto.oid, + schema_id: proto.schema_id.into_rust()?, + }) + } +} + #[cfg(test)] mod tests { use mz_audit_log::VersionedEvent; diff --git a/src/catalog/src/durable/objects/state_update.rs b/src/catalog/src/durable/objects/state_update.rs index 4039c61d3d52e..132d890b2ec38 100644 --- a/src/catalog/src/durable/objects/state_update.rs +++ b/src/catalog/src/durable/objects/state_update.rs @@ -142,6 +142,7 @@ impl StateUpdate { cluster_replicas, network_policies, introspection_sources, + persisted_introspection_sources, id_allocator, configs, settings, @@ -169,6 +170,10 @@ impl StateUpdate { introspection_sources, StateUpdateKind::IntrospectionSourceIndex, ); + let persisted_introspection_sources = from_batch( + persisted_introspection_sources, + StateUpdateKind::PersistedIntrospectionSource, + ); let id_allocators = from_batch(id_allocator, StateUpdateKind::IdAllocator); let configs = from_batch(configs, StateUpdateKind::Config); let settings = from_batch(settings, StateUpdateKind::Setting); @@ -197,6 +202,7 @@ impl StateUpdate { .chain(cluster_replicas) .chain(network_policies) .chain(introspection_sources) + .chain(persisted_introspection_sources) .chain(id_allocators) .chain(configs) .chain(settings) @@ -233,6 +239,10 @@ pub enum StateUpdateKind { ), Item(proto::ItemKey, proto::ItemValue), NetworkPolicy(proto::NetworkPolicyKey, proto::NetworkPolicyValue), + PersistedIntrospectionSource( + proto::PersistedIntrospectionSourceKey, + proto::PersistedIntrospectionSourceValue, + ), Role(proto::RoleKey, proto::RoleValue), RoleAuth(proto::RoleAuthKey, proto::RoleAuthValue), Schema(proto::SchemaKey, proto::SchemaValue), @@ -269,6 +279,9 @@ impl StateUpdateKind { } StateUpdateKind::Item(_, _) => Some(CollectionType::Item), StateUpdateKind::NetworkPolicy(_, _) => Some(CollectionType::NetworkPolicy), + StateUpdateKind::PersistedIntrospectionSource(_, _) => { + Some(CollectionType::PersistedIntrospectionSource) + } StateUpdateKind::Role(_, _) => Some(CollectionType::Role), StateUpdateKind::RoleAuth(_, _) => Some(CollectionType::RoleAuth), StateUpdateKind::Schema(_, _) => Some(CollectionType::Schema), @@ -496,6 +509,10 @@ impl TryFrom<&StateUpdateKind> for Option { let policy = into_durable(key, value)?; Some(memory::objects::StateUpdateKind::NetworkPolicy(policy)) } + StateUpdateKind::PersistedIntrospectionSource(key, value) => { + let source = into_durable(key, value)?; + Some(memory::objects::StateUpdateKind::PersistedIntrospectionSource(source)) + } StateUpdateKind::Role(key, value) => { let role = into_durable(key, value)?; Some(memory::objects::StateUpdateKind::Role(role)) @@ -643,6 +660,11 @@ impl RustType for StateUpdateKind { StateUpdateKind::NetworkPolicy(key, value) => { proto::StateUpdateKind::NetworkPolicy(proto::NetworkPolicy { key, value }) } + StateUpdateKind::PersistedIntrospectionSource(key, value) => { + proto::StateUpdateKind::PersistedIntrospectionSource( + proto::PersistedIntrospectionSource { key, value }, + ) + } StateUpdateKind::Role(key, value) => { proto::StateUpdateKind::Role(proto::Role { key, value }) } @@ -725,6 +747,9 @@ impl RustType for StateUpdateKind { proto::StateUpdateKind::Item(proto::Item { key, value }) => { StateUpdateKind::Item(key, value) } + proto::StateUpdateKind::PersistedIntrospectionSource( + proto::PersistedIntrospectionSource { key, value }, + ) => StateUpdateKind::PersistedIntrospectionSource(key, value), proto::StateUpdateKind::Role(proto::Role { key, value }) => { StateUpdateKind::Role(key, value) } diff --git a/src/catalog/src/durable/persist.rs b/src/catalog/src/durable/persist.rs index c20af6b291095..cd9df548b7189 100644 --- a/src/catalog/src/durable/persist.rs +++ b/src/catalog/src/durable/persist.rs @@ -809,6 +809,14 @@ impl> PersistHandle { StateUpdateKind::NetworkPolicy(key, value) => { apply(&mut snapshot.network_policies, key, value, diff); } + StateUpdateKind::PersistedIntrospectionSource(key, value) => { + apply( + &mut snapshot.persisted_introspection_sources, + key, + value, + diff, + ); + } StateUpdateKind::Role(key, value) => { apply(&mut snapshot.roles, key, value, diff); } @@ -1958,6 +1966,10 @@ impl Trace { StateUpdateKind::NetworkPolicy(k, v) => { trace.network_policies.values.push(((k, v), ts, diff)) } + StateUpdateKind::PersistedIntrospectionSource(k, v) => trace + .persisted_introspection_sources + .values + .push(((k, v), ts, diff)), StateUpdateKind::Role(k, v) => trace.roles.values.push(((k, v), ts, diff)), StateUpdateKind::Schema(k, v) => trace.schemas.values.push(((k, v), ts, diff)), StateUpdateKind::Setting(k, v) => trace.settings.values.push(((k, v), ts, diff)), diff --git a/src/catalog/src/durable/transaction.rs b/src/catalog/src/durable/transaction.rs index d38806f37ee85..04e9db4d26a7a 100644 --- a/src/catalog/src/durable/transaction.rs +++ b/src/catalog/src/durable/transaction.rs @@ -54,9 +54,10 @@ use crate::durable::objects::{ Database, DatabaseKey, DatabaseValue, DefaultPrivilegesKey, DefaultPrivilegesValue, DurableType, GidMappingKey, GidMappingValue, IdAllocKey, IdAllocValue, IntrospectionSourceIndex, Item, ItemKey, ItemValue, NetworkPolicyKey, NetworkPolicyValue, - ReplicaConfig, Role, RoleKey, RoleValue, Schema, SchemaKey, SchemaValue, - ServerConfigurationKey, ServerConfigurationValue, SettingKey, SettingValue, SourceReference, - SourceReferencesKey, SourceReferencesValue, StorageCollectionMetadataKey, + PersistedIntrospectionSource, PersistedIntrospectionSourceKey, + PersistedIntrospectionSourceValue, ReplicaConfig, Role, RoleKey, RoleValue, Schema, SchemaKey, + SchemaValue, ServerConfigurationKey, ServerConfigurationValue, SettingKey, SettingValue, + SourceReference, SourceReferencesKey, SourceReferencesValue, StorageCollectionMetadataKey, StorageCollectionMetadataValue, SystemObjectDescription, SystemObjectMapping, SystemPrivilegesKey, SystemPrivilegesValue, TxnWalShardValue, UnfinalizedShardKey, }; @@ -91,6 +92,8 @@ pub struct Transaction<'a> { cluster_replicas: TableTransaction, introspection_sources: TableTransaction, + persisted_introspection_sources: + TableTransaction, id_allocator: TableTransaction, configs: TableTransaction, settings: TableTransaction, @@ -127,6 +130,7 @@ impl<'a> Transaction<'a> { network_policies, cluster_replicas, introspection_sources, + persisted_introspection_sources, id_allocator, configs, settings, @@ -177,6 +181,9 @@ impl<'a> Transaction<'a> { |a: &ClusterReplicaValue, b| a.cluster_id == b.cluster_id && a.name == b.name, )?, introspection_sources: TableTransaction::new(introspection_sources)?, + persisted_introspection_sources: TableTransaction::new( + persisted_introspection_sources, + )?, id_allocator: TableTransaction::new(id_allocator)?, configs: TableTransaction::new(configs)?, settings: TableTransaction::new(settings)?, @@ -832,7 +839,57 @@ impl<'a> Transaction<'a> { 0, "invalid cluster ID: {cluster_id}" ); - let log_variant: u8 = match log_variant { + let log_variant: u8 = Transaction::log_variant_to_u8(log_variant); + + let mut id: u64 = u64::from(cluster_variant) << 56; + id |= cluster_id << 8; + id |= u64::from(log_variant); + + ( + CatalogItemId::IntrospectionSourceIndex(id), + GlobalId::IntrospectionSourceIndex(id), + ) + } + + /// Allocate a deterministic ID for a per-replica persisted introspection source. + /// + /// The ID encodes: + /// * Bits 56-63: Replica variant (3 = System, 4 = User) + /// * Bits 8-55: Replica ID (inner value) + /// * Bits 0-7: Log variant + /// + /// Replica variants 3/4 are distinct from cluster variants 1/2 used by + /// `allocate_introspection_source_index_id` to avoid ID collisions. + pub fn allocate_persisted_introspection_source_id( + replica_id: &ReplicaId, + log_variant: LogVariant, + ) -> (CatalogItemId, GlobalId) { + let replica_variant: u8 = match replica_id { + ReplicaId::System(_) => 3, + ReplicaId::User(_) => 4, + }; + let replica_inner: u64 = replica_id.inner_id(); + const REPLICA_ID_MASK: u64 = 0xFFFF << 48; + assert_eq!( + REPLICA_ID_MASK & replica_inner, + 0, + "invalid replica ID: {replica_inner}" + ); + let log_variant: u8 = Transaction::log_variant_to_u8(log_variant); + + let mut id: u64 = u64::from(replica_variant) << 56; + id |= replica_inner << 8; + id |= u64::from(log_variant); + + ( + CatalogItemId::PersistedIntrospectionSource(id), + GlobalId::PersistedIntrospectionSource(id), + ) + } + + /// Map a `LogVariant` to a `u8` for encoding in IDs. + fn log_variant_to_u8(log_variant: LogVariant) -> u8 { + match log_variant { LogVariant::Timely(TimelyLog::Operates) => 1, LogVariant::Timely(TimelyLog::Channels) => 2, LogVariant::Timely(TimelyLog::Elapsed) => 3, @@ -864,16 +921,57 @@ impl<'a> Transaction<'a> { LogVariant::Compute(ComputeLog::LirMapping) => 30, LogVariant::Compute(ComputeLog::DataflowGlobal) => 31, LogVariant::Compute(ComputeLog::OperatorHydrationStatus) => 32, - }; + } + } - let mut id: u64 = u64::from(cluster_variant) << 56; - id |= cluster_id << 8; - id |= u64::from(log_variant); + /// Reverse of `log_variant_to_u8`. + pub fn u8_to_log_variant(v: u8) -> Option { + match v { + 1 => Some(LogVariant::Timely(TimelyLog::Operates)), + 2 => Some(LogVariant::Timely(TimelyLog::Channels)), + 3 => Some(LogVariant::Timely(TimelyLog::Elapsed)), + 4 => Some(LogVariant::Timely(TimelyLog::Histogram)), + 5 => Some(LogVariant::Timely(TimelyLog::Addresses)), + 6 => Some(LogVariant::Timely(TimelyLog::Parks)), + 7 => Some(LogVariant::Timely(TimelyLog::MessagesSent)), + 8 => Some(LogVariant::Timely(TimelyLog::MessagesReceived)), + 9 => Some(LogVariant::Timely(TimelyLog::Reachability)), + 10 => Some(LogVariant::Timely(TimelyLog::BatchesSent)), + 11 => Some(LogVariant::Timely(TimelyLog::BatchesReceived)), + 12 => Some(LogVariant::Differential( + DifferentialLog::ArrangementBatches, + )), + 13 => Some(LogVariant::Differential( + DifferentialLog::ArrangementRecords, + )), + 14 => Some(LogVariant::Differential(DifferentialLog::Sharing)), + 15 => Some(LogVariant::Differential(DifferentialLog::BatcherRecords)), + 16 => Some(LogVariant::Differential(DifferentialLog::BatcherSize)), + 17 => Some(LogVariant::Differential(DifferentialLog::BatcherCapacity)), + 18 => Some(LogVariant::Differential( + DifferentialLog::BatcherAllocations, + )), + 19 => Some(LogVariant::Compute(ComputeLog::DataflowCurrent)), + 20 => Some(LogVariant::Compute(ComputeLog::FrontierCurrent)), + 21 => Some(LogVariant::Compute(ComputeLog::PeekCurrent)), + 22 => Some(LogVariant::Compute(ComputeLog::PeekDuration)), + 23 => Some(LogVariant::Compute(ComputeLog::ImportFrontierCurrent)), + 24 => Some(LogVariant::Compute(ComputeLog::ArrangementHeapSize)), + 25 => Some(LogVariant::Compute(ComputeLog::ArrangementHeapCapacity)), + 26 => Some(LogVariant::Compute(ComputeLog::ArrangementHeapAllocations)), + 28 => Some(LogVariant::Compute(ComputeLog::ErrorCount)), + 29 => Some(LogVariant::Compute(ComputeLog::HydrationTime)), + 30 => Some(LogVariant::Compute(ComputeLog::LirMapping)), + 31 => Some(LogVariant::Compute(ComputeLog::DataflowGlobal)), + 32 => Some(LogVariant::Compute(ComputeLog::OperatorHydrationStatus)), + _ => None, + } + } - ( - CatalogItemId::IntrospectionSourceIndex(id), - GlobalId::IntrospectionSourceIndex(id), - ) + /// Extract the `LogVariant` from a persisted introspection source ID. + pub fn decode_persisted_introspection_id(encoded_id: u64) -> Option { + let log_variant_u8 = u8::try_from(encoded_id & 0xFF).expect("mask guarantees fit in u8"); + Self::u8_to_log_variant(log_variant_u8) } pub fn allocate_user_item_ids( @@ -1434,6 +1532,45 @@ impl<'a> Transaction<'a> { } } + pub fn remove_persisted_introspection_sources( + &mut self, + sources: BTreeSet<(ClusterId, ReplicaId, String)>, + ) -> Result<(), CatalogError> { + if sources.is_empty() { + return Ok(()); + } + + let ks: Vec<_> = sources + .clone() + .into_iter() + .map( + |(cluster_id, replica_id, name)| PersistedIntrospectionSourceKey { + cluster_id, + replica_id, + name, + }, + ) + .collect(); + let n = self + .persisted_introspection_sources + .delete_by_keys(ks, self.op_id) + .len(); + if n == sources.len() { + Ok(()) + } else { + let txn_sources = self + .persisted_introspection_sources + .items() + .keys() + .map(|k| (k.cluster_id, k.replica_id, k.name.clone())) + .collect(); + let mut unknown = sources + .difference(&txn_sources) + .map(|(cluster_id, replica_id, name)| format!("{cluster_id} {replica_id} {name}")); + Err(SqlCatalogError::UnknownItem(unknown.join(", ")).into()) + } + } + /// Updates item `id` in the transaction to `item_name` and `item`. /// /// Returns an error if `id` is not found. @@ -1881,6 +2018,51 @@ impl<'a> Transaction<'a> { Ok(()) } + pub fn insert_persisted_introspection_sources( + &mut self, + sources: Vec<( + ClusterId, + ReplicaId, + String, + CatalogItemId, + GlobalId, + SchemaId, + )>, + temporary_oids: &HashSet, + ) -> Result<(), CatalogError> { + if sources.is_empty() { + return Ok(()); + } + + let amount = usize_to_u64(sources.len()); + let oids = self.allocate_oids(amount, temporary_oids)?; + let sources: Vec<_> = sources + .into_iter() + .zip_eq(oids) + .map( + |((cluster_id, replica_id, name, item_id, global_id, schema_id), oid)| { + PersistedIntrospectionSource { + cluster_id, + replica_id, + name, + item_id, + global_id, + schema_id, + oid, + } + }, + ) + .collect(); + + for source in sources { + let (key, value) = source.into_key_value(); + self.persisted_introspection_sources + .insert(key, value, self.op_id)?; + } + + Ok(()) + } + /// Set persisted system object mappings. pub fn set_system_object_mappings( &mut self, @@ -2212,6 +2394,19 @@ impl<'a> Transaction<'a> { .collect() } + pub fn get_persisted_introspection_sources( + &self, + cluster_id: ClusterId, + replica_id: ReplicaId, + ) -> BTreeMap<&str, (GlobalId, u32)> { + self.persisted_introspection_sources + .items() + .into_iter() + .filter(|(k, _v)| k.cluster_id == cluster_id && k.replica_id == replica_id) + .map(|(k, v)| (k.name.as_str(), (v.global_id.into(), v.oid))) + .collect() + } + pub fn get_catalog_content_version(&self) -> Option<&str> { self.settings .get(&SettingKey { @@ -2303,6 +2498,7 @@ impl<'a> Transaction<'a> { network_policies, cluster_replicas, introspection_sources, + persisted_introspection_sources, system_gid_mapping, system_configurations, default_privileges, @@ -2371,11 +2567,18 @@ impl<'a> Transaction<'a> { StateUpdateKind::IntrospectionSourceIndex, self.op_id, )) + // Cluster replicas must be applied before persisted introspection + // sources, because the latter look up the replica during creation. .chain(get_collection_op_updates( cluster_replicas, StateUpdateKind::ClusterReplica, self.op_id, )) + .chain(get_collection_op_updates( + persisted_introspection_sources, + StateUpdateKind::PersistedIntrospectionSource, + self.op_id, + )) .chain(get_collection_op_updates( system_gid_mapping, StateUpdateKind::SystemObjectMapping, @@ -2455,6 +2658,7 @@ impl<'a> Transaction<'a> { cluster_replicas: self.cluster_replicas.pending(), network_policies: self.network_policies.pending(), introspection_sources: self.introspection_sources.pending(), + persisted_introspection_sources: self.persisted_introspection_sources.pending(), id_allocator: self.id_allocator.pending(), configs: self.configs.pending(), source_references: self.source_references.pending(), @@ -2500,6 +2704,7 @@ impl<'a> Transaction<'a> { cluster_replicas, network_policies, introspection_sources, + persisted_introspection_sources, id_allocator, configs, source_references, @@ -2526,6 +2731,7 @@ impl<'a> Transaction<'a> { differential_dataflow::consolidation::consolidate_updates(cluster_replicas); differential_dataflow::consolidation::consolidate_updates(network_policies); differential_dataflow::consolidation::consolidate_updates(introspection_sources); + differential_dataflow::consolidation::consolidate_updates(persisted_introspection_sources); differential_dataflow::consolidation::consolidate_updates(id_allocator); differential_dataflow::consolidation::consolidate_updates(configs); differential_dataflow::consolidation::consolidate_updates(settings); @@ -2723,6 +2929,11 @@ pub struct TransactionBatch { proto::ClusterIntrospectionSourceIndexValue, Diff, )>, + pub(crate) persisted_introspection_sources: Vec<( + proto::PersistedIntrospectionSourceKey, + proto::PersistedIntrospectionSourceValue, + Diff, + )>, pub(crate) id_allocator: Vec<(proto::IdAllocKey, proto::IdAllocValue, Diff)>, pub(crate) configs: Vec<(proto::ConfigKey, proto::ConfigValue, Diff)>, pub(crate) settings: Vec<(proto::SettingKey, proto::SettingValue, Diff)>, @@ -2772,6 +2983,7 @@ impl TransactionBatch { cluster_replicas, network_policies, introspection_sources, + persisted_introspection_sources, id_allocator, configs, settings, @@ -2796,6 +3008,7 @@ impl TransactionBatch { && cluster_replicas.is_empty() && network_policies.is_empty() && introspection_sources.is_empty() + && persisted_introspection_sources.is_empty() && id_allocator.is_empty() && configs.is_empty() && settings.is_empty() @@ -2874,6 +3087,7 @@ mod unique_name { DefaultPrivilegesValue, GidMappingValue, IdAllocValue, + PersistedIntrospectionSourceValue, ServerConfigurationValue, SettingValue, SourceReferencesValue, diff --git a/src/catalog/src/durable/upgrade.rs b/src/catalog/src/durable/upgrade.rs index 55cc40c2541a7..c213cdf386e7f 100644 --- a/src/catalog/src/durable/upgrade.rs +++ b/src/catalog/src/durable/upgrade.rs @@ -241,7 +241,7 @@ macro_rules! objects { } } -objects!([v74, v75, v76, v77, v78], [v79, v80]); +objects!([v74, v75, v76, v77, v78], [v79, v80, v81]); /// The current version of the `Catalog`. pub use mz_catalog_protos::CATALOG_VERSION; @@ -259,6 +259,7 @@ mod v76_to_v77; mod v77_to_v78; mod v78_to_v79; mod v79_to_v80; +mod v80_to_v81; /// Describes a single action to take during a migration from `V1` to `V2`. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)] @@ -395,6 +396,15 @@ async fn run_upgrade( ) .await } + 80 => { + run_versioned_upgrade( + unopened_catalog_state, + version, + commit_ts, + v80_to_v81::upgrade, + ) + .await + } // Up-to-date, no migration needed! CATALOG_VERSION => Ok((CATALOG_VERSION, commit_ts)), diff --git a/src/catalog/src/durable/upgrade/snapshots/objects_v81.txt b/src/catalog/src/durable/upgrade/snapshots/objects_v81.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/src/catalog/src/durable/upgrade/v80_to_v81.rs b/src/catalog/src/durable/upgrade/v80_to_v81.rs new file mode 100644 index 0000000000000..38da4d307c017 --- /dev/null +++ b/src/catalog/src/durable/upgrade/v80_to_v81.rs @@ -0,0 +1,21 @@ +// Copyright Materialize, Inc. and contributors. All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use crate::durable::upgrade::MigrationAction; +use crate::durable::upgrade::objects_v80 as v80; +use crate::durable::upgrade::objects_v81 as v81; + +/// No-op migration. All types are JSON-compatible between v80 and v81. +/// The only change is adding the `PersistedIntrospectionSource` state +/// update kind variant and related types. +pub fn upgrade( + _snapshot: Vec, +) -> Vec> { + Vec::new() +} diff --git a/src/catalog/src/memory/objects.rs b/src/catalog/src/memory/objects.rs index 26183da326fe6..558f0d1931df7 100644 --- a/src/catalog/src/memory/objects.rs +++ b/src/catalog/src/memory/objects.rs @@ -409,6 +409,7 @@ impl Cluster { replication_factor, optimizer_feature_overrides, schedule, + persist_introspection, }) => { let introspection = match logging { ReplicaLogging { @@ -431,6 +432,7 @@ impl Cluster { compute, optimizer_feature_overrides: optimizer_feature_overrides.clone(), schedule: schedule.clone(), + persist_introspection: *persist_introspection, }) } ClusterVariant::Unmanaged => { @@ -980,6 +982,10 @@ pub enum DataSourceDesc { }, /// Receives introspection data from an internal system Introspection(IntrospectionType), + /// Receives data from a persist shard written by a compute replica's + /// introspection logging. Unlike `Introspection`, these sources are + /// per-replica, durable, and queryable via SQL. + PersistedIntrospection(LogVariant), /// Receives data from the source's reclocking/remapping operations. Progress, /// Receives data from HTTP requests. @@ -1017,6 +1023,7 @@ impl DataSourceDesc { None => (None, None), }, DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) | DataSourceDesc::Webhook { .. } | DataSourceDesc::Progress => (None, None), } @@ -1064,6 +1071,7 @@ impl DataSourceDesc { Some(envelope_string(&data_config.envelope)) } DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) | DataSourceDesc::Webhook { .. } | DataSourceDesc::Progress => None, } @@ -1194,7 +1202,9 @@ impl Source { | DataSourceDesc::OldSyntaxIngestion { desc, .. } => desc.connection.name(), DataSourceDesc::Progress => "progress", DataSourceDesc::IngestionExport { .. } => "subsource", - DataSourceDesc::Introspection(_) => "source", + DataSourceDesc::Introspection(_) | DataSourceDesc::PersistedIntrospection(_) => { + "source" + } DataSourceDesc::Webhook { .. } => "webhook", } } @@ -1206,6 +1216,7 @@ impl Source { | DataSourceDesc::OldSyntaxIngestion { desc, .. } => desc.connection.connection_id(), DataSourceDesc::IngestionExport { .. } | DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) | DataSourceDesc::Webhook { .. } | DataSourceDesc::Progress => None, } @@ -1253,7 +1264,9 @@ impl Source { DataSourceDesc::Webhook { .. } => 1, // Introspection and progress subsources are not under the user's control, so shouldn't // count toward their quota. - DataSourceDesc::Introspection(_) | DataSourceDesc::Progress => 0, + DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) + | DataSourceDesc::Progress => 0, } } } @@ -1823,6 +1836,7 @@ impl CatalogItem { | DataSourceDesc::OldSyntaxIngestion { desc, .. } => Ok(Some(desc)), DataSourceDesc::IngestionExport { .. } | DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) | DataSourceDesc::Webhook { .. } | DataSourceDesc::Progress => Ok(None), }, @@ -2382,7 +2396,9 @@ impl CatalogItem { // cross-referencing the items DataSourceDesc::IngestionExport { .. } => None, DataSourceDesc::Webhook { cluster_id, .. } => Some(*cluster_id), - DataSourceDesc::Introspection(_) | DataSourceDesc::Progress => None, + DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) + | DataSourceDesc::Progress => None, }, CatalogItem::Sink(sink) => Some(sink.cluster_id), CatalogItem::ContinualTask(ct) => Some(ct.cluster_id), @@ -2795,6 +2811,7 @@ impl CatalogEntry { } => Some(*progress_subsource), DataSourceDesc::IngestionExport { .. } | DataSourceDesc::Introspection(_) + | DataSourceDesc::PersistedIntrospection(_) | DataSourceDesc::Progress | DataSourceDesc::Webhook { .. } => None, }, @@ -3250,6 +3267,7 @@ pub struct ClusterVariantManaged { pub size: String, pub availability_zones: Vec, pub logging: ReplicaLogging, + pub persist_introspection: bool, pub replication_factor: u32, pub optimizer_feature_overrides: OptimizerFeatureOverrides, pub schedule: ClusterSchedule, @@ -3261,6 +3279,7 @@ impl From for durable::ClusterVariantManaged { size: managed.size, availability_zones: managed.availability_zones, logging: managed.logging, + persist_introspection: managed.persist_introspection, replication_factor: managed.replication_factor, optimizer_feature_overrides: managed.optimizer_feature_overrides.into(), schedule: managed.schedule, @@ -3274,6 +3293,7 @@ impl From for ClusterVariantManaged { size: managed.size, availability_zones: managed.availability_zones, logging: managed.logging, + persist_introspection: managed.persist_introspection, replication_factor: managed.replication_factor, optimizer_feature_overrides: managed.optimizer_feature_overrides.into(), schedule: managed.schedule, @@ -3684,7 +3704,10 @@ pub enum StateUpdateKind { Cluster(durable::objects::Cluster), NetworkPolicy(durable::objects::NetworkPolicy), IntrospectionSourceIndex(durable::objects::IntrospectionSourceIndex), + // ClusterReplica must come before PersistedIntrospectionSource because + // the latter depends on the replica existing during catalog apply. ClusterReplica(durable::objects::ClusterReplica), + PersistedIntrospectionSource(durable::objects::PersistedIntrospectionSource), SourceReferences(durable::objects::SourceReferences), SystemObjectMapping(durable::objects::SystemObjectMapping), // Temporary items are not actually updated via the durable catalog, but @@ -3780,7 +3803,10 @@ pub enum BootstrapStateUpdateKind { Cluster(durable::objects::Cluster), NetworkPolicy(durable::objects::NetworkPolicy), IntrospectionSourceIndex(durable::objects::IntrospectionSourceIndex), + // ClusterReplica must come before PersistedIntrospectionSource because + // the latter depends on the replica existing during catalog apply. ClusterReplica(durable::objects::ClusterReplica), + PersistedIntrospectionSource(durable::objects::PersistedIntrospectionSource), SourceReferences(durable::objects::SourceReferences), SystemObjectMapping(durable::objects::SystemObjectMapping), Item(durable::objects::Item), @@ -3815,6 +3841,9 @@ impl From for StateUpdateKind { BootstrapStateUpdateKind::IntrospectionSourceIndex(kind) => { StateUpdateKind::IntrospectionSourceIndex(kind) } + BootstrapStateUpdateKind::PersistedIntrospectionSource(kind) => { + StateUpdateKind::PersistedIntrospectionSource(kind) + } BootstrapStateUpdateKind::ClusterReplica(kind) => StateUpdateKind::ClusterReplica(kind), BootstrapStateUpdateKind::SystemObjectMapping(kind) => { StateUpdateKind::SystemObjectMapping(kind) @@ -3857,6 +3886,9 @@ impl TryFrom for BootstrapStateUpdateKind { StateUpdateKind::IntrospectionSourceIndex(kind) => { Ok(BootstrapStateUpdateKind::IntrospectionSourceIndex(kind)) } + StateUpdateKind::PersistedIntrospectionSource(kind) => { + Ok(BootstrapStateUpdateKind::PersistedIntrospectionSource(kind)) + } StateUpdateKind::ClusterReplica(kind) => { Ok(BootstrapStateUpdateKind::ClusterReplica(kind)) } diff --git a/src/catalog/tests/debug.rs b/src/catalog/tests/debug.rs index d773b9125cdcc..000eaaeeca737 100644 --- a/src/catalog/tests/debug.rs +++ b/src/catalog/tests/debug.rs @@ -99,6 +99,7 @@ impl Debug for StableTrace<'_> { id_allocator, items, network_policies, + persisted_introspection_sources, roles, role_auth, schemas, @@ -143,6 +144,10 @@ impl Debug for StableTrace<'_> { .field("id_allocator", id_allocator) .field("items", items) .field("network_policies", network_policies) + .field( + "persisted_introspection_sources", + persisted_introspection_sources, + ) .field("roles", roles) .field("role_auth", role_auth) .field("schemas", schemas) diff --git a/src/catalog/tests/open.rs b/src/catalog/tests/open.rs index cc99b6ddf80ca..aeb714b485bef 100644 --- a/src/catalog/tests/open.rs +++ b/src/catalog/tests/open.rs @@ -89,6 +89,7 @@ impl Debug for StableSnapshot<'_> { network_policies, cluster_replicas, introspection_sources, + persisted_introspection_sources, id_allocator, configs, settings, @@ -118,6 +119,10 @@ impl Debug for StableSnapshot<'_> { .field("network_policies", network_policies) .field("cluster_replicas", cluster_replicas) .field("introspection_sources", introspection_sources) + .field( + "persisted_introspection_sources", + persisted_introspection_sources, + ) .field("id_allocator", id_allocator) .field("configs", &configs) .field("settings", &settings) diff --git a/src/catalog/tests/snapshots/debug__opened_trace.snap b/src/catalog/tests/snapshots/debug__opened_trace.snap index 68c433bcf29f6..9fc566105f204 100644 --- a/src/catalog/tests/snapshots/debug__opened_trace.snap +++ b/src/catalog/tests/snapshots/debug__opened_trace.snap @@ -1,6 +1,5 @@ --- source: src/catalog/tests/debug.rs -assertion_line: 233 expression: test_trace --- Trace { @@ -447,6 +446,7 @@ Trace { }, optimizer_feature_overrides: [], schedule: Manual, + persist_introspection: false, }, ), }, @@ -910,6 +910,9 @@ Trace { ), ], }, + persisted_introspection_sources: CollectionTrace { + values: [], + }, roles: CollectionTrace { values: [ ( diff --git a/src/catalog/tests/snapshots/open__initial_snapshot.snap b/src/catalog/tests/snapshots/open__initial_snapshot.snap index e7b38518d2279..d14b2523f0593 100644 --- a/src/catalog/tests/snapshots/open__initial_snapshot.snap +++ b/src/catalog/tests/snapshots/open__initial_snapshot.snap @@ -1,6 +1,5 @@ --- source: src/catalog/tests/open.rs -assertion_line: 518 expression: test_snapshot --- Snapshot { @@ -598,6 +597,7 @@ Snapshot { }, optimizer_feature_overrides: [], schedule: Manual, + persist_introspection: false, }, ), }, @@ -673,6 +673,7 @@ Snapshot { }, }, introspection_sources: {}, + persisted_introspection_sources: {}, id_allocator: { IdAllocKey { name: "auditlog", diff --git a/src/compute-client/src/controller.rs b/src/compute-client/src/controller.rs index 94a5801edf19e..34de5345b0cd3 100644 --- a/src/compute-client/src/controller.rs +++ b/src/compute-client/src/controller.rs @@ -55,6 +55,7 @@ use mz_ore::tracing::OpenTelemetryContext; use mz_persist_types::PersistLocation; use mz_repr::{Datum, GlobalId, RelationDesc, Row, TimestampManipulation}; use mz_storage_client::controller::StorageController; +use mz_storage_types::controller::CollectionMetadata; use mz_storage_types::dyncfgs::ORE_OVERFLOWING_BEHAVIOR; use mz_storage_types::read_holds::ReadHold; use mz_storage_types::read_policy::ReadPolicy; @@ -334,6 +335,11 @@ impl ComputeController { } } + /// Returns the persist location used by this compute controller. + pub fn persist_location(&self) -> &PersistLocation { + &self.peek_stash_persist_location + } + /// TODO(database-issues#7533): Add documentation. pub fn instance_exists(&self, id: ComputeInstanceId) -> bool { self.instances.contains_key(&id) @@ -691,6 +697,7 @@ where replica_id: ReplicaId, location: ClusterReplicaLocation, config: ComputeReplicaConfig, + sink_logs: BTreeMap, ) -> Result<(), ReplicaCreationError> { use ReplicaCreationError::*; @@ -715,6 +722,7 @@ where enable_logging, log_logging: config.logging.log_logging, index_logs: Default::default(), + sink_logs, }, grpc_client: self.config.grpc_client.clone(), expiration_offset: (!expiration_offset.is_zero()).then_some(expiration_offset), @@ -732,6 +740,10 @@ where } /// Removes a replica from an instance, including its service in the orchestrator. + /// + /// Shard cleanup for persist-backed introspection is handled by the catalog + /// transaction (`Op::DropObjects` → `storage_collections_to_drop` → + /// `unfinalized_shards`), not by the caller. pub fn drop_replica( &mut self, instance_id: ComputeInstanceId, @@ -748,7 +760,9 @@ where instance.replicas.remove(&replica_id); - instance.call(move |i| i.remove_replica(replica_id).expect("validated")); + instance.call(move |i| { + let _ = i.remove_replica(replica_id).expect("validated"); + }); Ok(()) } diff --git a/src/compute-client/src/controller/instance.rs b/src/compute-client/src/controller/instance.rs index cc3e31c7442a0..2fa97fb0c36b4 100644 --- a/src/compute-client/src/controller/instance.rs +++ b/src/compute-client/src/controller/instance.rs @@ -38,6 +38,7 @@ use mz_repr::adt::timestamp::CheckedTimestamp; use mz_repr::refresh_schedule::RefreshSchedule; use mz_repr::{Datum, Diff, GlobalId, RelationDesc, Row}; use mz_storage_client::controller::{IntrospectionType, WallclockLag, WallclockLagHistogramPeriod}; +use mz_storage_types::controller::CollectionMetadata; use mz_storage_types::read_holds::{self, ReadHold}; use mz_storage_types::read_policy::ReadPolicy; use thiserror::Error; @@ -939,6 +940,7 @@ where let instance_config = InstanceConfig { peek_stash_persist_location: self.peek_stash_persist_location.clone(), + read_only: self.read_only, // The remaining fields are replica-specific and will be set in // `ReplicaTask::specialize_command`. logging: Default::default(), @@ -1120,8 +1122,12 @@ where /// Remove an existing instance replica, by ID. #[mz_ore::instrument(level = "debug")] - pub fn remove_replica(&mut self, id: ReplicaId) -> Result<(), ReplicaMissing> { - self.replicas.remove(&id).ok_or(ReplicaMissing(id))?; + pub fn remove_replica( + &mut self, + id: ReplicaId, + ) -> Result, ReplicaMissing> { + let replica = self.replicas.remove(&id).ok_or(ReplicaMissing(id))?; + let sink_logs = replica.config.logging.sink_logs.clone(); // Subscribes targeting this replica either won't be served anymore (if the replica is // dropped) or might produce inconsistent output (if the target collection is an @@ -1166,7 +1172,7 @@ where // up the next replica, if the dropped replica was the only one in the cluster. self.forward_implied_capabilities(); - Ok(()) + Ok(sink_logs) } /// Rehydrate the given instance replica. @@ -1862,6 +1868,11 @@ where frontiers: FrontiersResponse, replica_id: ReplicaId, ) { + // Persisted introspection sinks are managed by the compute replica + // directly. The controller does not track their frontiers. + if matches!(id, GlobalId::PersistedIntrospectionSource(_)) { + return; + } if !self.collections.contains_key(&id) { soft_panic_or_log!( "frontiers update for an unknown collection \ diff --git a/src/compute-client/src/logging.rs b/src/compute-client/src/logging.rs index f7279e2c2f955..8d9676269ea32 100644 --- a/src/compute-client/src/logging.rs +++ b/src/compute-client/src/logging.rs @@ -13,6 +13,7 @@ use std::collections::BTreeMap; use std::time::Duration; use mz_repr::{GlobalId, RelationDesc, SqlScalarType}; +use mz_storage_types::controller::CollectionMetadata; use serde::{Deserialize, Serialize}; /// Logging configuration. @@ -32,6 +33,9 @@ pub struct LoggingConfig { pub log_logging: bool, /// Logs to keep in an arrangement pub index_logs: BTreeMap, + /// Logs to write to persist shards. + /// Maps each log variant to a (GlobalId, CollectionMetadata) identifying the target persist shard. + pub sink_logs: BTreeMap, } /// TODO(database-issues#7533): Add documentation. diff --git a/src/compute-client/src/protocol/command.rs b/src/compute-client/src/protocol/command.rs index 59b13ee6ff4a4..39f615298e6fa 100644 --- a/src/compute-client/src/protocol/command.rs +++ b/src/compute-client/src/protocol/command.rs @@ -275,6 +275,11 @@ pub struct InstanceConfig { pub expiration_offset: Option, /// The persist location where we can stash large peek results. pub peek_stash_persist_location: PersistLocation, + /// Whether this instance is in read-only mode. + /// + /// When read-only, logging persist sinks do not write to persist shards. + /// The instance transitions to writable on the first `AllowWrites` command. + pub read_only: bool, } impl InstanceConfig { @@ -291,11 +296,14 @@ impl InstanceConfig { logging: self_logging, expiration_offset: self_offset, peek_stash_persist_location: self_peek_stash_persist_location, + // read_only transitions are handled via AllowWrites, no compatibility check needed. + read_only: _, } = self; let InstanceConfig { logging: other_logging, expiration_offset: other_offset, peek_stash_persist_location: other_peek_stash_persist_location, + read_only: _, } = other; // Logging is compatible if exactly the same. diff --git a/src/compute-types/src/dyncfgs.rs b/src/compute-types/src/dyncfgs.rs index 2d8970b738595..e5d653bab2500 100644 --- a/src/compute-types/src/dyncfgs.rs +++ b/src/compute-types/src/dyncfgs.rs @@ -327,6 +327,17 @@ pub const PEEK_STASH_BATCH_SIZE: Config = Config::new( "The size, as number of rows, of each batch pumped from the peek result iterator (in one iteration through the worker loop) when stashing peek responses.", ); +/// Global kill switch for persist-backed introspection sources. +/// +/// When disabled, compute replicas will not write introspection data to persist, +/// even if the cluster has `PERSIST INTROSPECTION = true`. +pub const ENABLE_PERSIST_INTROSPECTION: Config = Config::new( + "enable_persist_introspection", + true, + "Global kill switch for persist-backed introspection sources. When disabled, compute \ + replicas will not write introspection data to persist shards.", +); + /// If set, skip fetching or processing the snapshot data for subscribes when possible. pub const SUBSCRIBE_SNAPSHOT_OPTIMIZATION: Config = Config::new( "compute_subscribe_snapshot_optimization", @@ -376,5 +387,6 @@ pub fn all_dyncfgs(configs: ConfigSet) -> ConfigSet { .add(&PEEK_RESPONSE_STASH_READ_MEMORY_BUDGET_BYTES) .add(&PEEK_STASH_NUM_BATCHES) .add(&PEEK_STASH_BATCH_SIZE) + .add(&ENABLE_PERSIST_INTROSPECTION) .add(&SUBSCRIBE_SNAPSHOT_OPTIMIZATION) } diff --git a/src/compute/src/compute_state.rs b/src/compute/src/compute_state.rs index 8be39a034707d..7bc01b7d44847 100644 --- a/src/compute/src/compute_state.rs +++ b/src/compute/src/compute_state.rs @@ -163,6 +163,20 @@ pub struct ComputeState { /// replica can drop diffs associated with timestamps beyond the replica expiration. /// The replica will panic if such dataflows are not dropped before the replica has expired. pub replica_expiration: Antichain, + + /// Tokens keeping logging persist sinks alive. + /// Dropped when the replica is shut down. + pub logging_persist_tokens: Vec>, + + /// Replica-level read-only signal for logging persist sinks. + /// + /// Logging persist sinks are not tracked as compute collections by the controller, + /// so they cannot use the per-collection `AllowWrites` mechanism. Instead, they + /// share this replica-level signal that starts as read-only (`true`) and flips to + /// writable (`false`) on the first `AllowWrites` command from the controller. + read_only_tx: watch::Sender, + /// Receiver for the replica-level read-only signal. Cloned into each logging persist sink. + pub read_only_rx: watch::Receiver, } impl ComputeState { @@ -177,6 +191,9 @@ impl ComputeState { let traces = TraceManager::new(metrics.clone()); let command_history = ComputeCommandHistory::new(metrics.for_history()); + // Start read-only; flipped to writable on first AllowWrites command. + let (read_only_tx, read_only_rx) = watch::channel(true); + Self { collections: Default::default(), traces, @@ -198,6 +215,9 @@ impl ComputeState { server_maintenance_interval: Duration::ZERO, init_system_time: mz_ore::now::SYSTEM_TIME(), replica_expiration: Antichain::default(), + logging_persist_tokens: Vec::new(), + read_only_tx, + read_only_rx, } } @@ -408,6 +428,12 @@ impl<'a, A: Allocate + 'static> ActiveComputeState<'a, A> { self.compute_state.apply_expiration_offset(offset); } + // Set the replica-level read-only state from the controller before + // initializing logging, so logging persist sinks pick up the correct value. + self.compute_state + .read_only_tx + .send_replace(config.read_only); + self.initialize_logging(config.logging); self.compute_state.peek_stash_persist_location = Some(config.peek_stash_persist_location); @@ -614,6 +640,17 @@ impl<'a, A: Allocate + 'static> ActiveComputeState<'a, A> { // such as appending a batch or advancing the upper. self.compute_state.persist_clients.cfg().enable_compaction(); + // On the first AllowWrites command, also flip the replica-level read-only + // signal so logging persist sinks can start writing. + self.compute_state.read_only_tx.send_if_modified(|val| { + if *val { + *val = false; + true + } else { + false + } + }); + if let Some(collection) = self.compute_state.collections.get_mut(&id) { collection.allow_writes(); } else { @@ -666,11 +703,28 @@ impl<'a, A: Allocate + 'static> ActiveComputeState<'a, A> { panic!("dataflow server has already initialized logging"); } + // Pre-create collection entries for persist sink IDs. + // The persist_sink function expects these to exist when it sets sink_write_frontier. + // We use a placeholder dataflow index (0) and update it after the dataflow is created. + for (_variant, (sink_id, _meta)) in config.sink_logs.iter() { + let is_subscribe_or_copy = false; + let as_of = Antichain::from_elem(Timestamp::MIN); + let metrics = self.compute_state.metrics.for_collection(*sink_id); + let collection = CollectionState::new( + Rc::new(0), // placeholder, updated below + is_subscribe_or_copy, + as_of, + metrics, + ); + self.compute_state.collections.insert(*sink_id, collection); + } + let LoggingTraces { traces, dataflow_index, compute_logger: logger, - } = logging::initialize(self.timely_worker, &config); + persist_tokens, + } = logging::initialize(self.timely_worker, &config, self.compute_state); let dataflow_index = Rc::new(dataflow_index); let mut log_index_ids = config.index_logs; @@ -705,6 +759,9 @@ impl<'a, A: Allocate + 'static> ActiveComputeState<'a, A> { } } + // Hold persist tokens to keep the sinks alive. + self.compute_state.logging_persist_tokens = persist_tokens; + // Sanity check. assert!( log_index_ids.is_empty(), diff --git a/src/compute/src/logging.rs b/src/compute/src/logging.rs index c144063268b23..86ff4561f4e22 100644 --- a/src/compute/src/logging.rs +++ b/src/compute/src/logging.rs @@ -12,6 +12,7 @@ pub mod compute; mod differential; pub(super) mod initialize; +pub(super) mod persist; mod reachability; mod timely; diff --git a/src/compute/src/logging/compute.rs b/src/compute/src/logging/compute.rs index 3088414cef17f..34a8c82254cea 100644 --- a/src/compute/src/logging/compute.rs +++ b/src/compute/src/logging/compute.rs @@ -290,6 +290,8 @@ impl LirMetadata { pub(super) struct Return { /// Collections returned by [`construct`]. pub collections: BTreeMap, + /// Tokens keeping persist sinks alive. + pub persist_tokens: Vec>, } /// Constructs the logging dataflow fragment for compute logs. @@ -307,6 +309,7 @@ pub(super) fn construct> config: &mz_compute_client::logging::LoggingConfig, event_queue: EventQueue>, shared_state: Rc>, + compute_state: &mut crate::compute_state::ComputeState, ) -> Return { let logging_interval_ms = std::cmp::max(1, config.interval.as_millis()); @@ -425,8 +428,9 @@ pub(super) fn construct> (PeekDuration, peek_duration), ]; - // Build the output arrangements. + // Build the output arrangements and persist sinks. let mut collections = BTreeMap::new(); + let mut persist_tokens: Vec> = Vec::new(); for (variant, stream) in logs { let variant = LogVariant::Compute(variant); if config.index_logs.contains_key(&variant) { @@ -447,9 +451,22 @@ pub(super) fn construct> }; collections.insert(variant, collection); } + if let Some((sink_id, meta)) = config.sink_logs.get(&variant) { + let token = super::persist::render_arranged( + &stream, + variant, + *sink_id, + meta, + compute_state, + ); + persist_tokens.push(token); + } } - Return { collections } + Return { + collections, + persist_tokens, + } }) } diff --git a/src/compute/src/logging/differential.rs b/src/compute/src/logging/differential.rs index 40b8980b77b2f..b5cf290938aa2 100644 --- a/src/compute/src/logging/differential.rs +++ b/src/compute/src/logging/differential.rs @@ -43,6 +43,8 @@ use crate::typedefs::{KeyBatcher, RowRowSpine}; pub(super) struct Return { /// Collections to export. pub collections: BTreeMap, + /// Tokens keeping persist sinks alive. + pub persist_tokens: Vec>, } /// Constructs the logging dataflow fragment for differential logs. @@ -57,6 +59,7 @@ pub(super) fn construct>( config: &mz_compute_client::logging::LoggingConfig, event_queue: EventQueue>, shared_state: Rc>, + compute_state: &mut crate::compute_state::ComputeState, ) -> Return { let logging_interval_ms = std::cmp::max(1, config.interval.as_millis()); @@ -170,8 +173,9 @@ pub(super) fn construct>( (BatcherAllocations, batcher_allocations), ]; - // Build the output arrangements. + // Build the output arrangements and persist sinks. let mut collections = BTreeMap::new(); + let mut persist_tokens: Vec> = Vec::new(); for (variant, collection) in logs { let variant = LogVariant::Differential(variant); if config.index_logs.contains_key(&variant) { @@ -192,9 +196,22 @@ pub(super) fn construct>( }; collections.insert(variant, collection); } + if let Some((sink_id, meta)) = config.sink_logs.get(&variant) { + let token = super::persist::render_arranged( + &collection, + variant, + *sink_id, + meta, + compute_state, + ); + persist_tokens.push(token); + } } - Return { collections } + Return { + collections, + persist_tokens, + } }) } diff --git a/src/compute/src/logging/initialize.rs b/src/compute/src/logging/initialize.rs index 735890652829d..ae93adbecf597 100644 --- a/src/compute/src/logging/initialize.rs +++ b/src/compute/src/logging/initialize.rs @@ -5,6 +5,7 @@ //! Initialization of logging dataflows. +use std::any::Any; use std::cell::RefCell; use std::collections::BTreeMap; use std::rc::Rc; @@ -32,6 +33,7 @@ use timely::progress::reachability::logging::{TrackerEvent, TrackerEventBuilder} use timely::worker::AsWorker; use crate::arrangement::manager::TraceBundle; +use crate::compute_state::ComputeState; use crate::extensions::arrange::{KeyCollection, MzArrange}; use crate::logging::compute::{ComputeEvent, ComputeEventBuilder}; use crate::logging::{BatchLogger, EventQueue, SharedLoggingState}; @@ -44,6 +46,7 @@ use crate::typedefs::{ErrBatcher, ErrBuilder}; pub fn initialize( worker: &mut timely::worker::Worker, config: &LoggingConfig, + compute_state: &mut ComputeState, ) -> LoggingTraces { let interval_ms = std::cmp::max(1, config.interval.as_millis()); @@ -58,6 +61,7 @@ pub fn initialize( let mut context = LoggingContext { worker, config, + compute_state, interval_ms, now, start_offset, @@ -71,13 +75,13 @@ pub fn initialize( // Depending on whether we should log the creation of the logging dataflows, we register the // loggers with timely either before or after creating them. let dataflow_index = context.worker.next_dataflow_index(); - let traces = if config.log_logging { + let (traces, persist_tokens) = if config.log_logging { context.register_loggers(); context.construct_dataflow() } else { - let traces = context.construct_dataflow(); + let result = context.construct_dataflow(); context.register_loggers(); - traces + result }; let compute_logger = worker.logger_for("materialize/compute").unwrap(); @@ -85,6 +89,7 @@ pub fn initialize( traces, dataflow_index, compute_logger, + persist_tokens, } } @@ -93,6 +98,7 @@ pub(super) type ReachabilityEvent = (usize, Vec<(usize, usize, bool, Timestamp, struct LoggingContext<'a, A: Allocate> { worker: &'a mut timely::worker::Worker, config: &'a LoggingConfig, + compute_state: &'a mut ComputeState, interval_ms: u128, now: Instant, start_offset: Duration, @@ -110,54 +116,71 @@ pub(crate) struct LoggingTraces { pub dataflow_index: usize, /// The compute logger. pub compute_logger: super::compute::Logger, + /// Tokens keeping persist sinks alive. + pub persist_tokens: Vec>, } impl LoggingContext<'_, A> { - fn construct_dataflow(&mut self) -> BTreeMap { + fn construct_dataflow(&mut self) -> (BTreeMap, Vec>) { + let compute_state = &mut *self.compute_state; + self.worker.dataflow_named("Dataflow: logging", |scope| { let scope = &mut scope.with_label(); let mut collections = BTreeMap::new(); + let mut persist_tokens: Vec> = Vec::new(); let super::timely::Return { collections: timely_collections, + persist_tokens: timely_persist_tokens, } = super::timely::construct( scope.clone(), self.config, self.t_event_queue.clone(), Rc::clone(&self.shared_state), + compute_state, ); collections.extend(timely_collections); + persist_tokens.extend(timely_persist_tokens); let super::reachability::Return { collections: reachability_collections, + persist_tokens: reachability_persist_tokens, } = super::reachability::construct( scope.clone(), self.config, self.r_event_queue.clone(), + compute_state, ); collections.extend(reachability_collections); + persist_tokens.extend(reachability_persist_tokens); let super::differential::Return { collections: differential_collections, + persist_tokens: differential_persist_tokens, } = super::differential::construct( scope.clone(), self.config, self.d_event_queue.clone(), Rc::clone(&self.shared_state), + compute_state, ); collections.extend(differential_collections); + persist_tokens.extend(differential_persist_tokens); let super::compute::Return { collections: compute_collections, + persist_tokens: compute_persist_tokens, } = super::compute::construct( scope.clone(), scope.parent().clone(), self.config, self.c_event_queue.clone(), Rc::clone(&self.shared_state), + compute_state, ); collections.extend(compute_collections); + persist_tokens.extend(compute_persist_tokens); let errs = scope.scoped("logging errors", |scope| { let collection: KeyCollection<_, DataflowError, Diff> = @@ -175,7 +198,7 @@ impl LoggingContext<'_, A> { (log, bundle) }) .collect(); - traces + (traces, persist_tokens) }) } diff --git a/src/compute/src/logging/persist.rs b/src/compute/src/logging/persist.rs new file mode 100644 index 0000000000000..ad2dd77583002 --- /dev/null +++ b/src/compute/src/logging/persist.rs @@ -0,0 +1,162 @@ +// 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 sink wrapper for logging collections. +//! +//! Renders a persist sink that writes logging collection data to a persist shard. +//! Reuses the MV persist sink for self-correction on replica restarts. + +use std::any::Any; +use std::rc::Rc; + +use columnar::Index; +use differential_dataflow::VecCollection; +use mz_compute_client::logging::LogVariant; +use mz_expr::{MirScalarExpr, permutation_for_arrangement}; +use mz_repr::{Diff, GlobalId, Row, RowRef, Timestamp}; +use mz_storage_types::controller::CollectionMetadata; +use mz_storage_types::errors::DataflowError; +use mz_timely_util::columnar::Column; +use mz_timely_util::operator::CollectionExt; +use timely::container::CapacityContainerBuilder; +use timely::dataflow::Scope; +use timely::dataflow::StreamCore; +use timely::dataflow::channels::pact::Pipeline; +use timely::dataflow::operators::Operator; +use timely::progress::frontier::Antichain; + +use crate::compute_state::ComputeState; +use crate::render::StartSignal; +use crate::sink::materialized_view; + +/// Render a persist sink for a logging collection. +/// +/// Takes a flat-row stream (already converted from key-value pairs) and writes it to the +/// specified persist shard using the MV persist sink for self-correction. +/// +/// Returns a token that keeps the sink alive and must be held as long as the sink should run. +pub(super) fn render( + sink_id: GlobalId, + target: &CollectionMetadata, + ok_collection: VecCollection, + compute_state: &mut ComputeState, +) -> Rc +where + S: Scope, +{ + // Empty error collection for logging (logging doesn't produce errors). + let err_collection: VecCollection = + VecCollection::empty(&ok_collection.scope()); + + // Start immediately - logging doesn't need dataflow suspension. + // Drop the suspension token to fire the start signal immediately. + let (start_signal, suspension_token) = StartSignal::new(); + drop(suspension_token); + + // Use the replica-level read-only signal. Logging persist sinks are not tracked + // as compute collections by the controller, so they cannot use the per-collection + // AllowWrites mechanism. Instead, they share this signal which flips to writable + // on the first AllowWrites command from the controller. + let read_only_rx = compute_state.read_only_rx.clone(); + + // Start from the beginning of time. + let as_of = Antichain::from_elem(Timestamp::MIN); + + materialized_view::persist_sink( + sink_id, + target, + ok_collection, + err_collection, + as_of, + compute_state, + start_signal, + read_only_rx, + ) +} + +/// Flatten an arranged key-value stream and render a persist sink for it. +/// +/// Combines the "Persist flatten" unary operator (which converts key-value pairs back to +/// flat rows using the arrangement permutation) with the persist sink rendering. +/// +/// Returns a token that keeps the sink alive and must be held as long as the sink should run. +pub(super) fn render_arranged( + stream: &StreamCore>, + variant: LogVariant, + sink_id: GlobalId, + target: &CollectionMetadata, + compute_state: &mut ComputeState, +) -> Rc +where + S: Scope, +{ + let (permutation, arity) = arrangement_permutation(variant); + let flat_stream = stream + .unary::>, _, _, _>( + Pipeline, + &format!("Persist flatten {variant:?}"), + |_cap, _info| { + move |input, output| { + input.for_each_time(|time, data| { + let mut session = output.session_with_builder(&time); + for ((key, value), ts, diff) in + data.flat_map(|c| c.borrow().into_index_iter()) + { + let flat_row = merge_kv_to_flat_row(key, value, &permutation, arity); + session.give((flat_row, ts, diff)); + } + }); + } + }, + ); + let ok_collection = differential_dataflow::Collection::new(flat_stream); + render(sink_id, target, ok_collection, compute_state) +} + +/// Compute the arrangement permutation for a log variant. +/// +/// Returns `(permutation, arity)` where `permutation[original_col]` gives the position +/// in the key++value concatenation for the datum at `original_col`. +pub(super) fn arrangement_permutation(variant: LogVariant) -> (Vec, usize) { + let desc = variant.desc(); + let arity = desc.arity(); + let key_indices = variant.index_by(); + let (permutation, _) = permutation_for_arrangement( + &key_indices + .iter() + .cloned() + .map(MirScalarExpr::column) + .collect::>(), + arity, + ); + (permutation, arity) +} + +/// Given key and value RowRefs and the arrangement permutation, reconstruct the flat row +/// in original column order. +/// +/// The `permutation` maps each original column index to its position in the concatenated +/// key++value representation. This function reverses that mapping. +pub(super) fn merge_kv_to_flat_row( + key: &RowRef, + value: &RowRef, + permutation: &[usize], + arity: usize, +) -> Row { + // Concatenate key and value datums. + let kv_datums: Vec<_> = key.iter().chain(value.iter()).collect(); + + // Re-order into original column order using the permutation. + let mut flat_row = Row::with_capacity(arity); + let mut packer = flat_row.packer(); + for original_col in 0..arity { + packer.push(kv_datums[permutation[original_col]]); + } + flat_row +} diff --git a/src/compute/src/logging/reachability.rs b/src/compute/src/logging/reachability.rs index b2bd918884531..b12138d67357b 100644 --- a/src/compute/src/logging/reachability.rs +++ b/src/compute/src/logging/reachability.rs @@ -14,6 +14,8 @@ use std::convert::TryInto; use std::rc::Rc; use std::time::Duration; +use std::any::Any; + use columnar::Index; use mz_compute_client::logging::LoggingConfig; use mz_ore::cast::CastFrom; @@ -36,6 +38,8 @@ use crate::typedefs::RowRowSpine; pub(super) struct Return { /// Collections to export. pub collections: BTreeMap, + /// Tokens keeping persist sinks alive. + pub persist_tokens: Vec>, } /// Constructs the logging dataflow fragment for reachability logs. @@ -48,8 +52,9 @@ pub(super) fn construct>( mut scope: G, config: &LoggingConfig, event_queue: EventQueue, 3>, + compute_state: &mut crate::compute_state::ComputeState, ) -> Return { - let collections = scope.scoped("timely reachability logging", move |scope| { + let (collections, persist_tokens) = scope.scoped("timely reachability logging", move |scope| { let enable_logging = config.enable_logging; let interval_ms = std::cmp::max(1, config.interval.as_millis()); type UpdatesKey = (bool, usize, usize, usize, Timestamp); @@ -117,6 +122,7 @@ pub(super) fn construct>( ); let mut result = BTreeMap::new(); + let mut persist_tokens: Vec> = Vec::new(); for variant in logs_active { if config.index_logs.contains_key(&variant) { let exchange = ExchangeCore::, _>::new_core( @@ -136,9 +142,22 @@ pub(super) fn construct>( }; result.insert(variant, collection); } + if let Some((sink_id, meta)) = config.sink_logs.get(&variant) { + let token = super::persist::render_arranged( + &updates, + variant, + *sink_id, + meta, + compute_state, + ); + persist_tokens.push(token); + } } - result + (result, persist_tokens) }); - Return { collections } + Return { + collections, + persist_tokens, + } } diff --git a/src/compute/src/logging/timely.rs b/src/compute/src/logging/timely.rs index b84e44b2c003b..9b663657f8f05 100644 --- a/src/compute/src/logging/timely.rs +++ b/src/compute/src/logging/timely.rs @@ -48,6 +48,8 @@ use crate::typedefs::{KeyBatcher, KeyValBatcher, RowRowSpine}; pub(super) struct Return { /// Collections to export. pub collections: BTreeMap, + /// Tokens keeping persist sinks alive. + pub persist_tokens: Vec>, } /// Constructs the logging dataflow fragment for timely logs. @@ -61,6 +63,7 @@ pub(super) fn construct>( config: &LoggingConfig, event_queue: EventQueue>, shared_state: Rc>, + compute_state: &mut crate::compute_state::ComputeState, ) -> Return { scope.scoped("timely logging", move |scope| { let enable_logging = config.enable_logging; @@ -349,15 +352,16 @@ pub(super) fn construct>( ] }; - // Build the output arrangements. + // Build the output arrangements and persist sinks. let mut collections = BTreeMap::new(); - for (variant, collection) in logs { - let variant = LogVariant::Timely(variant); + let mut persist_tokens: Vec> = Vec::new(); + for (variant, kv_stream) in &logs { + let variant = LogVariant::Timely(*variant); if config.index_logs.contains_key(&variant) { // Extract types to make rustfmt happy. type Batcher = Col2ValBatcher; type Builder = RowRowBuilder; - let trace = collection + let trace = kv_stream .mz_arrange_core::<_, Batcher<_, _, _, _>, Builder<_, _>, RowRowSpine<_, _>>( ExchangeCore::, _>::new_core( columnar_exchange::, @@ -365,15 +369,28 @@ pub(super) fn construct>( &format!("Arrange {variant:?}"), ) .trace; - let collection = LogCollection { + let log_collection = LogCollection { trace, token: Rc::clone(&token), }; - collections.insert(variant, collection); + collections.insert(variant, log_collection); + } + if let Some((sink_id, meta)) = config.sink_logs.get(&variant) { + let token = super::persist::render_arranged( + kv_stream, + variant, + *sink_id, + meta, + compute_state, + ); + persist_tokens.push(token); } } - Return { collections } + Return { + collections, + persist_tokens, + } }) } diff --git a/src/compute/src/metrics.rs b/src/compute/src/metrics.rs index 28a9d4ae2d2c7..e5375dd4bfa8f 100644 --- a/src/compute/src/metrics.rs +++ b/src/compute/src/metrics.rs @@ -474,7 +474,9 @@ impl CollectionMetrics { pub fn new(collection_id: GlobalId, metrics: WorkerMetrics) -> Self { let collection_type = match collection_id { GlobalId::System(_) => "system", - GlobalId::IntrospectionSourceIndex(_) => "log", + GlobalId::IntrospectionSourceIndex(_) | GlobalId::PersistedIntrospectionSource(_) => { + "log" + } GlobalId::User(_) => "user", GlobalId::Transient(_) => "transient", GlobalId::Explain => "explain", diff --git a/src/compute/src/sink.rs b/src/compute/src/sink.rs index 759874f6e52b5..4cc71ffd345fa 100644 --- a/src/compute/src/sink.rs +++ b/src/compute/src/sink.rs @@ -10,7 +10,7 @@ mod copy_to_s3_oneshot; mod correction; mod correction_v2; -mod materialized_view; +pub(crate) mod materialized_view; mod refresh; mod subscribe; diff --git a/src/compute/src/sink/materialized_view.rs b/src/compute/src/sink/materialized_view.rs index 471d3a7078c35..1f31229af9b0a 100644 --- a/src/compute/src/sink/materialized_view.rs +++ b/src/compute/src/sink/materialized_view.rs @@ -230,7 +230,7 @@ type BatchesStream = Stream; type SharedSinkFrontier = Rc>>; /// Renders an MV sink writing the given desired collection into the `target` persist collection. -pub(super) fn persist_sink( +pub(crate) fn persist_sink( sink_id: GlobalId, target: &CollectionMetadata, ok_collection: VecCollection, diff --git a/src/controller/src/clusters.rs b/src/controller/src/clusters.rs index de5752298b1f7..b7ef7f1f2127b 100644 --- a/src/controller/src/clusters.rs +++ b/src/controller/src/clusters.rs @@ -40,6 +40,7 @@ use mz_ore::task::{self, AbortOnDropHandle}; use mz_ore::{halt, instrument}; use mz_repr::GlobalId; use mz_repr::adt::numeric::Numeric; +use mz_storage_types::controller::CollectionMetadata; use regex::Regex; use serde::{Deserialize, Serialize}; use tokio::time; @@ -394,6 +395,9 @@ where /// Creates a replica of the specified cluster with the specified identifier /// and configuration. + /// + /// `sink_logs` specifies the persist shards to write logging data to. + /// If empty, no persist-backed introspection sinks are created. pub fn create_replica( &mut self, cluster_id: ClusterId, @@ -403,6 +407,7 @@ where role: ClusterRole, config: ReplicaConfig, enable_worker_core_affinity: bool, + sink_logs: BTreeMap, ) -> Result<(), anyhow::Error> { let storage_location: ClusterReplicaLocation; let compute_location: ClusterReplicaLocation; @@ -453,6 +458,7 @@ where replica_id, compute_location, config.compute, + sink_logs, )?; if let Some(task) = metrics_task { @@ -463,6 +469,9 @@ where } /// Drops the specified replica of the specified cluster. + /// + /// Shard cleanup for persist-backed introspection is handled by the catalog + /// transaction, not by the caller. pub fn drop_replica( &mut self, cluster_id: ClusterId, diff --git a/src/repr/src/catalog_item_id.proto b/src/repr/src/catalog_item_id.proto index 4645c37c05789..92a94e478483b 100644 --- a/src/repr/src/catalog_item_id.proto +++ b/src/repr/src/catalog_item_id.proto @@ -19,5 +19,6 @@ message ProtoCatalogItemId { uint64 user = 2; uint64 transient = 3; uint64 introspection_source_index = 4; + uint64 persisted_introspection_source = 5; } } diff --git a/src/repr/src/catalog_item_id.rs b/src/repr/src/catalog_item_id.rs index d0e4147e2db56..07e1845a8cdf0 100644 --- a/src/repr/src/catalog_item_id.rs +++ b/src/repr/src/catalog_item_id.rs @@ -38,6 +38,8 @@ pub enum CatalogItemId { System(u64), /// Introspection Source Index namespace. IntrospectionSourceIndex(u64), + /// Persisted introspection source namespace (per-replica persist shards). + PersistedIntrospectionSource(u64), /// User namespace. User(u64), /// Transient item. @@ -49,7 +51,9 @@ impl CatalogItemId { pub fn is_system(&self) -> bool { matches!( self, - CatalogItemId::System(_) | CatalogItemId::IntrospectionSourceIndex(_) + CatalogItemId::System(_) + | CatalogItemId::IntrospectionSourceIndex(_) + | CatalogItemId::PersistedIntrospectionSource(_) ) } @@ -75,7 +79,10 @@ impl FromStr for CatalogItemId { s = &s[1..]; let variant = match tag { 's' => { - if Some('i') == s.chars().next() { + if s.starts_with("ip") { + s = &s[2..]; + CatalogItemId::PersistedIntrospectionSource + } else if s.starts_with('i') { s = &s[1..]; CatalogItemId::IntrospectionSourceIndex } else { @@ -96,6 +103,7 @@ impl fmt::Display for CatalogItemId { match self { CatalogItemId::System(id) => write!(f, "s{}", id), CatalogItemId::IntrospectionSourceIndex(id) => write!(f, "si{}", id), + CatalogItemId::PersistedIntrospectionSource(id) => write!(f, "sip{}", id), CatalogItemId::User(id) => write!(f, "u{}", id), CatalogItemId::Transient(id) => write!(f, "t{}", id), } @@ -109,6 +117,7 @@ impl RustType for CatalogItemId { kind: Some(match self { CatalogItemId::System(x) => System(*x), CatalogItemId::IntrospectionSourceIndex(x) => IntrospectionSourceIndex(*x), + CatalogItemId::PersistedIntrospectionSource(x) => PersistedIntrospectionSource(*x), CatalogItemId::User(x) => User(*x), CatalogItemId::Transient(x) => Transient(*x), }), @@ -120,6 +129,9 @@ impl RustType for CatalogItemId { match proto.kind { Some(System(x)) => Ok(CatalogItemId::System(x)), Some(IntrospectionSourceIndex(x)) => Ok(CatalogItemId::IntrospectionSourceIndex(x)), + Some(PersistedIntrospectionSource(x)) => { + Ok(CatalogItemId::PersistedIntrospectionSource(x)) + } Some(User(x)) => Ok(CatalogItemId::User(x)), Some(Transient(x)) => Ok(CatalogItemId::Transient(x)), None => Err(TryFromProtoError::missing_field("ProtoCatalogItemId::kind")), diff --git a/src/repr/src/global_id.rs b/src/repr/src/global_id.rs index d5cf7d210bf17..5d75242cb75e3 100644 --- a/src/repr/src/global_id.rs +++ b/src/repr/src/global_id.rs @@ -44,6 +44,8 @@ pub enum GlobalId { System(u64), /// Introspection Source Index namespace. IntrospectionSourceIndex(u64), + /// Persisted introspection source namespace (per-replica persist shards). + PersistedIntrospectionSource(u64), /// User namespace. User(u64), /// Transient namespace. @@ -61,7 +63,9 @@ impl GlobalId { pub fn is_system(&self) -> bool { matches!( self, - GlobalId::System(_) | GlobalId::IntrospectionSourceIndex(_) + GlobalId::System(_) + | GlobalId::IntrospectionSourceIndex(_) + | GlobalId::PersistedIntrospectionSource(_) ) } @@ -90,7 +94,10 @@ impl FromStr for GlobalId { s = &s[1..]; let variant = match tag { 's' => { - if Some('i') == s.chars().next() { + if s.starts_with("ip") { + s = &s[2..]; + GlobalId::PersistedIntrospectionSource + } else if s.starts_with('i') { s = &s[1..]; GlobalId::IntrospectionSourceIndex } else { @@ -111,6 +118,7 @@ impl fmt::Display for GlobalId { match self { GlobalId::System(id) => write!(f, "s{}", id), GlobalId::IntrospectionSourceIndex(id) => write!(f, "si{}", id), + GlobalId::PersistedIntrospectionSource(id) => write!(f, "sip{}", id), GlobalId::User(id) => write!(f, "u{}", id), GlobalId::Transient(id) => write!(f, "t{}", id), GlobalId::Explain => write!(f, "Explained Query"), diff --git a/src/sql-lexer/src/keywords.txt b/src/sql-lexer/src/keywords.txt index a0a975f067787..ceea0821f3767 100644 --- a/src/sql-lexer/src/keywords.txt +++ b/src/sql-lexer/src/keywords.txt @@ -343,6 +343,7 @@ Partitions Password Path Pattern +Persist Physical Plan Plans diff --git a/src/sql-parser/src/ast/defs/statement.rs b/src/sql-parser/src/ast/defs/statement.rs index a67c7d97a5f67..de9ef431fee40 100644 --- a/src/sql-parser/src/ast/defs/statement.rs +++ b/src/sql-parser/src/ast/defs/statement.rs @@ -2222,6 +2222,8 @@ pub enum ClusterOptionName { ReplicationFactor, /// The `SIZE` option. Size, + /// The `PERSIST INTROSPECTION [[=] ]` option. + PersistIntrospection, /// The `SCHEDULE` option. Schedule, /// The `WORKLOAD CLASS` option. @@ -2236,6 +2238,7 @@ impl AstDisplay for ClusterOptionName { ClusterOptionName::IntrospectionDebugging => f.write_str("INTROSPECTION DEBUGGING"), ClusterOptionName::IntrospectionInterval => f.write_str("INTROSPECTION INTERVAL"), ClusterOptionName::Managed => f.write_str("MANAGED"), + ClusterOptionName::PersistIntrospection => f.write_str("PERSIST INTROSPECTION"), ClusterOptionName::Replicas => f.write_str("REPLICAS"), ClusterOptionName::ReplicationFactor => f.write_str("REPLICATION FACTOR"), ClusterOptionName::Size => f.write_str("SIZE"), @@ -2258,6 +2261,7 @@ impl WithOptionName for ClusterOptionName { | ClusterOptionName::IntrospectionDebugging | ClusterOptionName::IntrospectionInterval | ClusterOptionName::Managed + | ClusterOptionName::PersistIntrospection | ClusterOptionName::Replicas | ClusterOptionName::ReplicationFactor | ClusterOptionName::Size diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index 1d5444d0b0c1d..4e9c9ce9532c6 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -4567,6 +4567,7 @@ impl<'a> Parser<'a> { DISK, INTROSPECTION, MANAGED, + PERSIST, REPLICAS, REPLICATION, SIZE, @@ -4585,6 +4586,10 @@ impl<'a> Parser<'a> { _ => unreachable!(), }, MANAGED => ClusterOptionName::Managed, + PERSIST => { + self.expect_keyword(INTROSPECTION)?; + ClusterOptionName::PersistIntrospection + } REPLICAS => ClusterOptionName::Replicas, REPLICATION => { self.expect_keyword(FACTOR)?; diff --git a/src/sql-parser/tests/testdata/ddl b/src/sql-parser/tests/testdata/ddl index 088b8061b5a86..8d3a7559c7af8 100644 --- a/src/sql-parser/tests/testdata/ddl +++ b/src/sql-parser/tests/testdata/ddl @@ -1796,14 +1796,14 @@ CreateCluster(CreateClusterStatement { name: Ident("cluster"), options: [Cluster parse-statement CREATE CLUSTER cluster WITH REPLICAS () ---- -error: Expected one of AVAILABILITY or DISK or INTROSPECTION or MANAGED or REPLICAS or REPLICATION or SIZE or SCHEDULE or WORKLOAD, found WITH +error: Expected one of AVAILABILITY or DISK or INTROSPECTION or MANAGED or PERSIST or REPLICAS or REPLICATION or SIZE or SCHEDULE or WORKLOAD, found WITH CREATE CLUSTER cluster WITH REPLICAS () ^ parse-statement CREATE CLUSTER cluster REPLICAS (), BADOPT ---- -error: Expected one of AVAILABILITY or DISK or INTROSPECTION or MANAGED or REPLICAS or REPLICATION or SIZE or SCHEDULE or WORKLOAD, found identifier "badopt" +error: Expected one of AVAILABILITY or DISK or INTROSPECTION or MANAGED or PERSIST or REPLICAS or REPLICATION or SIZE or SCHEDULE or WORKLOAD, found identifier "badopt" CREATE CLUSTER cluster REPLICAS (), BADOPT ^ diff --git a/src/sql/src/ast/transform.rs b/src/sql/src/ast/transform.rs index 5ad7825337746..436ce2c65a471 100644 --- a/src/sql/src/ast/transform.rs +++ b/src/sql/src/ast/transform.rs @@ -500,3 +500,112 @@ impl<'ast> VisitMut<'ast, Raw> for CreateSqlIdReplacer<'_> { } } } + +/// Rewrites references to the `mz_introspection` schema in `stmt` to point to +/// `new_schema`. Handles both 2-component (`mz_introspection.item`) and +/// 3-component (`materialize.mz_introspection.item`) names. +/// +/// If `name_to_id` is provided, item-name references whose item component +/// appears in the map are converted from `RawItemName::Name` to +/// `RawItemName::Id` so that the catalog's topological-sort-based apply +/// pipeline can detect inter-view dependencies. +pub fn rewrite_introspection_schema_refs( + stmt: &mut Statement, + new_schema: &str, + name_to_id: Option<&BTreeMap>, +) { + let mut visitor = IntrospectionSchemaRewriter { + new_schema, + name_to_id, + }; + visitor.visit_statement_mut(stmt); +} + +struct IntrospectionSchemaRewriter<'a> { + new_schema: &'a str, + name_to_id: Option<&'a BTreeMap>, +} + +impl<'a> IntrospectionSchemaRewriter<'a> { + /// Returns `true` and rewrites the schema component if `name` references + /// the `mz_introspection` schema (2- or 3-component form). + fn maybe_rewrite_idents(&self, name: &mut [Ident]) -> bool { + match name { + [schema, _item] if schema.as_str() == "mz_introspection" => { + *schema = Ident::new_unchecked(self.new_schema); + true + } + [database, schema, _item] + if database.as_str() == "materialize" && schema.as_str() == "mz_introspection" => + { + *schema = Ident::new_unchecked(self.new_schema); + true + } + _ => false, + } + } +} + +impl<'a, 'ast> VisitMut<'ast, Raw> for IntrospectionSchemaRewriter<'a> { + fn visit_expr_mut(&mut self, e: &'ast mut Expr) { + match e { + Expr::Identifier(id) => { + // The last ID component is a column name that should not be + // considered in the rewrite. + let i = id.len() - 1; + self.maybe_rewrite_idents(&mut id[..i]); + } + Expr::QualifiedWildcard(id) => { + self.maybe_rewrite_idents(id); + } + _ => visit_mut::visit_expr_mut(self, e), + } + } + + fn visit_unresolved_item_name_mut( + &mut self, + unresolved_item_name: &'ast mut UnresolvedItemName, + ) { + self.maybe_rewrite_idents(&mut unresolved_item_name.0); + } + + fn visit_item_name_mut( + &mut self, + item_name: &'ast mut ::ItemName, + ) { + match item_name { + RawItemName::Name(n) | RawItemName::Id(_, n, _) => { + self.maybe_rewrite_idents(&mut n.0); + } + } + // After schema rewrite, optionally convert Name refs to Id refs so + // that the topological sort in the apply pipeline sees the + // dependency. + if let Some(name_to_id) = self.name_to_id { + if let RawItemName::Name(n) = &*item_name { + let item_str = match &n.0[..] { + [schema, item] if schema.as_str() == self.new_schema => Some(item.as_str()), + [_db, schema, item] if schema.as_str() == self.new_schema => { + Some(item.as_str()) + } + _ => None, + }; + if let Some(item_str) = item_str { + if let Some(id) = name_to_id.get(item_str) { + let id_str = id.to_string(); + // Take the UnresolvedItemName out and wrap in Id. + let name = std::mem::replace( + item_name, + RawItemName::Name(UnresolvedItemName(vec![])), + ); + let n = match name { + RawItemName::Name(n) => n, + _ => unreachable!(), + }; + *item_name = RawItemName::Id(id_str, n, None); + } + } + } + } + } +} diff --git a/src/sql/src/plan.rs b/src/sql/src/plan.rs index a8eb1c9d38c28..d3057294835f2 100644 --- a/src/sql/src/plan.rs +++ b/src/sql/src/plan.rs @@ -589,6 +589,7 @@ pub struct CreateClusterManagedPlan { pub size: String, pub availability_zones: Vec, pub compute: ComputeReplicaConfig, + pub persist_introspection: bool, pub optimizer_feature_overrides: OptimizerFeatureOverrides, pub schedule: ClusterSchedule, } @@ -2020,6 +2021,7 @@ pub struct PlanClusterOption { pub introspection_debugging: AlterOptionParameter, pub introspection_interval: AlterOptionParameter, pub managed: AlterOptionParameter, + pub persist_introspection: AlterOptionParameter, pub replicas: AlterOptionParameter>, pub replication_factor: AlterOptionParameter, pub size: AlterOptionParameter, @@ -2034,6 +2036,7 @@ impl Default for PlanClusterOption { introspection_debugging: AlterOptionParameter::Unchanged, introspection_interval: AlterOptionParameter::Unchanged, managed: AlterOptionParameter::Unchanged, + persist_introspection: AlterOptionParameter::Unchanged, replicas: AlterOptionParameter::Unchanged, replication_factor: AlterOptionParameter::Unchanged, size: AlterOptionParameter::Unchanged, diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index 0343f1c676584..1a8842e75f4f3 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -4765,6 +4765,7 @@ generate_extracted_config!( (IntrospectionDebugging, bool), (IntrospectionInterval, OptionalDuration), (Managed, bool), + (PersistIntrospection, bool), (Replicas, Vec>), (ReplicationFactor, u32), (Size, String), @@ -4860,6 +4861,7 @@ pub fn plan_create_cluster_inner( introspection_debugging, introspection_interval, managed, + persist_introspection, replicas, replication_factor, seen: _, @@ -4965,6 +4967,7 @@ pub fn plan_create_cluster_inner( size, availability_zones, compute, + persist_introspection: persist_introspection.unwrap_or(false), optimizer_feature_overrides, schedule, }), @@ -4986,6 +4989,9 @@ pub fn plan_create_cluster_inner( if introspection_interval.is_some() { sql_bail!("INTROSPECTION INTERVAL not supported for unmanaged clusters"); } + if persist_introspection.is_some() { + sql_bail!("PERSIST INTROSPECTION not supported for unmanaged clusters"); + } if size.is_some() { sql_bail!("SIZE not supported for unmanaged clusters"); } @@ -5031,6 +5037,7 @@ pub fn unplan_create_cluster( size, availability_zones, compute, + persist_introspection, optimizer_feature_overrides, schedule, }) => { @@ -5095,6 +5102,11 @@ pub fn unplan_create_cluster( introspection_debugging: Some(introspection_debugging), introspection_interval, managed: Some(true), + persist_introspection: if persist_introspection { + Some(true) + } else { + None + }, replicas: None, replication_factor, size: Some(size), @@ -6235,6 +6247,7 @@ pub fn plan_alter_cluster( introspection_debugging, introspection_interval, managed, + persist_introspection, replicas: replica_defs, replication_factor, seen: _, @@ -6338,6 +6351,9 @@ pub fn plan_alter_cluster( if introspection_interval.is_some() { sql_bail!("INTROSPECTION INTERVAL not supported for unmanaged clusters"); } + if persist_introspection.is_some() { + sql_bail!("PERSIST INTROSPECTION not supported for unmanaged clusters"); + } if size.is_some() { sql_bail!("SIZE not supported for unmanaged clusters"); } @@ -6391,6 +6407,9 @@ pub fn plan_alter_cluster( if let Some(introspection_interval) = introspection_interval { options.introspection_interval = AlterOptionParameter::Set(introspection_interval); } + if let Some(persist_introspection) = persist_introspection { + options.persist_introspection = AlterOptionParameter::Set(persist_introspection); + } if disk.is_some() { // The `DISK` option is a no-op for legacy cluster sizes and was never allowed for // `cc` sizes. The long term plan is to phase out the legacy sizes, at which point @@ -6436,6 +6455,7 @@ pub fn plan_alter_cluster( IntrospectionInterval => options.introspection_interval = Reset, IntrospectionDebugging => options.introspection_debugging = Reset, Managed => options.managed = Reset, + PersistIntrospection => options.persist_introspection = Reset, Replicas => options.replicas = Reset, ReplicationFactor => options.replication_factor = Reset, Size => options.size = Reset, diff --git a/test/persisted-introspection/mzcompose b/test/persisted-introspection/mzcompose new file mode 100755 index 0000000000000..1f866645dabc8 --- /dev/null +++ b/test/persisted-introspection/mzcompose @@ -0,0 +1,14 @@ +#!/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. +# +# mzcompose — runs Docker Compose with Materialize customizations. + +exec "$(dirname "$0")"/../../bin/pyactivate -m materialize.cli.mzcompose "$@" diff --git a/test/persisted-introspection/mzcompose.py b/test/persisted-introspection/mzcompose.py new file mode 100644 index 0000000000000..27919f4b78ccb --- /dev/null +++ b/test/persisted-introspection/mzcompose.py @@ -0,0 +1,556 @@ +# 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. + +"""Tests for persist-backed compute introspection sources.""" + +import time +from textwrap import dedent + +from materialize.mzcompose.composition import Composition +from materialize.mzcompose.services.materialized import Materialized + +SERVICES = [ + Materialized(), +] + + +def workflow_default(c: Composition) -> None: + """Run all test workflows.""" + + def process(name: str) -> None: + if name == "default": + return + with c.test_case(name): + c.workflow(name) + + c.test_parts(list(c.workflows.keys()), process) + + +def workflow_sql_syntax(c: Composition) -> None: + """Test that PERSIST INTROSPECTION option is accepted in CREATE/ALTER CLUSTER.""" + c.up("materialized") + + # CREATE CLUSTER with PERSIST INTROSPECTION = true + c.sql( + "CREATE CLUSTER pi_true (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # CREATE CLUSTER with PERSIST INTROSPECTION = false + c.sql( + "CREATE CLUSTER pi_false (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = false)" + ) + + # CREATE CLUSTER without the option (defaults to false) + c.sql("CREATE CLUSTER pi_default (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1)") + + # ALTER CLUSTER to toggle the option + c.sql("ALTER CLUSTER pi_false SET (PERSIST INTROSPECTION = true)") + c.sql("ALTER CLUSTER pi_true SET (PERSIST INTROSPECTION = false)") + + # RESET the option + c.sql("ALTER CLUSTER pi_default SET (PERSIST INTROSPECTION = true)") + c.sql("ALTER CLUSTER pi_default RESET (PERSIST INTROSPECTION)") + + # Clean up + c.sql("DROP CLUSTER pi_true CASCADE") + c.sql("DROP CLUSTER pi_false CASCADE") + c.sql("DROP CLUSTER pi_default CASCADE") + + c.kill("materialized") + + +def workflow_dyncfg_kill_switch(c: Composition) -> None: + """Test that the dyncfg kill switch controls persist introspection behavior.""" + c.up("materialized") + + # Disable the global kill switch. + c.sql( + "ALTER SYSTEM SET enable_persist_introspection = false", + port=6877, + user="mz_system", + ) + + # Creating a cluster with PERSIST INTROSPECTION = true should still succeed, + # even when the dyncfg is off. The dyncfg gates the runtime behavior, not + # the SQL syntax. + c.sql( + "CREATE CLUSTER pi_dyncfg (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # Verify the cluster is functional. + c.sql("SET CLUSTER = pi_dyncfg") + c.sql("CREATE TABLE t_dyncfg (x int)") + c.sql("CREATE MATERIALIZED VIEW mv_dyncfg AS SELECT count(*) FROM t_dyncfg") + _wait_for_view(c, "mv_dyncfg") + result = c.sql_query("SELECT * FROM mv_dyncfg") + assert result[0][0] == 0, f"expected 0, got {result[0][0]}" + + # Re-enable the kill switch. + c.sql( + "ALTER SYSTEM SET enable_persist_introspection = true", + port=6877, + user="mz_system", + ) + + # The cluster should remain functional after re-enabling. + c.sql("INSERT INTO t_dyncfg VALUES (1), (2), (3)") + _wait_for_view(c, "mv_dyncfg", expected_count=3) + result = c.sql_query("SELECT * FROM mv_dyncfg") + assert result[0][0] == 3, f"expected 3, got {result[0][0]}" + + # Clean up. + c.sql("SET CLUSTER = quickstart") + c.sql("DROP CLUSTER pi_dyncfg CASCADE") + c.sql("DROP TABLE t_dyncfg CASCADE") + + c.kill("materialized") + + +def workflow_cluster_lifecycle(c: Composition) -> None: + """Test creating, using, and dropping clusters with persist introspection.""" + c.up("materialized") + + # Create a cluster with persist introspection enabled. + c.sql( + "CREATE CLUSTER lifecycle_test (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # Create objects on the cluster. + c.sql("SET CLUSTER = lifecycle_test") + c.sql("CREATE TABLE t_lifecycle (x int)") + c.sql( + "CREATE MATERIALIZED VIEW mv_lifecycle AS SELECT count(*) AS cnt FROM t_lifecycle" + ) + + # Insert data and verify the MV works. + c.sql("INSERT INTO t_lifecycle VALUES (1), (2), (3)") + _wait_for_view(c, "mv_lifecycle", expected_count=3) + result = c.sql_query("SELECT cnt FROM mv_lifecycle") + assert result[0][0] == 3, f"expected 3, got {result[0][0]}" + + # Verify introspection data is flowing by checking that the cluster has + # dataflow operators (via the standard introspection subscribe path). + c.sql("SET CLUSTER = quickstart") + replica_count = c.sql_query( + dedent( + """ + SELECT count(*) + FROM mz_cluster_replicas r + JOIN mz_clusters c ON r.cluster_id = c.id + WHERE c.name = 'lifecycle_test' + """ + ) + ) + assert replica_count[0][0] == 1, f"expected 1 replica, got {replica_count[0][0]}" + + # Drop the cluster with CASCADE. + c.sql("DROP TABLE t_lifecycle CASCADE") + c.sql("DROP CLUSTER lifecycle_test CASCADE") + + # Verify the cluster is gone. + result = c.sql_query( + "SELECT count(*) FROM mz_clusters WHERE name = 'lifecycle_test'" + ) + assert result[0][0] == 0, "cluster should be dropped" + + # Recreate with the same name to verify no stale state. + c.sql( + "CREATE CLUSTER lifecycle_test (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + c.sql("SET CLUSTER = lifecycle_test") + c.sql("CREATE TABLE t_lifecycle2 (x int)") + c.sql("INSERT INTO t_lifecycle2 VALUES (42)") + c.sql( + "CREATE MATERIALIZED VIEW mv_lifecycle2 AS SELECT count(*) AS cnt FROM t_lifecycle2" + ) + _wait_for_view(c, "mv_lifecycle2", expected_count=1) + result = c.sql_query("SELECT cnt FROM mv_lifecycle2") + assert result[0][0] == 1, f"expected 1, got {result[0][0]}" + + # Clean up. + c.sql("SET CLUSTER = quickstart") + c.sql("DROP TABLE t_lifecycle2 CASCADE") + c.sql("DROP CLUSTER lifecycle_test CASCADE") + + c.kill("materialized") + + +def workflow_replica_restart(c: Composition) -> None: + """Test that a replica restart leaves data in a consistent state. + + After dropping and recreating the replica, old introspection data from the + previous replica should not linger. The MV persist sink's self-correction + mechanism retracts stale data on restart. + """ + c.up("materialized") + + # Create a cluster with persist introspection. + c.sql( + "CREATE CLUSTER restart_test (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # Populate with data. + c.sql("SET CLUSTER = restart_test") + c.sql("CREATE TABLE t_restart (x int)") + c.sql("INSERT INTO t_restart VALUES (1), (2), (3)") + c.sql( + "CREATE MATERIALIZED VIEW mv_restart AS SELECT count(*) AS cnt FROM t_restart" + ) + _wait_for_view(c, "mv_restart", expected_count=3) + + # Verify MV returns correct data before restart. + result = c.sql_query("SELECT cnt FROM mv_restart") + assert result[0][0] == 3, f"expected 3 before restart, got {result[0][0]}" + + # Simulate a replica restart by scaling down and back up. + # This drops the old replica and creates a new one. + c.sql("ALTER CLUSTER restart_test SET (REPLICATION FACTOR 0)") + + # Verify the cluster has no replicas. + c.sql("SET CLUSTER = quickstart") + replica_count = c.sql_query( + dedent( + """ + SELECT count(*) + FROM mz_cluster_replicas r + JOIN mz_clusters c ON r.cluster_id = c.id + WHERE c.name = 'restart_test' + """ + ) + ) + assert replica_count[0][0] == 0, f"expected 0 replicas, got {replica_count[0][0]}" + + # Scale back up. A new replica is created with fresh persist sinks. + c.sql("ALTER CLUSTER restart_test SET (REPLICATION FACTOR 1)") + + # Wait for the replica to become ready. + _wait_for_replica_online(c, "restart_test") + + # Verify the MV returns correct data after restart. The self-correction + # mechanism ensures the new replica's persist sinks retract any stale data + # and write fresh data. + c.sql("SET CLUSTER = restart_test") + _wait_for_view(c, "mv_restart", expected_count=3) + result = c.sql_query("SELECT cnt FROM mv_restart") + assert result[0][0] == 3, f"expected 3 after restart, got {result[0][0]}" + + # Insert more data to verify the cluster is fully operational after restart. + c.sql("INSERT INTO t_restart VALUES (4), (5)") + _wait_for_view(c, "mv_restart", expected_count=5) + result = c.sql_query("SELECT cnt FROM mv_restart") + assert result[0][0] == 5, f"expected 5 after insert, got {result[0][0]}" + + # Clean up. + c.sql("SET CLUSTER = quickstart") + c.sql("DROP TABLE t_restart CASCADE") + c.sql("DROP CLUSTER restart_test CASCADE") + + c.kill("materialized") + + +def workflow_catalog_changes(c: Composition) -> None: + """Test that adding/removing clusters with persist introspection has the + expected catalog effects.""" + c.up("materialized") + + # Create a cluster with persist introspection. + c.sql( + "CREATE CLUSTER cat_test (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # The cluster should appear in mz_clusters. + result = c.sql_query( + "SELECT name, managed FROM mz_clusters WHERE name = 'cat_test'" + ) + assert len(result) == 1, f"expected 1 cluster, got {len(result)}" + assert result[0][0] == "cat_test" + assert result[0][1] is True # managed + + # Verify the cluster has a replica. + replicas = c.sql_query( + dedent( + """ + SELECT r.name + FROM mz_cluster_replicas r + JOIN mz_clusters c ON r.cluster_id = c.id + WHERE c.name = 'cat_test' + """ + ) + ) + assert len(replicas) == 1, f"expected 1 replica, got {len(replicas)}" + + # ALTER CLUSTER to toggle persist introspection off. + c.sql("ALTER CLUSTER cat_test SET (PERSIST INTROSPECTION = false)") + + # The cluster should still exist and be managed. + result = c.sql_query( + "SELECT name, managed FROM mz_clusters WHERE name = 'cat_test'" + ) + assert len(result) == 1 + assert result[0][1] is True + + # ALTER CLUSTER to change size (triggers replica replacement). + c.sql("ALTER CLUSTER cat_test SET (SIZE 'scale=2,workers=2')") + _wait_for_replica_online(c, "cat_test") + + # Verify replica still exists after size change. + replicas = c.sql_query( + dedent( + """ + SELECT r.name + FROM mz_cluster_replicas r + JOIN mz_clusters c ON r.cluster_id = c.id + WHERE c.name = 'cat_test' + """ + ) + ) + assert len(replicas) == 1, f"expected 1 replica after resize, got {len(replicas)}" + + # Drop the cluster. + c.sql("DROP CLUSTER cat_test CASCADE") + + # Verify complete removal from catalog. + result = c.sql_query("SELECT count(*) FROM mz_clusters WHERE name = 'cat_test'") + assert result[0][0] == 0, "cluster should be removed from catalog" + + result = c.sql_query( + dedent( + """ + SELECT count(*) + FROM mz_cluster_replicas r + JOIN mz_clusters c ON r.cluster_id = c.id + WHERE c.name = 'cat_test' + """ + ) + ) + assert result[0][0] == 0, "replicas should be removed from catalog" + + c.kill("materialized") + + +def workflow_multiple_clusters(c: Composition) -> None: + """Test that multiple clusters with persist introspection can coexist.""" + c.up("materialized") + + # Create two clusters with persist introspection. + c.sql( + "CREATE CLUSTER multi_a (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + c.sql( + "CREATE CLUSTER multi_b (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # Create tables and MVs on each cluster. + c.sql("CREATE TABLE t_multi (x int)") + c.sql("INSERT INTO t_multi VALUES (1), (2)") + + c.sql("SET CLUSTER = multi_a") + c.sql("CREATE MATERIALIZED VIEW mv_a AS SELECT count(*) AS cnt FROM t_multi") + + c.sql("SET CLUSTER = multi_b") + c.sql("CREATE MATERIALIZED VIEW mv_b AS SELECT sum(x) AS total FROM t_multi") + + # Verify both MVs produce correct results. + c.sql("SET CLUSTER = multi_a") + _wait_for_view(c, "mv_a", expected_count=2) + result = c.sql_query("SELECT cnt FROM mv_a") + assert result[0][0] == 2, f"expected 2, got {result[0][0]}" + + c.sql("SET CLUSTER = multi_b") + _wait_for_view(c, "mv_b", expected_total=3) + result = c.sql_query("SELECT total FROM mv_b") + assert result[0][0] == 3, f"expected 3, got {result[0][0]}" + + # Drop one cluster; the other should be unaffected. + c.sql("SET CLUSTER = quickstart") + c.sql("DROP CLUSTER multi_a CASCADE") + + c.sql("SET CLUSTER = multi_b") + result = c.sql_query("SELECT total FROM mv_b") + assert result[0][0] == 3, f"expected 3 after dropping multi_a, got {result[0][0]}" + + # Clean up. + c.sql("SET CLUSTER = quickstart") + c.sql("DROP TABLE t_multi CASCADE") + c.sql("DROP CLUSTER multi_b CASCADE") + + c.kill("materialized") + + +def workflow_environmentd_restart(c: Composition) -> None: + """Test that persist introspection clusters survive an environmentd restart.""" + c.up("materialized") + + # Create a cluster with persist introspection and populate data. + c.sql( + "CREATE CLUSTER restart_env (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + c.sql("CREATE TABLE t_env (x int)") + c.sql("INSERT INTO t_env VALUES (10), (20), (30)") + c.sql("SET CLUSTER = restart_env") + c.sql("CREATE MATERIALIZED VIEW mv_env AS SELECT count(*) AS cnt FROM t_env") + _wait_for_view(c, "mv_env", expected_count=3) + + # Restart environmentd. + c.kill("materialized") + c.up("materialized") + + # Verify the cluster and data survived. + c.sql("SET CLUSTER = restart_env") + _wait_for_view(c, "mv_env", expected_count=3) + result = c.sql_query("SELECT cnt FROM mv_env") + assert result[0][0] == 3, f"expected 3 after restart, got {result[0][0]}" + + # Verify the cluster is still managed with persist introspection. + c.sql("SET CLUSTER = quickstart") + result = c.sql_query("SELECT managed FROM mz_clusters WHERE name = 'restart_env'") + assert result[0][0] is True + + # Clean up. + c.sql("DROP TABLE t_env CASCADE") + c.sql("DROP CLUSTER restart_env CASCADE") + + c.kill("materialized") + + +def workflow_per_replica_views(c: Composition) -> None: + """Test that per-replica introspection views are created and queryable.""" + c.up("materialized") + + # Enable the dyncfg so persist introspection actually creates schemas. + c.sql( + "ALTER SYSTEM SET enable_persist_introspection = true", + port=6877, + user="mz_system", + ) + + # Create a cluster with persist introspection enabled. + c.sql( + "CREATE CLUSTER view_test (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true)" + ) + + # Wait for the replica to be online. + _wait_for_replica_online(c, "view_test") + + # Look up the per-replica schema name. Schema names use IDs, not cluster + # names: mz_introspection_{cluster_id}_{replica_id}. + schema_row = c.sql_query( + dedent( + """ + SELECT s.name + FROM mz_schemas s + JOIN mz_clusters c ON s.name LIKE 'mz_introspection_' || c.id || '_%' + WHERE c.name = 'view_test' + """ + ) + ) + assert len(schema_row) == 1, f"expected 1 per-replica schema, got {len(schema_row)}" + schema_name = schema_row[0][0] + + # Query a per-replica view (mz_dataflows aggregates mz_dataflows_per_worker). + # Create some dataflows to ensure the view has data. + c.sql("SET CLUSTER = view_test") + c.sql("CREATE TABLE t_views (x int)") + c.sql("CREATE MATERIALIZED VIEW mv_views AS SELECT count(*) FROM t_views") + _wait_for_view(c, "mv_views", expected_count=0) + + # Query a derived view (mz_dataflows) in the per-replica schema. + result = c.sql_query(f"SELECT count(*) FROM materialize.{schema_name}.mz_dataflows") + assert result[0][0] >= 0, f"expected non-negative count, got {result[0][0]}" + + # Query a base per-worker view. + result = c.sql_query( + f"SELECT count(*) FROM materialize.{schema_name}.mz_dataflows_per_worker" + ) + assert result[0][0] >= 0, f"expected non-negative count, got {result[0][0]}" + + # Query mz_arrangement_sizes which chains multiple views. + result = c.sql_query( + f"SELECT count(*) FROM materialize.{schema_name}.mz_arrangement_sizes" + ) + assert result[0][0] >= 0, f"expected non-negative count, got {result[0][0]}" + + # Drop the cluster and verify the schema is removed. + c.sql("SET CLUSTER = quickstart") + c.sql("DROP TABLE t_views CASCADE") + c.sql("DROP CLUSTER view_test CASCADE") + schema_count = c.sql_query( + f"SELECT count(*) FROM mz_schemas WHERE name = '{schema_name}'" + ) + assert schema_count[0][0] == 0, "per-replica schema should be dropped" + + c.kill("materialized") + + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + + +def _wait_for_view( + c: Composition, + view_name: str, + expected_count: int | None = None, + expected_total: int | None = None, + timeout_secs: int = 30, +) -> None: + """Poll a materialized view until it returns the expected result.""" + deadline = time.time() + timeout_secs + last_result = None + while time.time() < deadline: + try: + if expected_count is not None: + result = c.sql_query(f"SELECT * FROM {view_name}") + last_result = result + if result and result[0][0] == expected_count: + return + elif expected_total is not None: + result = c.sql_query(f"SELECT * FROM {view_name}") + last_result = result + if result and result[0][0] == expected_total: + return + else: + # Just check the view is queryable. + c.sql_query(f"SELECT * FROM {view_name}") + return + except Exception: + pass + time.sleep(0.5) + raise TimeoutError( + f"View {view_name} did not reach expected state within {timeout_secs}s. " + f"Last result: {last_result}" + ) + + +def _wait_for_replica_online( + c: Composition, cluster_name: str, timeout_secs: int = 60 +) -> None: + """Wait until all replicas of a cluster report 'ready' status.""" + deadline = time.time() + timeout_secs + while time.time() < deadline: + try: + result = c.sql_query( + dedent( + f""" + SELECT count(*) + FROM mz_internal.mz_cluster_replica_statuses s + JOIN mz_cluster_replicas r ON s.replica_id = r.id + JOIN mz_clusters c ON r.cluster_id = c.id + WHERE c.name = '{cluster_name}' AND s.status = 'online' + """ + ) + ) + if result and result[0][0] > 0: + return + except Exception: + pass + time.sleep(1) + raise TimeoutError( + f"Replicas of cluster {cluster_name} did not become ready within {timeout_secs}s" + ) diff --git a/test/sqllogictest/cluster.slt b/test/sqllogictest/cluster.slt index dc6cf3139407c..858c18f807f2e 100644 --- a/test/sqllogictest/cluster.slt +++ b/test/sqllogictest/cluster.slt @@ -19,7 +19,7 @@ ALTER SYSTEM SET unsafe_enable_unorchestrated_cluster_replicas = on; ---- COMPLETE 0 -statement error Expected one of AVAILABILITY or DISK or INTROSPECTION or MANAGED or REPLICAS or REPLICATION or SIZE or SCHEDULE or WORKLOAD, found EOF +statement error Expected one of AVAILABILITY or DISK or INTROSPECTION or MANAGED or PERSIST or REPLICAS or REPLICATION or SIZE or SCHEDULE or WORKLOAD, found EOF CREATE CLUSTER foo statement ok diff --git a/test/sqllogictest/persisted_introspection.slt b/test/sqllogictest/persisted_introspection.slt new file mode 100644 index 0000000000000..44a9933380f56 --- /dev/null +++ b/test/sqllogictest/persisted_introspection.slt @@ -0,0 +1,31 @@ +# 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. + +# Tests for persist-backed introspection sources. + +mode cockroach + +reset-server + +# Creating a cluster with PERSIST INTROSPECTION should succeed without +# panicking. Regression test for a catalog ordering bug where persisted +# introspection source updates were applied before the cluster replica +# existed. +statement ok +CREATE CLUSTER pi_test (SIZE 'scale=1,workers=1', REPLICATION FACTOR 1, PERSIST INTROSPECTION = true) + +# Verify the cluster and its replica were created. +query T +SELECT name FROM mz_cluster_replicas WHERE cluster_id = (SELECT id FROM mz_clusters WHERE name = 'pi_test') +---- +r1 + +# Clean up. +statement ok +DROP CLUSTER pi_test CASCADE