Skip to main content

mz_adapter/
catalog.rs

1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10// TODO(jkosh44) Move to mz_catalog crate.
11
12//! Persistent metadata storage for the coordinator.
13
14use std::borrow::Cow;
15use std::collections::{BTreeMap, BTreeSet};
16use std::convert;
17use std::sync::Arc;
18
19use futures::future::BoxFuture;
20use futures::{Future, FutureExt};
21use itertools::Itertools;
22use mz_adapter_types::bootstrap_builtin_cluster_config::{
23    ANALYTICS_CLUSTER_DEFAULT_REPLICATION_FACTOR, BootstrapBuiltinClusterConfig,
24    CATALOG_SERVER_CLUSTER_DEFAULT_REPLICATION_FACTOR, PROBE_CLUSTER_DEFAULT_REPLICATION_FACTOR,
25    SUPPORT_CLUSTER_DEFAULT_REPLICATION_FACTOR, SYSTEM_CLUSTER_DEFAULT_REPLICATION_FACTOR,
26};
27use mz_adapter_types::connection::ConnectionId;
28use mz_audit_log::{EventType, FullNameV1, ObjectType, VersionedStorageUsage};
29use mz_build_info::{BuildInfo, DUMMY_BUILD_INFO};
30use mz_catalog::builtin::{
31    BUILTIN_PREFIXES, BuiltinCluster, BuiltinLog, BuiltinSource, BuiltinTable,
32    MZ_CATALOG_SERVER_CLUSTER,
33};
34use mz_catalog::config::{BuiltinItemMigrationConfig, ClusterReplicaSizeMap, Config, StateConfig};
35#[cfg(test)]
36use mz_catalog::durable::CatalogError;
37use mz_catalog::durable::{
38    BootstrapArgs, DurableCatalogState, TestCatalogStateBuilder, test_bootstrap_args,
39};
40use mz_catalog::expr_cache::{ExpressionCacheHandle, GlobalExpressions, LocalExpressions};
41use mz_catalog::memory::error::{Error, ErrorKind};
42use mz_catalog::memory::objects::{
43    CatalogCollectionEntry, CatalogEntry, CatalogItem, Cluster, ClusterReplica, Database,
44    NetworkPolicy, Role, RoleAuth, Schema,
45};
46use mz_compute_types::dataflows::DataflowDescription;
47use mz_controller::clusters::ReplicaLocation;
48use mz_controller_types::{ClusterId, ReplicaId};
49use mz_expr::OptimizedMirRelationExpr;
50use mz_license_keys::ValidatedLicenseKey;
51use mz_ore::metrics::MetricsRegistry;
52use mz_ore::now::{EpochMillis, NowFn, SYSTEM_TIME};
53use mz_ore::result::ResultExt as _;
54use mz_ore::{soft_assert_eq_or_log, soft_assert_or_log, soft_panic_or_log};
55use mz_persist_client::PersistClient;
56use mz_repr::adt::mz_acl_item::{AclMode, PrivilegeMap};
57use mz_repr::explain::ExprHumanizer;
58use mz_repr::namespaces::MZ_TEMP_SCHEMA;
59use mz_repr::network_policy_id::NetworkPolicyId;
60use mz_repr::optimize::OptimizerFeatures;
61use mz_repr::role_id::RoleId;
62use mz_repr::{CatalogItemId, Diff, GlobalId, RelationVersionSelector, SqlScalarType};
63use mz_secrets::InMemorySecretsController;
64use mz_sql::catalog::{
65    CatalogCluster, CatalogClusterReplica, CatalogDatabase, CatalogError as SqlCatalogError,
66    CatalogItem as SqlCatalogItem, CatalogItemType as SqlCatalogItemType, CatalogNetworkPolicy,
67    CatalogRole, CatalogSchema, DefaultPrivilegeAclItem, DefaultPrivilegeObject, EnvironmentId,
68    SessionCatalog, SystemObjectType,
69};
70use mz_sql::names::{
71    CommentObjectId, DatabaseId, FullItemName, FullSchemaName, ItemQualifiers, ObjectId,
72    PUBLIC_ROLE_NAME, PartialItemName, QualifiedItemName, QualifiedSchemaName,
73    ResolvedDatabaseSpecifier, ResolvedIds, SchemaId, SchemaSpecifier, SystemObjectId,
74};
75use mz_sql::plan::{Plan, PlanNotice, StatementDesc};
76use mz_sql::rbac;
77use mz_sql::session::metadata::SessionMetadata;
78use mz_sql::session::user::{MZ_SYSTEM_ROLE_ID, SUPPORT_USER, SYSTEM_USER};
79use mz_sql::session::vars::SystemVars;
80use mz_sql_parser::ast::QualifiedReplica;
81use mz_storage_types::connections::ConnectionContext;
82use mz_storage_types::connections::inline::{ConnectionResolver, InlinedConnection};
83use mz_transform::dataflow::DataflowMetainfo;
84use mz_transform::notice::OptimizerNotice;
85use smallvec::SmallVec;
86use tokio::sync::MutexGuard;
87use tokio::sync::mpsc::UnboundedSender;
88use uuid::Uuid;
89
90// DO NOT add any more imports from `crate` outside of `crate::catalog`.
91pub use crate::catalog::builtin_table_updates::BuiltinTableUpdate;
92pub use crate::catalog::open::{InitializeStateResult, OpenCatalogResult};
93pub use crate::catalog::state::CatalogState;
94pub use crate::catalog::transact::{
95    DropObjectInfo, Op, ReplicaCreateDropReason, TransactionResult,
96};
97use crate::command::CatalogDump;
98use crate::coord::TargetCluster;
99#[cfg(test)]
100use crate::coord::catalog_implications::parsed_state_updates::ParsedStateUpdate;
101use crate::session::{Portal, PreparedStatement, Session};
102use crate::util::ResultExt;
103use crate::{AdapterError, AdapterNotice, ExecuteResponse};
104
105mod builtin_table_updates;
106pub(crate) mod consistency;
107mod migrate;
108
109mod apply;
110mod open;
111mod state;
112mod timeline;
113mod transact;
114
115/// A `Catalog` keeps track of the SQL objects known to the planner.
116///
117/// For each object, it keeps track of both forward and reverse dependencies:
118/// i.e., which objects are depended upon by the object, and which objects
119/// depend upon the object. It enforces the SQL rules around dropping: an object
120/// cannot be dropped until all of the objects that depend upon it are dropped.
121/// It also enforces uniqueness of names.
122///
123/// SQL mandates a hierarchy of exactly three layers. A catalog contains
124/// databases, databases contain schemas, and schemas contain catalog items,
125/// like sources, sinks, view, and indexes.
126///
127/// To the outside world, databases, schemas, and items are all identified by
128/// name. Items can be referred to by their [`FullItemName`], which fully and
129/// unambiguously specifies the item, or a [`PartialItemName`], which can omit the
130/// database name and/or the schema name. Partial names can be converted into
131/// full names via a complicated resolution process documented by the
132/// [`CatalogState::resolve`] method.
133///
134/// The catalog also maintains special "ambient schemas": virtual schemas,
135/// implicitly present in all databases, that house various system views.
136/// The big examples of ambient schemas are `pg_catalog` and `mz_catalog`.
137#[derive(Debug)]
138pub struct Catalog {
139    state: CatalogState,
140    plans: CatalogPlans,
141    expr_cache_handle: Option<ExpressionCacheHandle>,
142    storage: Arc<tokio::sync::Mutex<Box<dyn mz_catalog::durable::DurableCatalogState>>>,
143    transient_revision: u64,
144}
145
146// Implement our own Clone because derive can't unless S is Clone, which it's
147// not (hence the Arc).
148impl Clone for Catalog {
149    fn clone(&self) -> Self {
150        Self {
151            state: self.state.clone(),
152            plans: self.plans.clone(),
153            expr_cache_handle: self.expr_cache_handle.clone(),
154            storage: Arc::clone(&self.storage),
155            transient_revision: self.transient_revision,
156        }
157    }
158}
159
160#[derive(Default, Debug, Clone)]
161pub struct CatalogPlans {
162    optimized_plan_by_id: BTreeMap<GlobalId, Arc<DataflowDescription<OptimizedMirRelationExpr>>>,
163    physical_plan_by_id: BTreeMap<GlobalId, Arc<DataflowDescription<mz_compute_types::plan::Plan>>>,
164    dataflow_metainfos: BTreeMap<GlobalId, DataflowMetainfo<Arc<OptimizerNotice>>>,
165    notices_by_dep_id: BTreeMap<GlobalId, SmallVec<[Arc<OptimizerNotice>; 4]>>,
166}
167
168impl Catalog {
169    /// Set the optimized plan for the item identified by `id`.
170    #[mz_ore::instrument(level = "trace")]
171    pub fn set_optimized_plan(
172        &mut self,
173        id: GlobalId,
174        plan: DataflowDescription<OptimizedMirRelationExpr>,
175    ) {
176        self.plans.optimized_plan_by_id.insert(id, plan.into());
177    }
178
179    /// Set the physical plan for the item identified by `id`.
180    #[mz_ore::instrument(level = "trace")]
181    pub fn set_physical_plan(
182        &mut self,
183        id: GlobalId,
184        plan: DataflowDescription<mz_compute_types::plan::Plan>,
185    ) {
186        self.plans.physical_plan_by_id.insert(id, plan.into());
187    }
188
189    /// Try to get the optimized plan for the item identified by `id`.
190    #[mz_ore::instrument(level = "trace")]
191    pub fn try_get_optimized_plan(
192        &self,
193        id: &GlobalId,
194    ) -> Option<&DataflowDescription<OptimizedMirRelationExpr>> {
195        self.plans.optimized_plan_by_id.get(id).map(AsRef::as_ref)
196    }
197
198    /// Try to get the physical plan for the item identified by `id`.
199    #[mz_ore::instrument(level = "trace")]
200    pub fn try_get_physical_plan(
201        &self,
202        id: &GlobalId,
203    ) -> Option<&DataflowDescription<mz_compute_types::plan::Plan>> {
204        self.plans.physical_plan_by_id.get(id).map(AsRef::as_ref)
205    }
206
207    /// Set the `DataflowMetainfo` for the item identified by `id`.
208    #[mz_ore::instrument(level = "trace")]
209    pub fn set_dataflow_metainfo(
210        &mut self,
211        id: GlobalId,
212        metainfo: DataflowMetainfo<Arc<OptimizerNotice>>,
213    ) {
214        // Add entries to the `notices_by_dep_id` lookup map.
215        for notice in metainfo.optimizer_notices.iter() {
216            for dep_id in notice.dependencies.iter() {
217                let entry = self.plans.notices_by_dep_id.entry(*dep_id).or_default();
218                entry.push(Arc::clone(notice))
219            }
220            if let Some(item_id) = notice.item_id {
221                soft_assert_eq_or_log!(
222                    item_id,
223                    id,
224                    "notice.item_id should match the id for whom we are saving the notice"
225                );
226            }
227        }
228        // Add the dataflow with the scoped entries.
229        self.plans.dataflow_metainfos.insert(id, metainfo);
230    }
231
232    /// Try to get the `DataflowMetainfo` for the item identified by `id`.
233    #[mz_ore::instrument(level = "trace")]
234    pub fn try_get_dataflow_metainfo(
235        &self,
236        id: &GlobalId,
237    ) -> Option<&DataflowMetainfo<Arc<OptimizerNotice>>> {
238        self.plans.dataflow_metainfos.get(id)
239    }
240
241    /// Drop all optimized and physical plans and `DataflowMetainfo`s for the
242    /// item identified by `id`.
243    ///
244    /// Ignore requests for non-existing plans or `DataflowMetainfo`s.
245    ///
246    /// Return a set containing all dropped notices. Note that if for some
247    /// reason we end up with two identical notices being dropped by the same
248    /// call, the result will contain only one instance of that notice.
249    #[mz_ore::instrument(level = "trace")]
250    pub fn drop_plans_and_metainfos(
251        &mut self,
252        drop_ids: &BTreeSet<GlobalId>,
253    ) -> BTreeSet<Arc<OptimizerNotice>> {
254        // Collect dropped notices in this set.
255        let mut dropped_notices = BTreeSet::new();
256
257        // Remove plans and metainfo.optimizer_notices entries.
258        for id in drop_ids {
259            self.plans.optimized_plan_by_id.remove(id);
260            self.plans.physical_plan_by_id.remove(id);
261            if let Some(mut metainfo) = self.plans.dataflow_metainfos.remove(id) {
262                soft_assert_or_log!(
263                    metainfo.optimizer_notices.iter().all_unique(),
264                    "should have been pushed there by `push_optimizer_notice_dedup`"
265                );
266                for n in metainfo.optimizer_notices.drain(..) {
267                    // Remove the corresponding notices_by_dep_id entries.
268                    for dep_id in n.dependencies.iter() {
269                        if let Some(notices) = self.plans.notices_by_dep_id.get_mut(dep_id) {
270                            soft_assert_or_log!(
271                                notices.iter().any(|x| &n == x),
272                                "corrupt notices_by_dep_id"
273                            );
274                            notices.retain(|x| &n != x)
275                        }
276                    }
277                    dropped_notices.insert(n);
278                }
279            }
280        }
281
282        // Remove notices_by_dep_id entries.
283        for id in drop_ids {
284            if let Some(mut notices) = self.plans.notices_by_dep_id.remove(id) {
285                for n in notices.drain(..) {
286                    // Remove the corresponding metainfo.optimizer_notices entries.
287                    if let Some(item_id) = n.item_id.as_ref() {
288                        if let Some(metainfo) = self.plans.dataflow_metainfos.get_mut(item_id) {
289                            metainfo.optimizer_notices.iter().for_each(|n2| {
290                                if let Some(item_id_2) = n2.item_id {
291                                    soft_assert_eq_or_log!(item_id_2, *item_id, "a notice's item_id should match the id for whom we have saved the notice");
292                                }
293                            });
294                            metainfo.optimizer_notices.retain(|x| &n != x);
295                        }
296                    }
297                    dropped_notices.insert(n);
298                }
299            }
300        }
301
302        // Collect dependency ids not in drop_ids with at least one dropped
303        // notice.
304        let mut todo_dep_ids = BTreeSet::new();
305        for notice in dropped_notices.iter() {
306            for dep_id in notice.dependencies.iter() {
307                if !drop_ids.contains(dep_id) {
308                    todo_dep_ids.insert(*dep_id);
309                }
310            }
311        }
312        // Remove notices in `dropped_notices` for all `notices_by_dep_id`
313        // entries in `todo_dep_ids`.
314        for id in todo_dep_ids {
315            if let Some(notices) = self.plans.notices_by_dep_id.get_mut(&id) {
316                notices.retain(|n| !dropped_notices.contains(n))
317            }
318        }
319
320        // (We used to have a sanity check here that
321        // `dropped_notices.iter().any(|n| Arc::strong_count(n) != 1)`
322        // but this is not a correct assertion: There might be other clones of the catalog (e.g. if
323        // a peek is running concurrently to an index drop), in which case those clones also hold
324        // references to the notices.)
325
326        dropped_notices
327    }
328}
329
330#[derive(Debug)]
331pub struct ConnCatalog<'a> {
332    state: Cow<'a, CatalogState>,
333    /// Because we don't have any way of removing items from the catalog
334    /// temporarily, we allow the ConnCatalog to pretend that a set of items
335    /// don't exist during resolution.
336    ///
337    /// This feature is necessary to allow re-planning of statements, which is
338    /// either incredibly useful or required when altering item definitions.
339    ///
340    /// Note that uses of this field should be used by short-lived
341    /// catalogs.
342    unresolvable_ids: BTreeSet<CatalogItemId>,
343    conn_id: ConnectionId,
344    cluster: String,
345    database: Option<DatabaseId>,
346    search_path: Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)>,
347    role_id: RoleId,
348    prepared_statements: Option<&'a BTreeMap<String, PreparedStatement>>,
349    portals: Option<&'a BTreeMap<String, Portal>>,
350    notices_tx: UnboundedSender<AdapterNotice>,
351}
352
353impl ConnCatalog<'_> {
354    pub fn conn_id(&self) -> &ConnectionId {
355        &self.conn_id
356    }
357
358    pub fn state(&self) -> &CatalogState {
359        &*self.state
360    }
361
362    /// Prevent planning from resolving item with the provided ID. Instead,
363    /// return an error as if the item did not exist.
364    ///
365    /// This feature is meant exclusively to permit re-planning statements
366    /// during update operations and should not be used otherwise given its
367    /// extremely "powerful" semantics.
368    ///
369    /// # Panics
370    /// If the catalog's role ID is not [`MZ_SYSTEM_ROLE_ID`].
371    pub fn mark_id_unresolvable_for_replanning(&mut self, id: CatalogItemId) {
372        assert_eq!(
373            self.role_id, MZ_SYSTEM_ROLE_ID,
374            "only the system role can mark IDs unresolvable",
375        );
376        self.unresolvable_ids.insert(id);
377    }
378
379    /// Returns the schemas:
380    /// - mz_catalog
381    /// - pg_catalog
382    /// - temp (if requested)
383    /// - all schemas from the session's search_path var that exist
384    pub fn effective_search_path(
385        &self,
386        include_temp_schema: bool,
387    ) -> Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)> {
388        self.state
389            .effective_search_path(&self.search_path, include_temp_schema)
390    }
391}
392
393impl ConnectionResolver for ConnCatalog<'_> {
394    fn resolve_connection(
395        &self,
396        id: CatalogItemId,
397    ) -> mz_storage_types::connections::Connection<InlinedConnection> {
398        self.state().resolve_connection(id)
399    }
400}
401
402impl Catalog {
403    /// Returns the catalog's transient revision, which starts at 1 and is
404    /// incremented on every change. This is not persisted to disk, and will
405    /// restart on every load.
406    pub fn transient_revision(&self) -> u64 {
407        self.transient_revision
408    }
409
410    /// Creates a debug catalog from the current
411    /// `METADATA_BACKEND_URL` with parameters set appropriately for debug contexts,
412    /// like in tests.
413    ///
414    /// WARNING! This function can arbitrarily fail because it does not make any
415    /// effort to adjust the catalog's contents' structure or semantics to the
416    /// currently running version, i.e. it does not apply any migrations.
417    ///
418    /// This function must not be called in production contexts. Use
419    /// [`Catalog::open`] with appropriately set configuration parameters
420    /// instead.
421    pub async fn with_debug<F, Fut, T>(f: F) -> T
422    where
423        F: FnOnce(Catalog) -> Fut,
424        Fut: Future<Output = T>,
425    {
426        let persist_client = PersistClient::new_for_tests().await;
427        let organization_id = Uuid::new_v4();
428        let bootstrap_args = test_bootstrap_args();
429        let catalog = Self::open_debug_catalog(persist_client, organization_id, &bootstrap_args)
430            .await
431            .expect("can open debug catalog");
432        f(catalog).await
433    }
434
435    /// Like [`Catalog::with_debug`], but the catalog created believes that bootstrap is still
436    /// in progress.
437    pub async fn with_debug_in_bootstrap<F, Fut, T>(f: F) -> T
438    where
439        F: FnOnce(Catalog) -> Fut,
440        Fut: Future<Output = T>,
441    {
442        let persist_client = PersistClient::new_for_tests().await;
443        let organization_id = Uuid::new_v4();
444        let bootstrap_args = test_bootstrap_args();
445        let mut catalog =
446            Self::open_debug_catalog(persist_client.clone(), organization_id, &bootstrap_args)
447                .await
448                .expect("can open debug catalog");
449
450        // Replace `storage` in `catalog` with one that doesn't think bootstrap is over.
451        let now = SYSTEM_TIME.clone();
452        let openable_storage = TestCatalogStateBuilder::new(persist_client)
453            .with_organization_id(organization_id)
454            .with_default_deploy_generation()
455            .build()
456            .await
457            .expect("can create durable catalog");
458        let mut storage = openable_storage
459            .open(now().into(), &bootstrap_args)
460            .await
461            .expect("can open durable catalog")
462            .0;
463        // Drain updates.
464        let _ = storage
465            .sync_to_current_updates()
466            .await
467            .expect("can sync to current updates");
468        catalog.storage = Arc::new(tokio::sync::Mutex::new(storage));
469
470        f(catalog).await
471    }
472
473    /// Opens a debug catalog.
474    ///
475    /// See [`Catalog::with_debug`].
476    pub async fn open_debug_catalog(
477        persist_client: PersistClient,
478        organization_id: Uuid,
479        bootstrap_args: &BootstrapArgs,
480    ) -> Result<Catalog, anyhow::Error> {
481        let now = SYSTEM_TIME.clone();
482        let environment_id = None;
483        let openable_storage = TestCatalogStateBuilder::new(persist_client.clone())
484            .with_organization_id(organization_id)
485            .with_default_deploy_generation()
486            .build()
487            .await?;
488        let storage = openable_storage.open(now().into(), bootstrap_args).await?.0;
489        let system_parameter_defaults = BTreeMap::default();
490        Self::open_debug_catalog_inner(
491            persist_client,
492            storage,
493            now,
494            environment_id,
495            &DUMMY_BUILD_INFO,
496            system_parameter_defaults,
497            bootstrap_args,
498            None,
499        )
500        .await
501    }
502
503    /// Opens a read only debug persist backed catalog defined by `persist_client` and
504    /// `organization_id`.
505    ///
506    /// See [`Catalog::with_debug`].
507    pub async fn open_debug_read_only_catalog(
508        persist_client: PersistClient,
509        organization_id: Uuid,
510        bootstrap_args: &BootstrapArgs,
511    ) -> Result<Catalog, anyhow::Error> {
512        let now = SYSTEM_TIME.clone();
513        let environment_id = None;
514        let openable_storage = TestCatalogStateBuilder::new(persist_client.clone())
515            .with_organization_id(organization_id)
516            .build()
517            .await?;
518        let storage = openable_storage
519            .open_read_only(&test_bootstrap_args())
520            .await?;
521        let system_parameter_defaults = BTreeMap::default();
522        Self::open_debug_catalog_inner(
523            persist_client,
524            storage,
525            now,
526            environment_id,
527            &DUMMY_BUILD_INFO,
528            system_parameter_defaults,
529            bootstrap_args,
530            None,
531        )
532        .await
533    }
534
535    /// Opens a read only debug persist backed catalog defined by `persist_client` and
536    /// `organization_id`.
537    ///
538    /// See [`Catalog::with_debug`].
539    pub async fn open_debug_read_only_persist_catalog_config(
540        persist_client: PersistClient,
541        now: NowFn,
542        environment_id: EnvironmentId,
543        system_parameter_defaults: BTreeMap<String, String>,
544        build_info: &'static BuildInfo,
545        bootstrap_args: &BootstrapArgs,
546        enable_expression_cache_override: Option<bool>,
547    ) -> Result<Catalog, anyhow::Error> {
548        let openable_storage = TestCatalogStateBuilder::new(persist_client.clone())
549            .with_organization_id(environment_id.organization_id())
550            .with_version(
551                build_info
552                    .version
553                    .parse()
554                    .expect("build version is parseable"),
555            )
556            .build()
557            .await?;
558        let storage = openable_storage.open_read_only(bootstrap_args).await?;
559        Self::open_debug_catalog_inner(
560            persist_client,
561            storage,
562            now,
563            Some(environment_id),
564            build_info,
565            system_parameter_defaults,
566            bootstrap_args,
567            enable_expression_cache_override,
568        )
569        .await
570    }
571
572    async fn open_debug_catalog_inner(
573        persist_client: PersistClient,
574        storage: Box<dyn DurableCatalogState>,
575        now: NowFn,
576        environment_id: Option<EnvironmentId>,
577        build_info: &'static BuildInfo,
578        system_parameter_defaults: BTreeMap<String, String>,
579        bootstrap_args: &BootstrapArgs,
580        enable_expression_cache_override: Option<bool>,
581    ) -> Result<Catalog, anyhow::Error> {
582        let metrics_registry = &MetricsRegistry::new();
583        let secrets_reader = Arc::new(InMemorySecretsController::new());
584        // Used as a lower boundary of the boot_ts, but it's ok to use now() for
585        // debugging/testing.
586        let previous_ts = now().into();
587        let replica_size = &bootstrap_args.default_cluster_replica_size;
588        let read_only = false;
589
590        let OpenCatalogResult {
591            catalog,
592            migrated_storage_collections_0dt: _,
593            new_builtin_collections: _,
594            builtin_table_updates: _,
595            cached_global_exprs: _,
596            uncached_local_exprs: _,
597        } = Catalog::open(Config {
598            storage,
599            metrics_registry,
600            state: StateConfig {
601                unsafe_mode: true,
602                all_features: false,
603                build_info,
604                deploy_generation: 0,
605                environment_id: environment_id.unwrap_or_else(EnvironmentId::for_tests),
606                read_only,
607                now,
608                boot_ts: previous_ts,
609                skip_migrations: true,
610                cluster_replica_sizes: bootstrap_args.cluster_replica_size_map.clone(),
611                builtin_system_cluster_config: BootstrapBuiltinClusterConfig {
612                    size: replica_size.clone(),
613                    replication_factor: SYSTEM_CLUSTER_DEFAULT_REPLICATION_FACTOR,
614                },
615                builtin_catalog_server_cluster_config: BootstrapBuiltinClusterConfig {
616                    size: replica_size.clone(),
617                    replication_factor: CATALOG_SERVER_CLUSTER_DEFAULT_REPLICATION_FACTOR,
618                },
619                builtin_probe_cluster_config: BootstrapBuiltinClusterConfig {
620                    size: replica_size.clone(),
621                    replication_factor: PROBE_CLUSTER_DEFAULT_REPLICATION_FACTOR,
622                },
623                builtin_support_cluster_config: BootstrapBuiltinClusterConfig {
624                    size: replica_size.clone(),
625                    replication_factor: SUPPORT_CLUSTER_DEFAULT_REPLICATION_FACTOR,
626                },
627                builtin_analytics_cluster_config: BootstrapBuiltinClusterConfig {
628                    size: replica_size.clone(),
629                    replication_factor: ANALYTICS_CLUSTER_DEFAULT_REPLICATION_FACTOR,
630                },
631                system_parameter_defaults,
632                remote_system_parameters: None,
633                availability_zones: vec![],
634                egress_addresses: vec![],
635                aws_principal_context: None,
636                aws_privatelink_availability_zones: None,
637                http_host_name: None,
638                connection_context: ConnectionContext::for_tests(secrets_reader),
639                builtin_item_migration_config: BuiltinItemMigrationConfig {
640                    persist_client: persist_client.clone(),
641                    read_only,
642                    force_migration: None,
643                },
644                persist_client,
645                enable_expression_cache_override,
646                helm_chart_version: None,
647                external_login_password_mz_system: None,
648                license_key: ValidatedLicenseKey::for_tests(),
649            },
650        })
651        .await?;
652        Ok(catalog)
653    }
654
655    pub fn for_session<'a>(&'a self, session: &'a Session) -> ConnCatalog<'a> {
656        self.state.for_session(session)
657    }
658
659    pub fn for_sessionless_user(&self, role_id: RoleId) -> ConnCatalog<'_> {
660        self.state.for_sessionless_user(role_id)
661    }
662
663    pub fn for_system_session(&self) -> ConnCatalog<'_> {
664        self.state.for_system_session()
665    }
666
667    async fn storage<'a>(
668        &'a self,
669    ) -> MutexGuard<'a, Box<dyn mz_catalog::durable::DurableCatalogState>> {
670        self.storage.lock().await
671    }
672
673    pub async fn current_upper(&self) -> mz_repr::Timestamp {
674        self.storage().await.current_upper().await
675    }
676
677    pub async fn allocate_user_id(
678        &self,
679        commit_ts: mz_repr::Timestamp,
680    ) -> Result<(CatalogItemId, GlobalId), Error> {
681        self.storage()
682            .await
683            .allocate_user_id(commit_ts)
684            .await
685            .maybe_terminate("allocating user ids")
686            .err_into()
687    }
688
689    /// Allocate `amount` many user IDs. See [`DurableCatalogState::allocate_user_ids`].
690    pub async fn allocate_user_ids(
691        &self,
692        amount: u64,
693        commit_ts: mz_repr::Timestamp,
694    ) -> Result<Vec<(CatalogItemId, GlobalId)>, Error> {
695        self.storage()
696            .await
697            .allocate_user_ids(amount, commit_ts)
698            .await
699            .maybe_terminate("allocating user ids")
700            .err_into()
701    }
702
703    pub async fn allocate_user_id_for_test(&self) -> Result<(CatalogItemId, GlobalId), Error> {
704        let commit_ts = self.storage().await.current_upper().await;
705        self.allocate_user_id(commit_ts).await
706    }
707
708    /// Get the next user item ID without allocating it.
709    pub async fn get_next_user_item_id(&self) -> Result<u64, Error> {
710        self.storage()
711            .await
712            .get_next_user_item_id()
713            .await
714            .err_into()
715    }
716
717    #[cfg(test)]
718    pub async fn allocate_system_id(
719        &self,
720        commit_ts: mz_repr::Timestamp,
721    ) -> Result<(CatalogItemId, GlobalId), Error> {
722        use mz_ore::collections::CollectionExt;
723
724        let mut storage = self.storage().await;
725        let mut txn = storage.transaction().await?;
726        let id = txn
727            .allocate_system_item_ids(1)
728            .maybe_terminate("allocating system ids")?
729            .into_element();
730        // Drain transaction.
731        let _ = txn.get_and_commit_op_updates();
732        txn.commit(commit_ts).await?;
733        Ok(id)
734    }
735
736    /// Get the next system item ID without allocating it.
737    pub async fn get_next_system_item_id(&self) -> Result<u64, Error> {
738        self.storage()
739            .await
740            .get_next_system_item_id()
741            .await
742            .err_into()
743    }
744
745    pub async fn allocate_user_cluster_id(
746        &self,
747        commit_ts: mz_repr::Timestamp,
748    ) -> Result<ClusterId, Error> {
749        self.storage()
750            .await
751            .allocate_user_cluster_id(commit_ts)
752            .await
753            .maybe_terminate("allocating user cluster ids")
754            .err_into()
755    }
756
757    /// Get the next system replica id without allocating it.
758    pub async fn get_next_system_replica_id(&self) -> Result<u64, Error> {
759        self.storage()
760            .await
761            .get_next_system_replica_id()
762            .await
763            .err_into()
764    }
765
766    /// Get the next user replica id without allocating it.
767    pub async fn get_next_user_replica_id(&self) -> Result<u64, Error> {
768        self.storage()
769            .await
770            .get_next_user_replica_id()
771            .await
772            .err_into()
773    }
774
775    pub fn resolve_database(&self, database_name: &str) -> Result<&Database, SqlCatalogError> {
776        self.state.resolve_database(database_name)
777    }
778
779    pub fn resolve_schema(
780        &self,
781        current_database: Option<&DatabaseId>,
782        database_name: Option<&str>,
783        schema_name: &str,
784        conn_id: &ConnectionId,
785    ) -> Result<&Schema, SqlCatalogError> {
786        self.state
787            .resolve_schema(current_database, database_name, schema_name, conn_id)
788    }
789
790    pub fn resolve_schema_in_database(
791        &self,
792        database_spec: &ResolvedDatabaseSpecifier,
793        schema_name: &str,
794        conn_id: &ConnectionId,
795    ) -> Result<&Schema, SqlCatalogError> {
796        self.state
797            .resolve_schema_in_database(database_spec, schema_name, conn_id)
798    }
799
800    pub fn resolve_replica_in_cluster(
801        &self,
802        cluster_id: &ClusterId,
803        replica_name: &str,
804    ) -> Result<&ClusterReplica, SqlCatalogError> {
805        self.state
806            .resolve_replica_in_cluster(cluster_id, replica_name)
807    }
808
809    pub fn resolve_system_schema(&self, name: &'static str) -> SchemaId {
810        self.state.resolve_system_schema(name)
811    }
812
813    pub fn resolve_search_path(
814        &self,
815        session: &Session,
816    ) -> Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)> {
817        self.state.resolve_search_path(session)
818    }
819
820    /// Resolves `name` to a non-function [`CatalogEntry`].
821    pub fn resolve_entry(
822        &self,
823        current_database: Option<&DatabaseId>,
824        search_path: &Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)>,
825        name: &PartialItemName,
826        conn_id: &ConnectionId,
827    ) -> Result<&CatalogEntry, SqlCatalogError> {
828        self.state
829            .resolve_entry(current_database, search_path, name, conn_id)
830    }
831
832    /// Resolves a `BuiltinTable`.
833    pub fn resolve_builtin_table(&self, builtin: &'static BuiltinTable) -> CatalogItemId {
834        self.state.resolve_builtin_table(builtin)
835    }
836
837    /// Resolves a `BuiltinLog`.
838    pub fn resolve_builtin_log(&self, builtin: &'static BuiltinLog) -> CatalogItemId {
839        self.state.resolve_builtin_log(builtin).0
840    }
841
842    /// Resolves a `BuiltinSource`.
843    pub fn resolve_builtin_storage_collection(
844        &self,
845        builtin: &'static BuiltinSource,
846    ) -> CatalogItemId {
847        self.state.resolve_builtin_source(builtin)
848    }
849
850    /// Resolves `name` to a function [`CatalogEntry`].
851    pub fn resolve_function(
852        &self,
853        current_database: Option<&DatabaseId>,
854        search_path: &Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)>,
855        name: &PartialItemName,
856        conn_id: &ConnectionId,
857    ) -> Result<&CatalogEntry, SqlCatalogError> {
858        self.state
859            .resolve_function(current_database, search_path, name, conn_id)
860    }
861
862    /// Resolves `name` to a type [`CatalogEntry`].
863    pub fn resolve_type(
864        &self,
865        current_database: Option<&DatabaseId>,
866        search_path: &Vec<(ResolvedDatabaseSpecifier, SchemaSpecifier)>,
867        name: &PartialItemName,
868        conn_id: &ConnectionId,
869    ) -> Result<&CatalogEntry, SqlCatalogError> {
870        self.state
871            .resolve_type(current_database, search_path, name, conn_id)
872    }
873
874    pub fn resolve_cluster(&self, name: &str) -> Result<&Cluster, SqlCatalogError> {
875        self.state.resolve_cluster(name)
876    }
877
878    /// Resolves a [`Cluster`] for a [`BuiltinCluster`].
879    ///
880    /// # Panics
881    /// * If the [`BuiltinCluster`] doesn't exist.
882    ///
883    pub fn resolve_builtin_cluster(&self, cluster: &BuiltinCluster) -> &Cluster {
884        self.state.resolve_builtin_cluster(cluster)
885    }
886
887    pub fn get_mz_catalog_server_cluster_id(&self) -> &ClusterId {
888        &self.resolve_builtin_cluster(&MZ_CATALOG_SERVER_CLUSTER).id
889    }
890
891    /// Resolves a [`Cluster`] for a TargetCluster.
892    pub fn resolve_target_cluster(
893        &self,
894        target_cluster: TargetCluster,
895        session: &Session,
896    ) -> Result<&Cluster, AdapterError> {
897        match target_cluster {
898            TargetCluster::CatalogServer => {
899                Ok(self.resolve_builtin_cluster(&MZ_CATALOG_SERVER_CLUSTER))
900            }
901            TargetCluster::Active => self.active_cluster(session),
902            TargetCluster::Transaction(cluster_id) => self
903                .try_get_cluster(cluster_id)
904                .ok_or(AdapterError::ConcurrentClusterDrop),
905        }
906    }
907
908    pub fn active_cluster(&self, session: &Session) -> Result<&Cluster, AdapterError> {
909        // TODO(benesch): this check here is not sufficiently protective. It'd
910        // be very easy for a code path to accidentally avoid this check by
911        // calling `resolve_cluster(session.vars().cluster())`.
912        if session.user().name != SYSTEM_USER.name
913            && session.user().name != SUPPORT_USER.name
914            && session.vars().cluster() == SYSTEM_USER.name
915        {
916            coord_bail!(
917                "system cluster '{}' cannot execute user queries",
918                SYSTEM_USER.name
919            );
920        }
921        let cluster = self.resolve_cluster(session.vars().cluster())?;
922        Ok(cluster)
923    }
924
925    pub fn state(&self) -> &CatalogState {
926        &self.state
927    }
928
929    pub fn resolve_full_name(
930        &self,
931        name: &QualifiedItemName,
932        conn_id: Option<&ConnectionId>,
933    ) -> FullItemName {
934        self.state.resolve_full_name(name, conn_id)
935    }
936
937    pub fn try_get_entry(&self, id: &CatalogItemId) -> Option<&CatalogEntry> {
938        self.state.try_get_entry(id)
939    }
940
941    pub fn try_get_entry_by_global_id(&self, id: &GlobalId) -> Option<&CatalogEntry> {
942        self.state.try_get_entry_by_global_id(id)
943    }
944
945    pub fn get_entry(&self, id: &CatalogItemId) -> &CatalogEntry {
946        self.state.get_entry(id)
947    }
948
949    pub fn get_entry_by_global_id(&self, id: &GlobalId) -> CatalogCollectionEntry {
950        self.state.get_entry_by_global_id(id)
951    }
952
953    pub fn get_global_ids<'a>(
954        &'a self,
955        id: &CatalogItemId,
956    ) -> impl Iterator<Item = GlobalId> + use<'a> {
957        self.get_entry(id).global_ids()
958    }
959
960    pub fn resolve_item_id(&self, id: &GlobalId) -> CatalogItemId {
961        self.get_entry_by_global_id(id).id()
962    }
963
964    pub fn try_resolve_item_id(&self, id: &GlobalId) -> Option<CatalogItemId> {
965        let item = self.try_get_entry_by_global_id(id)?;
966        Some(item.id())
967    }
968
969    pub fn get_schema(
970        &self,
971        database_spec: &ResolvedDatabaseSpecifier,
972        schema_spec: &SchemaSpecifier,
973        conn_id: &ConnectionId,
974    ) -> &Schema {
975        self.state.get_schema(database_spec, schema_spec, conn_id)
976    }
977
978    pub fn try_get_schema(
979        &self,
980        database_spec: &ResolvedDatabaseSpecifier,
981        schema_spec: &SchemaSpecifier,
982        conn_id: &ConnectionId,
983    ) -> Option<&Schema> {
984        self.state
985            .try_get_schema(database_spec, schema_spec, conn_id)
986    }
987
988    pub fn get_mz_catalog_schema_id(&self) -> SchemaId {
989        self.state.get_mz_catalog_schema_id()
990    }
991
992    pub fn get_pg_catalog_schema_id(&self) -> SchemaId {
993        self.state.get_pg_catalog_schema_id()
994    }
995
996    pub fn get_information_schema_id(&self) -> SchemaId {
997        self.state.get_information_schema_id()
998    }
999
1000    pub fn get_mz_internal_schema_id(&self) -> SchemaId {
1001        self.state.get_mz_internal_schema_id()
1002    }
1003
1004    pub fn get_mz_introspection_schema_id(&self) -> SchemaId {
1005        self.state.get_mz_introspection_schema_id()
1006    }
1007
1008    pub fn get_mz_unsafe_schema_id(&self) -> SchemaId {
1009        self.state.get_mz_unsafe_schema_id()
1010    }
1011
1012    pub fn system_schema_ids(&self) -> impl Iterator<Item = SchemaId> + '_ {
1013        self.state.system_schema_ids()
1014    }
1015
1016    pub fn get_database(&self, id: &DatabaseId) -> &Database {
1017        self.state.get_database(id)
1018    }
1019
1020    pub fn try_get_role(&self, id: &RoleId) -> Option<&Role> {
1021        self.state.try_get_role(id)
1022    }
1023
1024    pub fn get_role(&self, id: &RoleId) -> &Role {
1025        self.state.get_role(id)
1026    }
1027
1028    pub fn try_get_role_by_name(&self, role_name: &str) -> Option<&Role> {
1029        self.state.try_get_role_by_name(role_name)
1030    }
1031
1032    pub fn try_get_role_auth_by_id(&self, id: &RoleId) -> Option<&RoleAuth> {
1033        self.state.try_get_role_auth_by_id(id)
1034    }
1035
1036    /// Creates a new schema in the `Catalog` for temporary items
1037    /// indicated by the TEMPORARY or TEMP keywords.
1038    pub fn create_temporary_schema(
1039        &mut self,
1040        conn_id: &ConnectionId,
1041        owner_id: RoleId,
1042    ) -> Result<(), Error> {
1043        self.state.create_temporary_schema(conn_id, owner_id)
1044    }
1045
1046    fn item_exists_in_temp_schemas(&self, conn_id: &ConnectionId, item_name: &str) -> bool {
1047        // Temporary schemas are created lazily, so it's valid for one to not exist yet.
1048        self.state
1049            .temporary_schemas
1050            .get(conn_id)
1051            .map(|schema| schema.items.contains_key(item_name))
1052            .unwrap_or(false)
1053    }
1054
1055    /// Drops schema for connection if it exists. Returns an error if it exists and has items.
1056    /// Returns Ok if conn_id's temp schema does not exist.
1057    pub fn drop_temporary_schema(&mut self, conn_id: &ConnectionId) -> Result<(), Error> {
1058        let Some(schema) = self.state.temporary_schemas.remove(conn_id) else {
1059            return Ok(());
1060        };
1061        if !schema.items.is_empty() {
1062            return Err(Error::new(ErrorKind::SchemaNotEmpty(MZ_TEMP_SCHEMA.into())));
1063        }
1064        Ok(())
1065    }
1066
1067    pub(crate) fn object_dependents(
1068        &self,
1069        object_ids: &Vec<ObjectId>,
1070        conn_id: &ConnectionId,
1071    ) -> Vec<ObjectId> {
1072        let mut seen = BTreeSet::new();
1073        self.state.object_dependents(object_ids, conn_id, &mut seen)
1074    }
1075
1076    fn full_name_detail(name: &FullItemName) -> FullNameV1 {
1077        FullNameV1 {
1078            database: name.database.to_string(),
1079            schema: name.schema.clone(),
1080            item: name.item.clone(),
1081        }
1082    }
1083
1084    pub fn find_available_cluster_name(&self, name: &str) -> String {
1085        let mut i = 0;
1086        let mut candidate = name.to_string();
1087        while self.state.clusters_by_name.contains_key(&candidate) {
1088            i += 1;
1089            candidate = format!("{}{}", name, i);
1090        }
1091        candidate
1092    }
1093
1094    pub fn get_role_allowed_cluster_sizes(&self, role_id: &Option<RoleId>) -> Vec<String> {
1095        if role_id == &Some(MZ_SYSTEM_ROLE_ID) {
1096            self.cluster_replica_sizes()
1097                .enabled_allocations()
1098                .map(|a| a.0.to_owned())
1099                .collect::<Vec<_>>()
1100        } else {
1101            self.system_config().allowed_cluster_replica_sizes()
1102        }
1103    }
1104
1105    pub fn concretize_replica_location(
1106        &self,
1107        location: mz_catalog::durable::ReplicaLocation,
1108        allowed_sizes: &Vec<String>,
1109        allowed_availability_zones: Option<&[String]>,
1110    ) -> Result<ReplicaLocation, Error> {
1111        self.state
1112            .concretize_replica_location(location, allowed_sizes, allowed_availability_zones)
1113    }
1114
1115    pub(crate) fn ensure_valid_replica_size(
1116        &self,
1117        allowed_sizes: &[String],
1118        size: &String,
1119    ) -> Result<(), Error> {
1120        self.state.ensure_valid_replica_size(allowed_sizes, size)
1121    }
1122
1123    pub fn cluster_replica_sizes(&self) -> &ClusterReplicaSizeMap {
1124        &self.state.cluster_replica_sizes
1125    }
1126
1127    /// Returns the privileges of an object by its ID.
1128    pub fn get_privileges(
1129        &self,
1130        id: &SystemObjectId,
1131        conn_id: &ConnectionId,
1132    ) -> Option<&PrivilegeMap> {
1133        match id {
1134            SystemObjectId::Object(id) => match id {
1135                ObjectId::Cluster(id) => Some(self.get_cluster(*id).privileges()),
1136                ObjectId::Database(id) => Some(self.get_database(id).privileges()),
1137                ObjectId::Schema((database_spec, schema_spec)) => Some(
1138                    self.get_schema(database_spec, schema_spec, conn_id)
1139                        .privileges(),
1140                ),
1141                ObjectId::Item(id) => Some(self.get_entry(id).privileges()),
1142                ObjectId::ClusterReplica(_) | ObjectId::Role(_) => None,
1143                ObjectId::NetworkPolicy(id) => Some(self.get_network_policy(*id).privileges()),
1144            },
1145            SystemObjectId::System => Some(&self.state.system_privileges),
1146        }
1147    }
1148
1149    #[mz_ore::instrument(level = "debug")]
1150    pub async fn confirm_leadership(&self) -> Result<(), AdapterError> {
1151        Ok(self.storage().await.confirm_leadership().await?)
1152    }
1153
1154    /// Return the ids of all log sources the given object depends on.
1155    pub fn introspection_dependencies(&self, id: CatalogItemId) -> Vec<CatalogItemId> {
1156        self.state.introspection_dependencies(id)
1157    }
1158
1159    /// Serializes the catalog's in-memory state.
1160    ///
1161    /// There are no guarantees about the format of the serialized state, except
1162    /// that the serialized state for two identical catalogs will compare
1163    /// identically.
1164    pub fn dump(&self) -> Result<CatalogDump, Error> {
1165        Ok(CatalogDump::new(self.state.dump(None)?))
1166    }
1167
1168    /// Checks the [`Catalog`]s internal consistency.
1169    ///
1170    /// Returns a JSON object describing the inconsistencies, if there are any.
1171    pub fn check_consistency(&self) -> Result<(), serde_json::Value> {
1172        self.state.check_consistency().map_err(|inconsistencies| {
1173            serde_json::to_value(inconsistencies).unwrap_or_else(|_| {
1174                serde_json::Value::String("failed to serialize inconsistencies".to_string())
1175            })
1176        })
1177    }
1178
1179    pub fn config(&self) -> &mz_sql::catalog::CatalogConfig {
1180        self.state.config()
1181    }
1182
1183    pub fn entries(&self) -> impl Iterator<Item = &CatalogEntry> {
1184        self.state.entry_by_id.values()
1185    }
1186
1187    pub fn user_connections(&self) -> impl Iterator<Item = &CatalogEntry> {
1188        self.entries()
1189            .filter(|entry| entry.is_connection() && entry.id().is_user())
1190    }
1191
1192    pub fn user_tables(&self) -> impl Iterator<Item = &CatalogEntry> {
1193        self.entries()
1194            .filter(|entry| entry.is_table() && entry.id().is_user())
1195    }
1196
1197    pub fn user_sources(&self) -> impl Iterator<Item = &CatalogEntry> {
1198        self.entries()
1199            .filter(|entry| entry.is_source() && entry.id().is_user())
1200    }
1201
1202    pub fn user_sinks(&self) -> impl Iterator<Item = &CatalogEntry> {
1203        self.entries()
1204            .filter(|entry| entry.is_sink() && entry.id().is_user())
1205    }
1206
1207    pub fn user_materialized_views(&self) -> impl Iterator<Item = &CatalogEntry> {
1208        self.entries()
1209            .filter(|entry| entry.is_materialized_view() && entry.id().is_user())
1210    }
1211
1212    pub fn user_secrets(&self) -> impl Iterator<Item = &CatalogEntry> {
1213        self.entries()
1214            .filter(|entry| entry.is_secret() && entry.id().is_user())
1215    }
1216
1217    pub fn get_network_policy(&self, network_policy_id: NetworkPolicyId) -> &NetworkPolicy {
1218        self.state.get_network_policy(&network_policy_id)
1219    }
1220
1221    pub fn get_network_policy_by_name(&self, name: &str) -> Option<&NetworkPolicy> {
1222        self.state.try_get_network_policy_by_name(name)
1223    }
1224
1225    pub fn clusters(&self) -> impl Iterator<Item = &Cluster> {
1226        self.state.clusters_by_id.values()
1227    }
1228
1229    pub fn get_cluster(&self, cluster_id: ClusterId) -> &Cluster {
1230        self.state.get_cluster(cluster_id)
1231    }
1232
1233    pub fn try_get_cluster(&self, cluster_id: ClusterId) -> Option<&Cluster> {
1234        self.state.try_get_cluster(cluster_id)
1235    }
1236
1237    pub fn user_clusters(&self) -> impl Iterator<Item = &Cluster> {
1238        self.clusters().filter(|cluster| cluster.id.is_user())
1239    }
1240
1241    pub fn get_cluster_replica(
1242        &self,
1243        cluster_id: ClusterId,
1244        replica_id: ReplicaId,
1245    ) -> &ClusterReplica {
1246        self.state.get_cluster_replica(cluster_id, replica_id)
1247    }
1248
1249    pub fn try_get_cluster_replica(
1250        &self,
1251        cluster_id: ClusterId,
1252        replica_id: ReplicaId,
1253    ) -> Option<&ClusterReplica> {
1254        self.state.try_get_cluster_replica(cluster_id, replica_id)
1255    }
1256
1257    pub fn user_cluster_replicas(&self) -> impl Iterator<Item = &ClusterReplica> {
1258        self.user_clusters()
1259            .flat_map(|cluster| cluster.user_replicas())
1260    }
1261
1262    pub fn databases(&self) -> impl Iterator<Item = &Database> {
1263        self.state.database_by_id.values()
1264    }
1265
1266    pub fn user_roles(&self) -> impl Iterator<Item = &Role> {
1267        self.state
1268            .roles_by_id
1269            .values()
1270            .filter(|role| role.is_user())
1271    }
1272
1273    pub fn user_continual_tasks(&self) -> impl Iterator<Item = &CatalogEntry> {
1274        self.entries()
1275            .filter(|entry| entry.is_continual_task() && entry.id().is_user())
1276    }
1277
1278    pub fn user_network_policies(&self) -> impl Iterator<Item = &NetworkPolicy> {
1279        self.state
1280            .network_policies_by_id
1281            .iter()
1282            .filter(|(id, _)| id.is_user())
1283            .map(|(_, policy)| policy)
1284    }
1285
1286    pub fn system_privileges(&self) -> &PrivilegeMap {
1287        &self.state.system_privileges
1288    }
1289
1290    pub fn default_privileges(
1291        &self,
1292    ) -> impl Iterator<
1293        Item = (
1294            &DefaultPrivilegeObject,
1295            impl Iterator<Item = &DefaultPrivilegeAclItem>,
1296        ),
1297    > {
1298        self.state.default_privileges.iter()
1299    }
1300
1301    pub fn pack_item_update(&self, id: CatalogItemId, diff: Diff) -> Vec<BuiltinTableUpdate> {
1302        self.state
1303            .resolve_builtin_table_updates(self.state.pack_item_update(id, diff))
1304    }
1305
1306    pub fn pack_storage_usage_update(
1307        &self,
1308        event: VersionedStorageUsage,
1309        diff: Diff,
1310    ) -> BuiltinTableUpdate {
1311        self.state
1312            .resolve_builtin_table_update(self.state.pack_storage_usage_update(event, diff))
1313    }
1314
1315    pub fn system_config(&self) -> &SystemVars {
1316        self.state.system_config()
1317    }
1318
1319    pub fn system_config_mut(&mut self) -> &mut SystemVars {
1320        self.state.system_config_mut()
1321    }
1322
1323    pub fn ensure_not_reserved_role(&self, role_id: &RoleId) -> Result<(), Error> {
1324        self.state.ensure_not_reserved_role(role_id)
1325    }
1326
1327    pub fn ensure_grantable_role(&self, role_id: &RoleId) -> Result<(), Error> {
1328        self.state.ensure_grantable_role(role_id)
1329    }
1330
1331    pub fn ensure_not_system_role(&self, role_id: &RoleId) -> Result<(), Error> {
1332        self.state.ensure_not_system_role(role_id)
1333    }
1334
1335    pub fn ensure_not_predefined_role(&self, role_id: &RoleId) -> Result<(), Error> {
1336        self.state.ensure_not_predefined_role(role_id)
1337    }
1338
1339    pub fn ensure_not_reserved_network_policy(
1340        &self,
1341        network_policy_id: &NetworkPolicyId,
1342    ) -> Result<(), Error> {
1343        self.state
1344            .ensure_not_reserved_network_policy(network_policy_id)
1345    }
1346
1347    pub fn ensure_not_reserved_object(
1348        &self,
1349        object_id: &ObjectId,
1350        conn_id: &ConnectionId,
1351    ) -> Result<(), Error> {
1352        match object_id {
1353            ObjectId::Cluster(cluster_id) => {
1354                if cluster_id.is_system() {
1355                    let cluster = self.get_cluster(*cluster_id);
1356                    Err(Error::new(ErrorKind::ReadOnlyCluster(
1357                        cluster.name().to_string(),
1358                    )))
1359                } else {
1360                    Ok(())
1361                }
1362            }
1363            ObjectId::ClusterReplica((cluster_id, replica_id)) => {
1364                if replica_id.is_system() {
1365                    let replica = self.get_cluster_replica(*cluster_id, *replica_id);
1366                    Err(Error::new(ErrorKind::ReadOnlyClusterReplica(
1367                        replica.name().to_string(),
1368                    )))
1369                } else {
1370                    Ok(())
1371                }
1372            }
1373            ObjectId::Database(database_id) => {
1374                if database_id.is_system() {
1375                    let database = self.get_database(database_id);
1376                    Err(Error::new(ErrorKind::ReadOnlyDatabase(
1377                        database.name().to_string(),
1378                    )))
1379                } else {
1380                    Ok(())
1381                }
1382            }
1383            ObjectId::Schema((database_spec, schema_spec)) => {
1384                if schema_spec.is_system() {
1385                    let schema = self.get_schema(database_spec, schema_spec, conn_id);
1386                    Err(Error::new(ErrorKind::ReadOnlySystemSchema(
1387                        schema.name().schema.clone(),
1388                    )))
1389                } else {
1390                    Ok(())
1391                }
1392            }
1393            ObjectId::Role(role_id) => self.ensure_not_reserved_role(role_id),
1394            ObjectId::Item(item_id) => {
1395                if item_id.is_system() {
1396                    let item = self.get_entry(item_id);
1397                    let name = self.resolve_full_name(item.name(), Some(conn_id));
1398                    Err(Error::new(ErrorKind::ReadOnlyItem(name.to_string())))
1399                } else {
1400                    Ok(())
1401                }
1402            }
1403            ObjectId::NetworkPolicy(network_policy_id) => {
1404                self.ensure_not_reserved_network_policy(network_policy_id)
1405            }
1406        }
1407    }
1408
1409    /// See [`CatalogState::deserialize_plan_with_enable_for_item_parsing`].
1410    pub(crate) fn deserialize_plan_with_enable_for_item_parsing(
1411        &mut self,
1412        create_sql: &str,
1413        force_if_exists_skip: bool,
1414    ) -> Result<(Plan, ResolvedIds), AdapterError> {
1415        self.state
1416            .deserialize_plan_with_enable_for_item_parsing(create_sql, force_if_exists_skip)
1417    }
1418
1419    /// Cache global and, optionally, local expressions for the given `GlobalId`.
1420    ///
1421    /// This takes the required plans and metainfo from the catalog and expects that they were
1422    /// previously stored via [`Catalog::set_optimized_plan`], [`Catalog::set_physical_plan`], and
1423    /// [`Catalog::set_dataflow_metainfo`].
1424    pub(crate) fn cache_expressions(
1425        &self,
1426        id: GlobalId,
1427        local_mir: Option<OptimizedMirRelationExpr>,
1428        optimizer_features: OptimizerFeatures,
1429    ) {
1430        let Some(mut global_mir) = self.try_get_optimized_plan(&id).cloned() else {
1431            soft_panic_or_log!("optimized plan missing for ID {id}");
1432            return;
1433        };
1434        let Some(mut physical_plan) = self.try_get_physical_plan(&id).cloned() else {
1435            soft_panic_or_log!("physical plan missing for ID {id}");
1436            return;
1437        };
1438        let Some(dataflow_metainfos) = self.try_get_dataflow_metainfo(&id).cloned() else {
1439            soft_panic_or_log!("dataflow metainfo missing for ID {id}");
1440            return;
1441        };
1442
1443        // Make sure we're not caching the result of timestamp selection, as it will almost
1444        // certainly be wrong if we re-install the dataflow at a later time.
1445        global_mir.as_of = None;
1446        global_mir.until = Default::default();
1447        physical_plan.as_of = None;
1448        physical_plan.until = Default::default();
1449
1450        let mut local_exprs = Vec::new();
1451        if let Some(local_mir) = local_mir {
1452            local_exprs.push((
1453                id,
1454                LocalExpressions {
1455                    local_mir,
1456                    optimizer_features: optimizer_features.clone(),
1457                },
1458            ));
1459        }
1460        let global_exprs = vec![(
1461            id,
1462            GlobalExpressions {
1463                global_mir,
1464                physical_plan,
1465                dataflow_metainfos,
1466                optimizer_features,
1467            },
1468        )];
1469        let _fut = self.update_expression_cache(local_exprs, global_exprs, Default::default());
1470    }
1471
1472    pub(crate) fn update_expression_cache<'a, 'b>(
1473        &'a self,
1474        new_local_expressions: Vec<(GlobalId, LocalExpressions)>,
1475        new_global_expressions: Vec<(GlobalId, GlobalExpressions)>,
1476        invalidate_ids: BTreeSet<GlobalId>,
1477    ) -> BoxFuture<'b, ()> {
1478        if let Some(expr_cache) = &self.expr_cache_handle {
1479            expr_cache
1480                .update(
1481                    new_local_expressions,
1482                    new_global_expressions,
1483                    invalidate_ids,
1484                )
1485                .boxed()
1486        } else {
1487            async {}.boxed()
1488        }
1489    }
1490
1491    /// Listen for and apply all unconsumed updates to the durable catalog state.
1492    // TODO(jkosh44) When this method is actually used outside of a test we can remove the
1493    // `#[cfg(test)]` annotation.
1494    #[cfg(test)]
1495    async fn sync_to_current_updates(
1496        &mut self,
1497    ) -> Result<
1498        (
1499            Vec<BuiltinTableUpdate<&'static BuiltinTable>>,
1500            Vec<ParsedStateUpdate>,
1501        ),
1502        CatalogError,
1503    > {
1504        let updates = self.storage().await.sync_to_current_updates().await?;
1505        let (builtin_table_updates, catalog_updates) = self
1506            .state
1507            .apply_updates(updates, &mut state::LocalExpressionCache::Closed)
1508            .await;
1509        Ok((builtin_table_updates, catalog_updates))
1510    }
1511}
1512
1513pub fn is_reserved_name(name: &str) -> bool {
1514    BUILTIN_PREFIXES
1515        .iter()
1516        .any(|prefix| name.starts_with(prefix))
1517}
1518
1519pub fn is_reserved_role_name(name: &str) -> bool {
1520    is_reserved_name(name) || is_public_role(name)
1521}
1522
1523pub fn is_public_role(name: &str) -> bool {
1524    name == &*PUBLIC_ROLE_NAME
1525}
1526
1527pub(crate) fn catalog_type_to_audit_object_type(sql_type: SqlCatalogItemType) -> ObjectType {
1528    object_type_to_audit_object_type(sql_type.into())
1529}
1530
1531pub(crate) fn comment_id_to_audit_object_type(id: CommentObjectId) -> ObjectType {
1532    match id {
1533        CommentObjectId::Table(_) => ObjectType::Table,
1534        CommentObjectId::View(_) => ObjectType::View,
1535        CommentObjectId::MaterializedView(_) => ObjectType::MaterializedView,
1536        CommentObjectId::Source(_) => ObjectType::Source,
1537        CommentObjectId::Sink(_) => ObjectType::Sink,
1538        CommentObjectId::Index(_) => ObjectType::Index,
1539        CommentObjectId::Func(_) => ObjectType::Func,
1540        CommentObjectId::Connection(_) => ObjectType::Connection,
1541        CommentObjectId::Type(_) => ObjectType::Type,
1542        CommentObjectId::Secret(_) => ObjectType::Secret,
1543        CommentObjectId::Role(_) => ObjectType::Role,
1544        CommentObjectId::Database(_) => ObjectType::Database,
1545        CommentObjectId::Schema(_) => ObjectType::Schema,
1546        CommentObjectId::Cluster(_) => ObjectType::Cluster,
1547        CommentObjectId::ClusterReplica(_) => ObjectType::ClusterReplica,
1548        CommentObjectId::ContinualTask(_) => ObjectType::ContinualTask,
1549        CommentObjectId::NetworkPolicy(_) => ObjectType::NetworkPolicy,
1550    }
1551}
1552
1553pub(crate) fn object_type_to_audit_object_type(
1554    object_type: mz_sql::catalog::ObjectType,
1555) -> ObjectType {
1556    system_object_type_to_audit_object_type(&SystemObjectType::Object(object_type))
1557}
1558
1559pub(crate) fn system_object_type_to_audit_object_type(
1560    system_type: &SystemObjectType,
1561) -> ObjectType {
1562    match system_type {
1563        SystemObjectType::Object(object_type) => match object_type {
1564            mz_sql::catalog::ObjectType::Table => ObjectType::Table,
1565            mz_sql::catalog::ObjectType::View => ObjectType::View,
1566            mz_sql::catalog::ObjectType::MaterializedView => ObjectType::MaterializedView,
1567            mz_sql::catalog::ObjectType::Source => ObjectType::Source,
1568            mz_sql::catalog::ObjectType::Sink => ObjectType::Sink,
1569            mz_sql::catalog::ObjectType::Index => ObjectType::Index,
1570            mz_sql::catalog::ObjectType::Type => ObjectType::Type,
1571            mz_sql::catalog::ObjectType::Role => ObjectType::Role,
1572            mz_sql::catalog::ObjectType::Cluster => ObjectType::Cluster,
1573            mz_sql::catalog::ObjectType::ClusterReplica => ObjectType::ClusterReplica,
1574            mz_sql::catalog::ObjectType::Secret => ObjectType::Secret,
1575            mz_sql::catalog::ObjectType::Connection => ObjectType::Connection,
1576            mz_sql::catalog::ObjectType::Database => ObjectType::Database,
1577            mz_sql::catalog::ObjectType::Schema => ObjectType::Schema,
1578            mz_sql::catalog::ObjectType::Func => ObjectType::Func,
1579            mz_sql::catalog::ObjectType::ContinualTask => ObjectType::ContinualTask,
1580            mz_sql::catalog::ObjectType::NetworkPolicy => ObjectType::NetworkPolicy,
1581        },
1582        SystemObjectType::System => ObjectType::System,
1583    }
1584}
1585
1586#[derive(Debug, Copy, Clone)]
1587pub enum UpdatePrivilegeVariant {
1588    Grant,
1589    Revoke,
1590}
1591
1592impl From<UpdatePrivilegeVariant> for ExecuteResponse {
1593    fn from(variant: UpdatePrivilegeVariant) -> Self {
1594        match variant {
1595            UpdatePrivilegeVariant::Grant => ExecuteResponse::GrantedPrivilege,
1596            UpdatePrivilegeVariant::Revoke => ExecuteResponse::RevokedPrivilege,
1597        }
1598    }
1599}
1600
1601impl From<UpdatePrivilegeVariant> for EventType {
1602    fn from(variant: UpdatePrivilegeVariant) -> Self {
1603        match variant {
1604            UpdatePrivilegeVariant::Grant => EventType::Grant,
1605            UpdatePrivilegeVariant::Revoke => EventType::Revoke,
1606        }
1607    }
1608}
1609
1610impl ConnCatalog<'_> {
1611    fn resolve_item_name(
1612        &self,
1613        name: &PartialItemName,
1614    ) -> Result<&QualifiedItemName, SqlCatalogError> {
1615        self.resolve_item(name).map(|entry| entry.name())
1616    }
1617
1618    fn resolve_function_name(
1619        &self,
1620        name: &PartialItemName,
1621    ) -> Result<&QualifiedItemName, SqlCatalogError> {
1622        self.resolve_function(name).map(|entry| entry.name())
1623    }
1624
1625    fn resolve_type_name(
1626        &self,
1627        name: &PartialItemName,
1628    ) -> Result<&QualifiedItemName, SqlCatalogError> {
1629        self.resolve_type(name).map(|entry| entry.name())
1630    }
1631}
1632
1633impl ExprHumanizer for ConnCatalog<'_> {
1634    fn humanize_id(&self, id: GlobalId) -> Option<String> {
1635        let entry = self.state.try_get_entry_by_global_id(&id)?;
1636        Some(self.resolve_full_name(entry.name()).to_string())
1637    }
1638
1639    fn humanize_id_unqualified(&self, id: GlobalId) -> Option<String> {
1640        let entry = self.state.try_get_entry_by_global_id(&id)?;
1641        Some(entry.name().item.clone())
1642    }
1643
1644    fn humanize_id_parts(&self, id: GlobalId) -> Option<Vec<String>> {
1645        let entry = self.state.try_get_entry_by_global_id(&id)?;
1646        Some(self.resolve_full_name(entry.name()).into_parts())
1647    }
1648
1649    fn humanize_sql_scalar_type(&self, typ: &SqlScalarType, postgres_compat: bool) -> String {
1650        use SqlScalarType::*;
1651
1652        match typ {
1653            Array(t) => format!("{}[]", self.humanize_sql_scalar_type(t, postgres_compat)),
1654            List {
1655                custom_id: Some(item_id),
1656                ..
1657            }
1658            | Map {
1659                custom_id: Some(item_id),
1660                ..
1661            } => {
1662                let item = self.get_item(item_id);
1663                self.minimal_qualification(item.name()).to_string()
1664            }
1665            List { element_type, .. } => {
1666                format!(
1667                    "{} list",
1668                    self.humanize_sql_scalar_type(element_type, postgres_compat)
1669                )
1670            }
1671            Map { value_type, .. } => format!(
1672                "map[{}=>{}]",
1673                self.humanize_sql_scalar_type(&SqlScalarType::String, postgres_compat),
1674                self.humanize_sql_scalar_type(value_type, postgres_compat)
1675            ),
1676            Record {
1677                custom_id: Some(item_id),
1678                ..
1679            } => {
1680                let item = self.get_item(item_id);
1681                self.minimal_qualification(item.name()).to_string()
1682            }
1683            Record { fields, .. } => format!(
1684                "record({})",
1685                fields
1686                    .iter()
1687                    .map(|f| format!(
1688                        "{}: {}",
1689                        f.0,
1690                        self.humanize_sql_column_type(&f.1, postgres_compat)
1691                    ))
1692                    .join(",")
1693            ),
1694            PgLegacyChar => "\"char\"".into(),
1695            Char { length } if !postgres_compat => match length {
1696                None => "char".into(),
1697                Some(length) => format!("char({})", length.into_u32()),
1698            },
1699            VarChar { max_length } if !postgres_compat => match max_length {
1700                None => "varchar".into(),
1701                Some(length) => format!("varchar({})", length.into_u32()),
1702            },
1703            UInt16 => "uint2".into(),
1704            UInt32 => "uint4".into(),
1705            UInt64 => "uint8".into(),
1706            ty => {
1707                let pgrepr_type = mz_pgrepr::Type::from(ty);
1708                let pg_catalog_schema = SchemaSpecifier::Id(self.state.get_pg_catalog_schema_id());
1709
1710                let res = if self
1711                    .effective_search_path(true)
1712                    .iter()
1713                    .any(|(_, schema)| schema == &pg_catalog_schema)
1714                {
1715                    pgrepr_type.name().to_string()
1716                } else {
1717                    // If PG_CATALOG_SCHEMA is not in search path, you need
1718                    // qualified object name to refer to type.
1719                    let name = QualifiedItemName {
1720                        qualifiers: ItemQualifiers {
1721                            database_spec: ResolvedDatabaseSpecifier::Ambient,
1722                            schema_spec: pg_catalog_schema,
1723                        },
1724                        item: pgrepr_type.name().to_string(),
1725                    };
1726                    self.resolve_full_name(&name).to_string()
1727                };
1728                res
1729            }
1730        }
1731    }
1732
1733    fn column_names_for_id(&self, id: GlobalId) -> Option<Vec<String>> {
1734        let entry = self.state.try_get_entry_by_global_id(&id)?;
1735
1736        match entry.index() {
1737            Some(index) => {
1738                let on_desc = self.state.try_get_desc_by_global_id(&index.on)?;
1739                let mut on_names = on_desc
1740                    .iter_names()
1741                    .map(|col_name| col_name.to_string())
1742                    .collect::<Vec<_>>();
1743
1744                let (p, _) = mz_expr::permutation_for_arrangement(&index.keys, on_desc.arity());
1745
1746                // Init ix_names with unknown column names. Unknown columns are
1747                // represented as an empty String and rendered as `#c` by the
1748                // Display::fmt implementation for HumanizedExpr<'a, usize, M>.
1749                let ix_arity = p.iter().map(|x| *x + 1).max().unwrap_or(0);
1750                let mut ix_names = vec![String::new(); ix_arity];
1751
1752                // Apply the permutation by swapping on_names with ix_names.
1753                for (on_pos, ix_pos) in p.into_iter().enumerate() {
1754                    let on_name = on_names.get_mut(on_pos).expect("on_name");
1755                    let ix_name = ix_names.get_mut(ix_pos).expect("ix_name");
1756                    std::mem::swap(on_name, ix_name);
1757                }
1758
1759                Some(ix_names) // Return the updated ix_names vector.
1760            }
1761            None => {
1762                let desc = self.state.try_get_desc_by_global_id(&id)?;
1763                let column_names = desc
1764                    .iter_names()
1765                    .map(|col_name| col_name.to_string())
1766                    .collect();
1767
1768                Some(column_names)
1769            }
1770        }
1771    }
1772
1773    fn humanize_column(&self, id: GlobalId, column: usize) -> Option<String> {
1774        let desc = self.state.try_get_desc_by_global_id(&id)?;
1775        Some(desc.get_name(column).to_string())
1776    }
1777
1778    fn id_exists(&self, id: GlobalId) -> bool {
1779        self.state.entry_by_global_id.contains_key(&id)
1780    }
1781}
1782
1783impl SessionCatalog for ConnCatalog<'_> {
1784    fn active_role_id(&self) -> &RoleId {
1785        &self.role_id
1786    }
1787
1788    fn get_prepared_statement_desc(&self, name: &str) -> Option<&StatementDesc> {
1789        self.prepared_statements
1790            .as_ref()
1791            .map(|ps| ps.get(name).map(|ps| ps.desc()))
1792            .flatten()
1793    }
1794
1795    fn get_portal_desc_unverified(&self, portal_name: &str) -> Option<&StatementDesc> {
1796        self.portals
1797            .and_then(|portals| portals.get(portal_name).map(|portal| &portal.desc))
1798    }
1799
1800    fn active_database(&self) -> Option<&DatabaseId> {
1801        self.database.as_ref()
1802    }
1803
1804    fn active_cluster(&self) -> &str {
1805        &self.cluster
1806    }
1807
1808    fn search_path(&self) -> &[(ResolvedDatabaseSpecifier, SchemaSpecifier)] {
1809        &self.search_path
1810    }
1811
1812    fn resolve_database(
1813        &self,
1814        database_name: &str,
1815    ) -> Result<&dyn mz_sql::catalog::CatalogDatabase, SqlCatalogError> {
1816        Ok(self.state.resolve_database(database_name)?)
1817    }
1818
1819    fn get_database(&self, id: &DatabaseId) -> &dyn mz_sql::catalog::CatalogDatabase {
1820        self.state
1821            .database_by_id
1822            .get(id)
1823            .expect("database doesn't exist")
1824    }
1825
1826    // `as` is ok to use to cast to a trait object.
1827    #[allow(clippy::as_conversions)]
1828    fn get_databases(&self) -> Vec<&dyn CatalogDatabase> {
1829        self.state
1830            .database_by_id
1831            .values()
1832            .map(|database| database as &dyn CatalogDatabase)
1833            .collect()
1834    }
1835
1836    fn resolve_schema(
1837        &self,
1838        database_name: Option<&str>,
1839        schema_name: &str,
1840    ) -> Result<&dyn mz_sql::catalog::CatalogSchema, SqlCatalogError> {
1841        Ok(self.state.resolve_schema(
1842            self.database.as_ref(),
1843            database_name,
1844            schema_name,
1845            &self.conn_id,
1846        )?)
1847    }
1848
1849    fn resolve_schema_in_database(
1850        &self,
1851        database_spec: &ResolvedDatabaseSpecifier,
1852        schema_name: &str,
1853    ) -> Result<&dyn mz_sql::catalog::CatalogSchema, SqlCatalogError> {
1854        Ok(self
1855            .state
1856            .resolve_schema_in_database(database_spec, schema_name, &self.conn_id)?)
1857    }
1858
1859    fn get_schema(
1860        &self,
1861        database_spec: &ResolvedDatabaseSpecifier,
1862        schema_spec: &SchemaSpecifier,
1863    ) -> &dyn CatalogSchema {
1864        self.state
1865            .get_schema(database_spec, schema_spec, &self.conn_id)
1866    }
1867
1868    // `as` is ok to use to cast to a trait object.
1869    #[allow(clippy::as_conversions)]
1870    fn get_schemas(&self) -> Vec<&dyn CatalogSchema> {
1871        self.get_databases()
1872            .into_iter()
1873            .flat_map(|database| database.schemas().into_iter())
1874            .chain(
1875                self.state
1876                    .ambient_schemas_by_id
1877                    .values()
1878                    .chain(self.state.temporary_schemas.values())
1879                    .map(|schema| schema as &dyn CatalogSchema),
1880            )
1881            .collect()
1882    }
1883
1884    fn get_mz_internal_schema_id(&self) -> SchemaId {
1885        self.state().get_mz_internal_schema_id()
1886    }
1887
1888    fn get_mz_unsafe_schema_id(&self) -> SchemaId {
1889        self.state().get_mz_unsafe_schema_id()
1890    }
1891
1892    fn is_system_schema_specifier(&self, schema: SchemaSpecifier) -> bool {
1893        self.state.is_system_schema_specifier(schema)
1894    }
1895
1896    fn resolve_role(
1897        &self,
1898        role_name: &str,
1899    ) -> Result<&dyn mz_sql::catalog::CatalogRole, SqlCatalogError> {
1900        match self.state.try_get_role_by_name(role_name) {
1901            Some(role) => Ok(role),
1902            None => Err(SqlCatalogError::UnknownRole(role_name.into())),
1903        }
1904    }
1905
1906    fn resolve_network_policy(
1907        &self,
1908        policy_name: &str,
1909    ) -> Result<&dyn mz_sql::catalog::CatalogNetworkPolicy, SqlCatalogError> {
1910        match self.state.try_get_network_policy_by_name(policy_name) {
1911            Some(policy) => Ok(policy),
1912            None => Err(SqlCatalogError::UnknownNetworkPolicy(policy_name.into())),
1913        }
1914    }
1915
1916    fn try_get_role(&self, id: &RoleId) -> Option<&dyn CatalogRole> {
1917        Some(self.state.roles_by_id.get(id)?)
1918    }
1919
1920    fn get_role(&self, id: &RoleId) -> &dyn mz_sql::catalog::CatalogRole {
1921        self.state.get_role(id)
1922    }
1923
1924    fn get_roles(&self) -> Vec<&dyn CatalogRole> {
1925        // `as` is ok to use to cast to a trait object.
1926        #[allow(clippy::as_conversions)]
1927        self.state
1928            .roles_by_id
1929            .values()
1930            .map(|role| role as &dyn CatalogRole)
1931            .collect()
1932    }
1933
1934    fn mz_system_role_id(&self) -> RoleId {
1935        MZ_SYSTEM_ROLE_ID
1936    }
1937
1938    fn collect_role_membership(&self, id: &RoleId) -> BTreeSet<RoleId> {
1939        self.state.collect_role_membership(id)
1940    }
1941
1942    fn get_network_policy(
1943        &self,
1944        id: &NetworkPolicyId,
1945    ) -> &dyn mz_sql::catalog::CatalogNetworkPolicy {
1946        self.state.get_network_policy(id)
1947    }
1948
1949    fn get_network_policies(&self) -> Vec<&dyn mz_sql::catalog::CatalogNetworkPolicy> {
1950        // `as` is ok to use to cast to a trait object.
1951        #[allow(clippy::as_conversions)]
1952        self.state
1953            .network_policies_by_id
1954            .values()
1955            .map(|policy| policy as &dyn CatalogNetworkPolicy)
1956            .collect()
1957    }
1958
1959    fn resolve_cluster(
1960        &self,
1961        cluster_name: Option<&str>,
1962    ) -> Result<&dyn mz_sql::catalog::CatalogCluster<'_>, SqlCatalogError> {
1963        Ok(self
1964            .state
1965            .resolve_cluster(cluster_name.unwrap_or_else(|| self.active_cluster()))?)
1966    }
1967
1968    fn resolve_cluster_replica(
1969        &self,
1970        cluster_replica_name: &QualifiedReplica,
1971    ) -> Result<&dyn CatalogClusterReplica<'_>, SqlCatalogError> {
1972        Ok(self.state.resolve_cluster_replica(cluster_replica_name)?)
1973    }
1974
1975    fn resolve_item(
1976        &self,
1977        name: &PartialItemName,
1978    ) -> Result<&dyn mz_sql::catalog::CatalogItem, SqlCatalogError> {
1979        let r = self.state.resolve_entry(
1980            self.database.as_ref(),
1981            &self.effective_search_path(true),
1982            name,
1983            &self.conn_id,
1984        )?;
1985        if self.unresolvable_ids.contains(&r.id()) {
1986            Err(SqlCatalogError::UnknownItem(name.to_string()))
1987        } else {
1988            Ok(r)
1989        }
1990    }
1991
1992    fn resolve_function(
1993        &self,
1994        name: &PartialItemName,
1995    ) -> Result<&dyn mz_sql::catalog::CatalogItem, SqlCatalogError> {
1996        let r = self.state.resolve_function(
1997            self.database.as_ref(),
1998            &self.effective_search_path(false),
1999            name,
2000            &self.conn_id,
2001        )?;
2002
2003        if self.unresolvable_ids.contains(&r.id()) {
2004            Err(SqlCatalogError::UnknownFunction {
2005                name: name.to_string(),
2006                alternative: None,
2007            })
2008        } else {
2009            Ok(r)
2010        }
2011    }
2012
2013    fn resolve_type(
2014        &self,
2015        name: &PartialItemName,
2016    ) -> Result<&dyn mz_sql::catalog::CatalogItem, SqlCatalogError> {
2017        let r = self.state.resolve_type(
2018            self.database.as_ref(),
2019            &self.effective_search_path(false),
2020            name,
2021            &self.conn_id,
2022        )?;
2023
2024        if self.unresolvable_ids.contains(&r.id()) {
2025            Err(SqlCatalogError::UnknownType {
2026                name: name.to_string(),
2027            })
2028        } else {
2029            Ok(r)
2030        }
2031    }
2032
2033    fn get_system_type(&self, name: &str) -> &dyn mz_sql::catalog::CatalogItem {
2034        self.state.get_system_type(name)
2035    }
2036
2037    fn try_get_item(&self, id: &CatalogItemId) -> Option<&dyn mz_sql::catalog::CatalogItem> {
2038        Some(self.state.try_get_entry(id)?)
2039    }
2040
2041    fn try_get_item_by_global_id(
2042        &self,
2043        id: &GlobalId,
2044    ) -> Option<Box<dyn mz_sql::catalog::CatalogCollectionItem>> {
2045        let entry = self.state.try_get_entry_by_global_id(id)?;
2046        let entry = match &entry.item {
2047            CatalogItem::Table(table) => {
2048                let (version, _gid) = table
2049                    .collections
2050                    .iter()
2051                    .find(|(_version, gid)| *gid == id)
2052                    .expect("catalog out of sync, mismatched GlobalId");
2053                entry.at_version(RelationVersionSelector::Specific(*version))
2054            }
2055            _ => entry.at_version(RelationVersionSelector::Latest),
2056        };
2057        Some(entry)
2058    }
2059
2060    fn get_item(&self, id: &CatalogItemId) -> &dyn mz_sql::catalog::CatalogItem {
2061        self.state.get_entry(id)
2062    }
2063
2064    fn get_item_by_global_id(
2065        &self,
2066        id: &GlobalId,
2067    ) -> Box<dyn mz_sql::catalog::CatalogCollectionItem> {
2068        let entry = self.state.get_entry_by_global_id(id);
2069        let entry = match &entry.item {
2070            CatalogItem::Table(table) => {
2071                let (version, _gid) = table
2072                    .collections
2073                    .iter()
2074                    .find(|(_version, gid)| *gid == id)
2075                    .expect("catalog out of sync, mismatched GlobalId");
2076                entry.at_version(RelationVersionSelector::Specific(*version))
2077            }
2078            _ => entry.at_version(RelationVersionSelector::Latest),
2079        };
2080        entry
2081    }
2082
2083    fn get_items(&self) -> Vec<&dyn mz_sql::catalog::CatalogItem> {
2084        self.get_schemas()
2085            .into_iter()
2086            .flat_map(|schema| schema.item_ids())
2087            .map(|id| self.get_item(&id))
2088            .collect()
2089    }
2090
2091    fn get_item_by_name(&self, name: &QualifiedItemName) -> Option<&dyn SqlCatalogItem> {
2092        self.state
2093            .get_item_by_name(name, &self.conn_id)
2094            .map(|item| convert::identity::<&dyn SqlCatalogItem>(item))
2095    }
2096
2097    fn get_type_by_name(&self, name: &QualifiedItemName) -> Option<&dyn SqlCatalogItem> {
2098        self.state
2099            .get_type_by_name(name, &self.conn_id)
2100            .map(|item| convert::identity::<&dyn SqlCatalogItem>(item))
2101    }
2102
2103    fn get_cluster(&self, id: ClusterId) -> &dyn mz_sql::catalog::CatalogCluster<'_> {
2104        &self.state.clusters_by_id[&id]
2105    }
2106
2107    fn get_clusters(&self) -> Vec<&dyn mz_sql::catalog::CatalogCluster<'_>> {
2108        self.state
2109            .clusters_by_id
2110            .values()
2111            .map(|cluster| convert::identity::<&dyn mz_sql::catalog::CatalogCluster>(cluster))
2112            .collect()
2113    }
2114
2115    fn get_cluster_replica(
2116        &self,
2117        cluster_id: ClusterId,
2118        replica_id: ReplicaId,
2119    ) -> &dyn mz_sql::catalog::CatalogClusterReplica<'_> {
2120        let cluster = self.get_cluster(cluster_id);
2121        cluster.replica(replica_id)
2122    }
2123
2124    fn get_cluster_replicas(&self) -> Vec<&dyn mz_sql::catalog::CatalogClusterReplica<'_>> {
2125        self.get_clusters()
2126            .into_iter()
2127            .flat_map(|cluster| cluster.replicas().into_iter())
2128            .collect()
2129    }
2130
2131    fn get_system_privileges(&self) -> &PrivilegeMap {
2132        &self.state.system_privileges
2133    }
2134
2135    fn get_default_privileges(
2136        &self,
2137    ) -> Vec<(&DefaultPrivilegeObject, Vec<&DefaultPrivilegeAclItem>)> {
2138        self.state
2139            .default_privileges
2140            .iter()
2141            .map(|(object, acl_items)| (object, acl_items.collect()))
2142            .collect()
2143    }
2144
2145    fn find_available_name(&self, name: QualifiedItemName) -> QualifiedItemName {
2146        self.state.find_available_name(name, &self.conn_id)
2147    }
2148
2149    fn resolve_full_name(&self, name: &QualifiedItemName) -> FullItemName {
2150        self.state.resolve_full_name(name, Some(&self.conn_id))
2151    }
2152
2153    fn resolve_full_schema_name(&self, name: &QualifiedSchemaName) -> FullSchemaName {
2154        self.state.resolve_full_schema_name(name)
2155    }
2156
2157    fn resolve_item_id(&self, global_id: &GlobalId) -> CatalogItemId {
2158        self.state.get_entry_by_global_id(global_id).id()
2159    }
2160
2161    fn resolve_global_id(
2162        &self,
2163        item_id: &CatalogItemId,
2164        version: RelationVersionSelector,
2165    ) -> GlobalId {
2166        self.state
2167            .get_entry(item_id)
2168            .at_version(version)
2169            .global_id()
2170    }
2171
2172    fn config(&self) -> &mz_sql::catalog::CatalogConfig {
2173        self.state.config()
2174    }
2175
2176    fn now(&self) -> EpochMillis {
2177        (self.state.config().now)()
2178    }
2179
2180    fn aws_privatelink_availability_zones(&self) -> Option<BTreeSet<String>> {
2181        self.state.aws_privatelink_availability_zones.clone()
2182    }
2183
2184    fn system_vars(&self) -> &SystemVars {
2185        &self.state.system_configuration
2186    }
2187
2188    fn system_vars_mut(&mut self) -> &mut SystemVars {
2189        Arc::make_mut(&mut self.state.to_mut().system_configuration)
2190    }
2191
2192    fn get_owner_id(&self, id: &ObjectId) -> Option<RoleId> {
2193        self.state().get_owner_id(id, self.conn_id())
2194    }
2195
2196    fn get_privileges(&self, id: &SystemObjectId) -> Option<&PrivilegeMap> {
2197        match id {
2198            SystemObjectId::System => Some(&self.state.system_privileges),
2199            SystemObjectId::Object(ObjectId::Cluster(id)) => {
2200                Some(self.get_cluster(*id).privileges())
2201            }
2202            SystemObjectId::Object(ObjectId::Database(id)) => {
2203                Some(self.get_database(id).privileges())
2204            }
2205            SystemObjectId::Object(ObjectId::Schema((database_spec, schema_spec))) => {
2206                // For temporary schemas that haven't been created yet (lazy creation),
2207                // we return None - the RBAC check will need to handle this case.
2208                self.state
2209                    .try_get_schema(database_spec, schema_spec, &self.conn_id)
2210                    .map(|schema| schema.privileges())
2211            }
2212            SystemObjectId::Object(ObjectId::Item(id)) => Some(self.get_item(id).privileges()),
2213            SystemObjectId::Object(ObjectId::NetworkPolicy(id)) => {
2214                Some(self.get_network_policy(id).privileges())
2215            }
2216            SystemObjectId::Object(ObjectId::ClusterReplica(_))
2217            | SystemObjectId::Object(ObjectId::Role(_)) => None,
2218        }
2219    }
2220
2221    fn object_dependents(&self, ids: &Vec<ObjectId>) -> Vec<ObjectId> {
2222        let mut seen = BTreeSet::new();
2223        self.state.object_dependents(ids, &self.conn_id, &mut seen)
2224    }
2225
2226    fn item_dependents(&self, id: CatalogItemId) -> Vec<ObjectId> {
2227        let mut seen = BTreeSet::new();
2228        self.state.item_dependents(id, &mut seen)
2229    }
2230
2231    fn all_object_privileges(&self, object_type: mz_sql::catalog::SystemObjectType) -> AclMode {
2232        rbac::all_object_privileges(object_type)
2233    }
2234
2235    fn get_object_type(&self, object_id: &ObjectId) -> mz_sql::catalog::ObjectType {
2236        self.state.get_object_type(object_id)
2237    }
2238
2239    fn get_system_object_type(&self, id: &SystemObjectId) -> mz_sql::catalog::SystemObjectType {
2240        self.state.get_system_object_type(id)
2241    }
2242
2243    /// Returns a [`PartialItemName`] with the minimum amount of qualifiers to unambiguously resolve
2244    /// the object.
2245    ///
2246    /// Warning: This is broken for temporary objects. Don't use this function for serious stuff,
2247    /// i.e., don't expect that what you get back is a thing you can resolve. Current usages are
2248    /// only for error msgs and other humanizations.
2249    fn minimal_qualification(&self, qualified_name: &QualifiedItemName) -> PartialItemName {
2250        if qualified_name.qualifiers.schema_spec.is_temporary() {
2251            // All bets are off. Just give up and return the qualified name as is.
2252            // TODO: Figure out what's going on with temporary objects.
2253
2254            // See e.g. `temporary_objects.slt` fail if you comment this out, which has the repro
2255            // from https://github.com/MaterializeInc/database-issues/issues/9973#issuecomment-3646382143
2256            // There is also https://github.com/MaterializeInc/database-issues/issues/9974, for
2257            // which we don't have a simple repro.
2258            return qualified_name.item.clone().into();
2259        }
2260
2261        let database_id = match &qualified_name.qualifiers.database_spec {
2262            ResolvedDatabaseSpecifier::Ambient => None,
2263            ResolvedDatabaseSpecifier::Id(id)
2264                if self.database.is_some() && self.database == Some(*id) =>
2265            {
2266                None
2267            }
2268            ResolvedDatabaseSpecifier::Id(id) => Some(id.clone()),
2269        };
2270
2271        let schema_spec = if database_id.is_none()
2272            && self.resolve_item_name(&PartialItemName {
2273                database: None,
2274                schema: None,
2275                item: qualified_name.item.clone(),
2276            }) == Ok(qualified_name)
2277            || self.resolve_function_name(&PartialItemName {
2278                database: None,
2279                schema: None,
2280                item: qualified_name.item.clone(),
2281            }) == Ok(qualified_name)
2282            || self.resolve_type_name(&PartialItemName {
2283                database: None,
2284                schema: None,
2285                item: qualified_name.item.clone(),
2286            }) == Ok(qualified_name)
2287        {
2288            None
2289        } else {
2290            // If `search_path` does not contain `full_name.schema`, the
2291            // `PartialName` must contain it.
2292            Some(qualified_name.qualifiers.schema_spec.clone())
2293        };
2294
2295        let res = PartialItemName {
2296            database: database_id.map(|id| self.get_database(&id).name().to_string()),
2297            schema: schema_spec.map(|spec| {
2298                self.get_schema(&qualified_name.qualifiers.database_spec, &spec)
2299                    .name()
2300                    .schema
2301                    .clone()
2302            }),
2303            item: qualified_name.item.clone(),
2304        };
2305        assert!(
2306            self.resolve_item_name(&res) == Ok(qualified_name)
2307                || self.resolve_function_name(&res) == Ok(qualified_name)
2308                || self.resolve_type_name(&res) == Ok(qualified_name)
2309        );
2310        res
2311    }
2312
2313    fn add_notice(&self, notice: PlanNotice) {
2314        let _ = self.notices_tx.send(notice.into());
2315    }
2316
2317    fn get_item_comments(&self, id: &CatalogItemId) -> Option<&BTreeMap<Option<usize>, String>> {
2318        let comment_id = self.state.get_comment_id(ObjectId::Item(*id));
2319        self.state.comments.get_object_comments(comment_id)
2320    }
2321
2322    fn is_cluster_size_cc(&self, size: &str) -> bool {
2323        self.state
2324            .cluster_replica_sizes
2325            .0
2326            .get(size)
2327            .map_or(false, |a| a.is_cc)
2328    }
2329}
2330
2331#[cfg(test)]
2332mod tests {
2333    use std::collections::{BTreeMap, BTreeSet};
2334    use std::sync::Arc;
2335    use std::{env, iter};
2336
2337    use itertools::Itertools;
2338    use mz_catalog::memory::objects::CatalogItem;
2339    use tokio_postgres::NoTls;
2340    use tokio_postgres::types::Type;
2341    use uuid::Uuid;
2342
2343    use mz_catalog::SYSTEM_CONN_ID;
2344    use mz_catalog::builtin::{BUILTINS, Builtin, BuiltinType};
2345    use mz_catalog::durable::{CatalogError, DurableCatalogError, FenceError, test_bootstrap_args};
2346    use mz_controller_types::{ClusterId, ReplicaId};
2347    use mz_expr::MirScalarExpr;
2348    use mz_ore::now::to_datetime;
2349    use mz_ore::{assert_err, assert_ok, soft_assert_eq_or_log, task};
2350    use mz_persist_client::PersistClient;
2351    use mz_pgrepr::oid::{FIRST_MATERIALIZE_OID, FIRST_UNPINNED_OID, FIRST_USER_OID};
2352    use mz_repr::namespaces::{INFORMATION_SCHEMA, PG_CATALOG_SCHEMA};
2353    use mz_repr::role_id::RoleId;
2354    use mz_repr::{
2355        CatalogItemId, Datum, GlobalId, RelationVersionSelector, RowArena, SqlRelationType,
2356        SqlScalarType, Timestamp,
2357    };
2358    use mz_sql::catalog::{BuiltinsConfig, CatalogSchema, CatalogType, SessionCatalog};
2359    use mz_sql::func::{Func, FuncImpl, OP_IMPLS, Operation};
2360    use mz_sql::names::{
2361        self, DatabaseId, ItemQualifiers, ObjectId, PartialItemName, QualifiedItemName,
2362        ResolvedDatabaseSpecifier, SchemaId, SchemaSpecifier, SystemObjectId,
2363    };
2364    use mz_sql::plan::{
2365        CoercibleScalarExpr, ExprContext, HirScalarExpr, PlanContext, QueryContext, QueryLifetime,
2366        Scope, StatementContext,
2367    };
2368    use mz_sql::session::user::MZ_SYSTEM_ROLE_ID;
2369    use mz_sql::session::vars::{SystemVars, VarInput};
2370
2371    use crate::catalog::state::LocalExpressionCache;
2372    use crate::catalog::{Catalog, Op};
2373    use crate::optimize::dataflows::{EvalTime, ExprPrep, ExprPrepOneShot};
2374    use crate::session::Session;
2375
2376    /// System sessions have an empty `search_path` so it's necessary to
2377    /// schema-qualify all referenced items.
2378    ///
2379    /// Dummy (and ostensibly client) sessions contain system schemas in their
2380    /// search paths, so do not require schema qualification on system objects such
2381    /// as types.
2382    #[mz_ore::test(tokio::test)]
2383    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2384    async fn test_minimal_qualification() {
2385        Catalog::with_debug(|catalog| async move {
2386            struct TestCase {
2387                input: QualifiedItemName,
2388                system_output: PartialItemName,
2389                normal_output: PartialItemName,
2390            }
2391
2392            let test_cases = vec![
2393                TestCase {
2394                    input: QualifiedItemName {
2395                        qualifiers: ItemQualifiers {
2396                            database_spec: ResolvedDatabaseSpecifier::Ambient,
2397                            schema_spec: SchemaSpecifier::Id(catalog.get_pg_catalog_schema_id()),
2398                        },
2399                        item: "numeric".to_string(),
2400                    },
2401                    system_output: PartialItemName {
2402                        database: None,
2403                        schema: None,
2404                        item: "numeric".to_string(),
2405                    },
2406                    normal_output: PartialItemName {
2407                        database: None,
2408                        schema: None,
2409                        item: "numeric".to_string(),
2410                    },
2411                },
2412                TestCase {
2413                    input: QualifiedItemName {
2414                        qualifiers: ItemQualifiers {
2415                            database_spec: ResolvedDatabaseSpecifier::Ambient,
2416                            schema_spec: SchemaSpecifier::Id(catalog.get_mz_catalog_schema_id()),
2417                        },
2418                        item: "mz_array_types".to_string(),
2419                    },
2420                    system_output: PartialItemName {
2421                        database: None,
2422                        schema: None,
2423                        item: "mz_array_types".to_string(),
2424                    },
2425                    normal_output: PartialItemName {
2426                        database: None,
2427                        schema: None,
2428                        item: "mz_array_types".to_string(),
2429                    },
2430                },
2431            ];
2432
2433            for tc in test_cases {
2434                assert_eq!(
2435                    catalog
2436                        .for_system_session()
2437                        .minimal_qualification(&tc.input),
2438                    tc.system_output
2439                );
2440                assert_eq!(
2441                    catalog
2442                        .for_session(&Session::dummy())
2443                        .minimal_qualification(&tc.input),
2444                    tc.normal_output
2445                );
2446            }
2447            catalog.expire().await;
2448        })
2449        .await
2450    }
2451
2452    #[mz_ore::test(tokio::test)]
2453    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2454    async fn test_catalog_revision() {
2455        let persist_client = PersistClient::new_for_tests().await;
2456        let organization_id = Uuid::new_v4();
2457        let bootstrap_args = test_bootstrap_args();
2458        {
2459            let mut catalog = Catalog::open_debug_catalog(
2460                persist_client.clone(),
2461                organization_id.clone(),
2462                &bootstrap_args,
2463            )
2464            .await
2465            .expect("unable to open debug catalog");
2466            assert_eq!(catalog.transient_revision(), 1);
2467            let commit_ts = catalog.current_upper().await;
2468            catalog
2469                .transact(
2470                    None,
2471                    commit_ts,
2472                    None,
2473                    vec![Op::CreateDatabase {
2474                        name: "test".to_string(),
2475                        owner_id: MZ_SYSTEM_ROLE_ID,
2476                    }],
2477                )
2478                .await
2479                .expect("failed to transact");
2480            assert_eq!(catalog.transient_revision(), 2);
2481            catalog.expire().await;
2482        }
2483        {
2484            let catalog =
2485                Catalog::open_debug_catalog(persist_client, organization_id, &bootstrap_args)
2486                    .await
2487                    .expect("unable to open debug catalog");
2488            // Re-opening the same catalog resets the transient_revision to 1.
2489            assert_eq!(catalog.transient_revision(), 1);
2490            catalog.expire().await;
2491        }
2492    }
2493
2494    #[mz_ore::test(tokio::test)]
2495    #[cfg_attr(miri, ignore)] // unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2496    async fn test_effective_search_path() {
2497        Catalog::with_debug(|catalog| async move {
2498            let mz_catalog_schema = (
2499                ResolvedDatabaseSpecifier::Ambient,
2500                SchemaSpecifier::Id(catalog.state().get_mz_catalog_schema_id()),
2501            );
2502            let pg_catalog_schema = (
2503                ResolvedDatabaseSpecifier::Ambient,
2504                SchemaSpecifier::Id(catalog.state().get_pg_catalog_schema_id()),
2505            );
2506            let mz_temp_schema = (
2507                ResolvedDatabaseSpecifier::Ambient,
2508                SchemaSpecifier::Temporary,
2509            );
2510
2511            // Behavior with the default search_schema (public)
2512            let session = Session::dummy();
2513            let conn_catalog = catalog.for_session(&session);
2514            assert_ne!(
2515                conn_catalog.effective_search_path(false),
2516                conn_catalog.search_path
2517            );
2518            assert_ne!(
2519                conn_catalog.effective_search_path(true),
2520                conn_catalog.search_path
2521            );
2522            assert_eq!(
2523                conn_catalog.effective_search_path(false),
2524                vec![
2525                    mz_catalog_schema.clone(),
2526                    pg_catalog_schema.clone(),
2527                    conn_catalog.search_path[0].clone()
2528                ]
2529            );
2530            assert_eq!(
2531                conn_catalog.effective_search_path(true),
2532                vec![
2533                    mz_temp_schema.clone(),
2534                    mz_catalog_schema.clone(),
2535                    pg_catalog_schema.clone(),
2536                    conn_catalog.search_path[0].clone()
2537                ]
2538            );
2539
2540            // missing schemas are added when missing
2541            let mut session = Session::dummy();
2542            session
2543                .vars_mut()
2544                .set(
2545                    &SystemVars::new(),
2546                    "search_path",
2547                    VarInput::Flat(mz_repr::namespaces::PG_CATALOG_SCHEMA),
2548                    false,
2549                )
2550                .expect("failed to set search_path");
2551            let conn_catalog = catalog.for_session(&session);
2552            assert_ne!(
2553                conn_catalog.effective_search_path(false),
2554                conn_catalog.search_path
2555            );
2556            assert_ne!(
2557                conn_catalog.effective_search_path(true),
2558                conn_catalog.search_path
2559            );
2560            assert_eq!(
2561                conn_catalog.effective_search_path(false),
2562                vec![mz_catalog_schema.clone(), pg_catalog_schema.clone()]
2563            );
2564            assert_eq!(
2565                conn_catalog.effective_search_path(true),
2566                vec![
2567                    mz_temp_schema.clone(),
2568                    mz_catalog_schema.clone(),
2569                    pg_catalog_schema.clone()
2570                ]
2571            );
2572
2573            let mut session = Session::dummy();
2574            session
2575                .vars_mut()
2576                .set(
2577                    &SystemVars::new(),
2578                    "search_path",
2579                    VarInput::Flat(mz_repr::namespaces::MZ_CATALOG_SCHEMA),
2580                    false,
2581                )
2582                .expect("failed to set search_path");
2583            let conn_catalog = catalog.for_session(&session);
2584            assert_ne!(
2585                conn_catalog.effective_search_path(false),
2586                conn_catalog.search_path
2587            );
2588            assert_ne!(
2589                conn_catalog.effective_search_path(true),
2590                conn_catalog.search_path
2591            );
2592            assert_eq!(
2593                conn_catalog.effective_search_path(false),
2594                vec![pg_catalog_schema.clone(), mz_catalog_schema.clone()]
2595            );
2596            assert_eq!(
2597                conn_catalog.effective_search_path(true),
2598                vec![
2599                    mz_temp_schema.clone(),
2600                    pg_catalog_schema.clone(),
2601                    mz_catalog_schema.clone()
2602                ]
2603            );
2604
2605            let mut session = Session::dummy();
2606            session
2607                .vars_mut()
2608                .set(
2609                    &SystemVars::new(),
2610                    "search_path",
2611                    VarInput::Flat(mz_repr::namespaces::MZ_TEMP_SCHEMA),
2612                    false,
2613                )
2614                .expect("failed to set search_path");
2615            let conn_catalog = catalog.for_session(&session);
2616            assert_ne!(
2617                conn_catalog.effective_search_path(false),
2618                conn_catalog.search_path
2619            );
2620            assert_ne!(
2621                conn_catalog.effective_search_path(true),
2622                conn_catalog.search_path
2623            );
2624            assert_eq!(
2625                conn_catalog.effective_search_path(false),
2626                vec![
2627                    mz_catalog_schema.clone(),
2628                    pg_catalog_schema.clone(),
2629                    mz_temp_schema.clone()
2630                ]
2631            );
2632            assert_eq!(
2633                conn_catalog.effective_search_path(true),
2634                vec![mz_catalog_schema, pg_catalog_schema, mz_temp_schema]
2635            );
2636            catalog.expire().await;
2637        })
2638        .await
2639    }
2640
2641    #[mz_ore::test(tokio::test)]
2642    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2643    async fn test_normalized_create() {
2644        use mz_ore::collections::CollectionExt;
2645        Catalog::with_debug(|catalog| async move {
2646            let conn_catalog = catalog.for_system_session();
2647            let scx = &mut StatementContext::new(None, &conn_catalog);
2648
2649            let parsed = mz_sql_parser::parser::parse_statements(
2650                "create view public.foo as select 1 as bar",
2651            )
2652            .expect("")
2653            .into_element()
2654            .ast;
2655
2656            let (stmt, _) = names::resolve(scx.catalog, parsed).expect("");
2657
2658            // Ensure that all identifiers are quoted.
2659            assert_eq!(
2660                r#"CREATE VIEW "materialize"."public"."foo" AS SELECT 1 AS "bar""#,
2661                mz_sql::normalize::create_statement(scx, stmt).expect(""),
2662            );
2663            catalog.expire().await;
2664        })
2665        .await;
2666    }
2667
2668    // Test that if a large catalog item is somehow committed, then we can still load the catalog.
2669    #[mz_ore::test(tokio::test)]
2670    #[cfg_attr(miri, ignore)] // slow
2671    async fn test_large_catalog_item() {
2672        let view_def = "CREATE VIEW \"materialize\".\"public\".\"v\" AS SELECT 1 FROM (SELECT 1";
2673        let column = ", 1";
2674        let view_def_size = view_def.bytes().count();
2675        let column_size = column.bytes().count();
2676        let column_count =
2677            (mz_sql_parser::parser::MAX_STATEMENT_BATCH_SIZE - view_def_size) / column_size + 1;
2678        let columns = iter::repeat(column).take(column_count).join("");
2679        let create_sql = format!("{view_def}{columns})");
2680        let create_sql_check = create_sql.clone();
2681        assert_ok!(mz_sql_parser::parser::parse_statements(&create_sql));
2682        assert_err!(mz_sql_parser::parser::parse_statements_with_limit(
2683            &create_sql
2684        ));
2685
2686        let persist_client = PersistClient::new_for_tests().await;
2687        let organization_id = Uuid::new_v4();
2688        let id = CatalogItemId::User(1);
2689        let gid = GlobalId::User(1);
2690        let bootstrap_args = test_bootstrap_args();
2691        {
2692            let mut catalog = Catalog::open_debug_catalog(
2693                persist_client.clone(),
2694                organization_id.clone(),
2695                &bootstrap_args,
2696            )
2697            .await
2698            .expect("unable to open debug catalog");
2699            let item = catalog
2700                .state()
2701                .deserialize_item(
2702                    gid,
2703                    &create_sql,
2704                    &BTreeMap::new(),
2705                    &mut LocalExpressionCache::Closed,
2706                    None,
2707                )
2708                .expect("unable to parse view");
2709            let commit_ts = catalog.current_upper().await;
2710            catalog
2711                .transact(
2712                    None,
2713                    commit_ts,
2714                    None,
2715                    vec![Op::CreateItem {
2716                        item,
2717                        name: QualifiedItemName {
2718                            qualifiers: ItemQualifiers {
2719                                database_spec: ResolvedDatabaseSpecifier::Id(DatabaseId::User(1)),
2720                                schema_spec: SchemaSpecifier::Id(SchemaId::User(3)),
2721                            },
2722                            item: "v".to_string(),
2723                        },
2724                        id,
2725                        owner_id: MZ_SYSTEM_ROLE_ID,
2726                    }],
2727                )
2728                .await
2729                .expect("failed to transact");
2730            catalog.expire().await;
2731        }
2732        {
2733            let catalog =
2734                Catalog::open_debug_catalog(persist_client, organization_id, &bootstrap_args)
2735                    .await
2736                    .expect("unable to open debug catalog");
2737            let view = catalog.get_entry(&id);
2738            assert_eq!("v", view.name.item);
2739            match &view.item {
2740                CatalogItem::View(view) => assert_eq!(create_sql_check, view.create_sql),
2741                item => panic!("expected view, got {}", item.typ()),
2742            }
2743            catalog.expire().await;
2744        }
2745    }
2746
2747    #[mz_ore::test(tokio::test)]
2748    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2749    async fn test_object_type() {
2750        Catalog::with_debug(|catalog| async move {
2751            let conn_catalog = catalog.for_system_session();
2752
2753            assert_eq!(
2754                mz_sql::catalog::ObjectType::ClusterReplica,
2755                conn_catalog.get_object_type(&ObjectId::ClusterReplica((
2756                    ClusterId::user(1).expect("1 is a valid ID"),
2757                    ReplicaId::User(1)
2758                )))
2759            );
2760            assert_eq!(
2761                mz_sql::catalog::ObjectType::Role,
2762                conn_catalog.get_object_type(&ObjectId::Role(RoleId::User(1)))
2763            );
2764            catalog.expire().await;
2765        })
2766        .await;
2767    }
2768
2769    #[mz_ore::test(tokio::test)]
2770    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2771    async fn test_get_privileges() {
2772        Catalog::with_debug(|catalog| async move {
2773            let conn_catalog = catalog.for_system_session();
2774
2775            assert_eq!(
2776                None,
2777                conn_catalog.get_privileges(&SystemObjectId::Object(ObjectId::ClusterReplica((
2778                    ClusterId::user(1).expect("1 is a valid ID"),
2779                    ReplicaId::User(1),
2780                ))))
2781            );
2782            assert_eq!(
2783                None,
2784                conn_catalog
2785                    .get_privileges(&SystemObjectId::Object(ObjectId::Role(RoleId::User(1))))
2786            );
2787            catalog.expire().await;
2788        })
2789        .await;
2790    }
2791
2792    #[mz_ore::test(tokio::test)]
2793    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2794    async fn verify_builtin_descs() {
2795        Catalog::with_debug(|catalog| async move {
2796            let conn_catalog = catalog.for_system_session();
2797
2798            let builtins_cfg = BuiltinsConfig {
2799                include_continual_tasks: true,
2800            };
2801            for builtin in BUILTINS::iter(&builtins_cfg) {
2802                let (schema, name, expected_desc) = match builtin {
2803                    Builtin::Table(t) => (&t.schema, &t.name, &t.desc),
2804                    Builtin::View(v) => (&v.schema, &v.name, &v.desc),
2805                    Builtin::Source(s) => (&s.schema, &s.name, &s.desc),
2806                    Builtin::Log(_)
2807                    | Builtin::Type(_)
2808                    | Builtin::Func(_)
2809                    | Builtin::ContinualTask(_)
2810                    | Builtin::Index(_)
2811                    | Builtin::Connection(_) => continue,
2812                };
2813                let item = conn_catalog
2814                    .resolve_item(&PartialItemName {
2815                        database: None,
2816                        schema: Some(schema.to_string()),
2817                        item: name.to_string(),
2818                    })
2819                    .expect("unable to resolve item")
2820                    .at_version(RelationVersionSelector::Latest);
2821
2822                let actual_desc = item.relation_desc().expect("invalid item type");
2823                for (index, ((actual_name, actual_typ), (expected_name, expected_typ))) in
2824                    actual_desc.iter().zip_eq(expected_desc.iter()).enumerate()
2825                {
2826                    assert_eq!(
2827                        actual_name, expected_name,
2828                        "item {schema}.{name} column {index} name did not match its expected name"
2829                    );
2830                    assert_eq!(
2831                        actual_typ, expected_typ,
2832                        "item {schema}.{name} column {index} ('{actual_name}') type did not match its expected type"
2833                    );
2834                }
2835                assert_eq!(
2836                    &*actual_desc, expected_desc,
2837                    "item {schema}.{name} did not match its expected RelationDesc"
2838                );
2839            }
2840            catalog.expire().await;
2841        })
2842        .await
2843    }
2844
2845    // Connect to a running Postgres server and verify that our builtin
2846    // types and functions match it, in addition to some other things.
2847    #[mz_ore::test(tokio::test)]
2848    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
2849    async fn test_compare_builtins_postgres() {
2850        async fn inner(catalog: Catalog) {
2851            // Verify that all builtin functions:
2852            // - have a unique OID
2853            // - if they have a postgres counterpart (same oid) then they have matching name
2854            let (client, connection) = tokio_postgres::connect(
2855                &env::var("POSTGRES_URL").unwrap_or_else(|_| "host=localhost user=postgres".into()),
2856                NoTls,
2857            )
2858            .await
2859            .expect("failed to connect to Postgres");
2860
2861            task::spawn(|| "compare_builtin_postgres", async move {
2862                if let Err(e) = connection.await {
2863                    panic!("connection error: {}", e);
2864                }
2865            });
2866
2867            struct PgProc {
2868                name: String,
2869                arg_oids: Vec<u32>,
2870                ret_oid: Option<u32>,
2871                ret_set: bool,
2872            }
2873
2874            struct PgType {
2875                name: String,
2876                ty: String,
2877                elem: u32,
2878                array: u32,
2879                input: u32,
2880                receive: u32,
2881            }
2882
2883            struct PgOper {
2884                oprresult: u32,
2885                name: String,
2886            }
2887
2888            let pg_proc: BTreeMap<_, _> = client
2889                .query(
2890                    "SELECT
2891                    p.oid,
2892                    proname,
2893                    proargtypes,
2894                    prorettype,
2895                    proretset
2896                FROM pg_proc p
2897                JOIN pg_namespace n ON p.pronamespace = n.oid",
2898                    &[],
2899                )
2900                .await
2901                .expect("pg query failed")
2902                .into_iter()
2903                .map(|row| {
2904                    let oid: u32 = row.get("oid");
2905                    let pg_proc = PgProc {
2906                        name: row.get("proname"),
2907                        arg_oids: row.get("proargtypes"),
2908                        ret_oid: row.get("prorettype"),
2909                        ret_set: row.get("proretset"),
2910                    };
2911                    (oid, pg_proc)
2912                })
2913                .collect();
2914
2915            let pg_type: BTreeMap<_, _> = client
2916                .query(
2917                    "SELECT oid, typname, typtype::text, typelem, typarray, typinput::oid, typreceive::oid as typreceive FROM pg_type",
2918                    &[],
2919                )
2920                .await
2921                .expect("pg query failed")
2922                .into_iter()
2923                .map(|row| {
2924                    let oid: u32 = row.get("oid");
2925                    let pg_type = PgType {
2926                        name: row.get("typname"),
2927                        ty: row.get("typtype"),
2928                        elem: row.get("typelem"),
2929                        array: row.get("typarray"),
2930                        input: row.get("typinput"),
2931                        receive: row.get("typreceive"),
2932                    };
2933                    (oid, pg_type)
2934                })
2935                .collect();
2936
2937            let pg_oper: BTreeMap<_, _> = client
2938                .query("SELECT oid, oprname, oprresult FROM pg_operator", &[])
2939                .await
2940                .expect("pg query failed")
2941                .into_iter()
2942                .map(|row| {
2943                    let oid: u32 = row.get("oid");
2944                    let pg_oper = PgOper {
2945                        name: row.get("oprname"),
2946                        oprresult: row.get("oprresult"),
2947                    };
2948                    (oid, pg_oper)
2949                })
2950                .collect();
2951
2952            let conn_catalog = catalog.for_system_session();
2953            let resolve_type_oid = |item: &str| {
2954                conn_catalog
2955                    .resolve_type(&PartialItemName {
2956                        database: None,
2957                        // All functions we check exist in PG, so the types must, as
2958                        // well
2959                        schema: Some(PG_CATALOG_SCHEMA.into()),
2960                        item: item.to_string(),
2961                    })
2962                    .expect("unable to resolve type")
2963                    .oid()
2964            };
2965
2966            let func_oids: BTreeSet<_> = BUILTINS::funcs()
2967                .flat_map(|f| f.inner.func_impls().into_iter().map(|f| f.oid))
2968                .collect();
2969
2970            let mut all_oids = BTreeSet::new();
2971
2972            // A function to determine if two oids are equivalent enough for these tests. We don't
2973            // support some types, so map exceptions here.
2974            let equivalent_types: BTreeSet<(Option<u32>, Option<u32>)> = BTreeSet::from_iter(
2975                [
2976                    // We don't support NAME.
2977                    (Type::NAME, Type::TEXT),
2978                    (Type::NAME_ARRAY, Type::TEXT_ARRAY),
2979                    // We don't support time with time zone.
2980                    (Type::TIME, Type::TIMETZ),
2981                    (Type::TIME_ARRAY, Type::TIMETZ_ARRAY),
2982                ]
2983                .map(|(a, b)| (Some(a.oid()), Some(b.oid()))),
2984            );
2985            let ignore_return_types: BTreeSet<u32> = BTreeSet::from([
2986                1619, // pg_typeof: TODO: We now have regtype and can correctly implement this.
2987            ]);
2988            let is_same_type = |fn_oid: u32, a: Option<u32>, b: Option<u32>| -> bool {
2989                if ignore_return_types.contains(&fn_oid) {
2990                    return true;
2991                }
2992                if equivalent_types.contains(&(a, b)) || equivalent_types.contains(&(b, a)) {
2993                    return true;
2994                }
2995                a == b
2996            };
2997
2998            let builtins_cfg = BuiltinsConfig {
2999                include_continual_tasks: true,
3000            };
3001            for builtin in BUILTINS::iter(&builtins_cfg) {
3002                match builtin {
3003                    Builtin::Type(ty) => {
3004                        assert!(all_oids.insert(ty.oid), "{} reused oid {}", ty.name, ty.oid);
3005
3006                        if ty.oid >= FIRST_MATERIALIZE_OID {
3007                            // High OIDs are reserved in Materialize and don't have
3008                            // PostgreSQL counterparts.
3009                            continue;
3010                        }
3011
3012                        // For types that have a PostgreSQL counterpart, verify that
3013                        // the name and oid match.
3014                        let pg_ty = pg_type.get(&ty.oid).unwrap_or_else(|| {
3015                            panic!("pg_proc missing type {}: oid {}", ty.name, ty.oid)
3016                        });
3017                        assert_eq!(
3018                            ty.name, pg_ty.name,
3019                            "oid {} has name {} in postgres; expected {}",
3020                            ty.oid, pg_ty.name, ty.name,
3021                        );
3022
3023                        let (typinput_oid, typreceive_oid) = match &ty.details.pg_metadata {
3024                            None => (0, 0),
3025                            Some(pgmeta) => (pgmeta.typinput_oid, pgmeta.typreceive_oid),
3026                        };
3027                        assert_eq!(
3028                            typinput_oid, pg_ty.input,
3029                            "type {} has typinput OID {:?} in mz but {:?} in pg",
3030                            ty.name, typinput_oid, pg_ty.input,
3031                        );
3032                        assert_eq!(
3033                            typreceive_oid, pg_ty.receive,
3034                            "type {} has typreceive OID {:?} in mz but {:?} in pg",
3035                            ty.name, typreceive_oid, pg_ty.receive,
3036                        );
3037                        if typinput_oid != 0 {
3038                            assert!(
3039                                func_oids.contains(&typinput_oid),
3040                                "type {} has typinput OID {} that does not exist in pg_proc",
3041                                ty.name,
3042                                typinput_oid,
3043                            );
3044                        }
3045                        if typreceive_oid != 0 {
3046                            assert!(
3047                                func_oids.contains(&typreceive_oid),
3048                                "type {} has typreceive OID {} that does not exist in pg_proc",
3049                                ty.name,
3050                                typreceive_oid,
3051                            );
3052                        }
3053
3054                        // Ensure the type matches.
3055                        match &ty.details.typ {
3056                            CatalogType::Array { element_reference } => {
3057                                let elem_ty = BUILTINS::iter(&builtins_cfg)
3058                                    .filter_map(|builtin| match builtin {
3059                                        Builtin::Type(ty @ BuiltinType { name, .. })
3060                                            if element_reference == name =>
3061                                        {
3062                                            Some(ty)
3063                                        }
3064                                        _ => None,
3065                                    })
3066                                    .next();
3067                                let elem_ty = match elem_ty {
3068                                    Some(ty) => ty,
3069                                    None => {
3070                                        panic!("{} is unexpectedly not a type", element_reference)
3071                                    }
3072                                };
3073                                assert_eq!(
3074                                    pg_ty.elem, elem_ty.oid,
3075                                    "type {} has mismatched element OIDs",
3076                                    ty.name
3077                                )
3078                            }
3079                            CatalogType::Pseudo => {
3080                                assert_eq!(
3081                                    pg_ty.ty, "p",
3082                                    "type {} is not a pseudo type as expected",
3083                                    ty.name
3084                                )
3085                            }
3086                            CatalogType::Range { .. } => {
3087                                assert_eq!(
3088                                    pg_ty.ty, "r",
3089                                    "type {} is not a range type as expected",
3090                                    ty.name
3091                                );
3092                            }
3093                            _ => {
3094                                assert_eq!(
3095                                    pg_ty.ty, "b",
3096                                    "type {} is not a base type as expected",
3097                                    ty.name
3098                                )
3099                            }
3100                        }
3101
3102                        // Ensure the array type reference is correct.
3103                        let schema = catalog
3104                            .resolve_schema_in_database(
3105                                &ResolvedDatabaseSpecifier::Ambient,
3106                                ty.schema,
3107                                &SYSTEM_CONN_ID,
3108                            )
3109                            .expect("unable to resolve schema");
3110                        let allocated_type = catalog
3111                            .resolve_type(
3112                                None,
3113                                &vec![(ResolvedDatabaseSpecifier::Ambient, schema.id().clone())],
3114                                &PartialItemName {
3115                                    database: None,
3116                                    schema: Some(schema.name().schema.clone()),
3117                                    item: ty.name.to_string(),
3118                                },
3119                                &SYSTEM_CONN_ID,
3120                            )
3121                            .expect("unable to resolve type");
3122                        let ty = if let CatalogItem::Type(ty) = &allocated_type.item {
3123                            ty
3124                        } else {
3125                            panic!("unexpectedly not a type")
3126                        };
3127                        match ty.details.array_id {
3128                            Some(array_id) => {
3129                                let array_ty = catalog.get_entry(&array_id);
3130                                assert_eq!(
3131                                    pg_ty.array, array_ty.oid,
3132                                    "type {} has mismatched array OIDs",
3133                                    allocated_type.name.item,
3134                                );
3135                            }
3136                            None => assert_eq!(
3137                                pg_ty.array, 0,
3138                                "type {} does not have an array type in mz but does in pg",
3139                                allocated_type.name.item,
3140                            ),
3141                        }
3142                    }
3143                    Builtin::Func(func) => {
3144                        for imp in func.inner.func_impls() {
3145                            assert!(
3146                                all_oids.insert(imp.oid),
3147                                "{} reused oid {}",
3148                                func.name,
3149                                imp.oid
3150                            );
3151
3152                            assert!(
3153                                imp.oid < FIRST_USER_OID,
3154                                "built-in function {} erroneously has OID in user space ({})",
3155                                func.name,
3156                                imp.oid,
3157                            );
3158
3159                            // For functions that have a postgres counterpart, verify that the name and
3160                            // oid match.
3161                            let pg_fn = if imp.oid >= FIRST_UNPINNED_OID {
3162                                continue;
3163                            } else {
3164                                pg_proc.get(&imp.oid).unwrap_or_else(|| {
3165                                    panic!(
3166                                        "pg_proc missing function {}: oid {}",
3167                                        func.name, imp.oid
3168                                    )
3169                                })
3170                            };
3171                            assert_eq!(
3172                                func.name, pg_fn.name,
3173                                "funcs with oid {} don't match names: {} in mz, {} in pg",
3174                                imp.oid, func.name, pg_fn.name
3175                            );
3176
3177                            // Complain, but don't fail, if argument oids don't match.
3178                            // TODO: make these match.
3179                            let imp_arg_oids = imp
3180                                .arg_typs
3181                                .iter()
3182                                .map(|item| resolve_type_oid(item))
3183                                .collect::<Vec<_>>();
3184
3185                            if imp_arg_oids != pg_fn.arg_oids {
3186                                println!(
3187                                    "funcs with oid {} ({}) don't match arguments: {:?} in mz, {:?} in pg",
3188                                    imp.oid, func.name, imp_arg_oids, pg_fn.arg_oids
3189                                );
3190                            }
3191
3192                            let imp_return_oid = imp.return_typ.map(resolve_type_oid);
3193
3194                            assert!(
3195                                is_same_type(imp.oid, imp_return_oid, pg_fn.ret_oid),
3196                                "funcs with oid {} ({}) don't match return types: {:?} in mz, {:?} in pg",
3197                                imp.oid,
3198                                func.name,
3199                                imp_return_oid,
3200                                pg_fn.ret_oid
3201                            );
3202
3203                            assert_eq!(
3204                                imp.return_is_set, pg_fn.ret_set,
3205                                "funcs with oid {} ({}) don't match set-returning value: {:?} in mz, {:?} in pg",
3206                                imp.oid, func.name, imp.return_is_set, pg_fn.ret_set
3207                            );
3208                        }
3209                    }
3210                    _ => (),
3211                }
3212            }
3213
3214            for (op, func) in OP_IMPLS.iter() {
3215                for imp in func.func_impls() {
3216                    assert!(all_oids.insert(imp.oid), "{} reused oid {}", op, imp.oid);
3217
3218                    // For operators that have a postgres counterpart, verify that the name and oid match.
3219                    let pg_op = if imp.oid >= FIRST_UNPINNED_OID {
3220                        continue;
3221                    } else {
3222                        pg_oper.get(&imp.oid).unwrap_or_else(|| {
3223                            panic!("pg_operator missing operator {}: oid {}", op, imp.oid)
3224                        })
3225                    };
3226
3227                    assert_eq!(*op, pg_op.name);
3228
3229                    let imp_return_oid =
3230                        imp.return_typ.map(resolve_type_oid).expect("must have oid");
3231                    if imp_return_oid != pg_op.oprresult {
3232                        panic!(
3233                            "operators with oid {} ({}) don't match return typs: {} in mz, {} in pg",
3234                            imp.oid, op, imp_return_oid, pg_op.oprresult
3235                        );
3236                    }
3237                }
3238            }
3239            catalog.expire().await;
3240        }
3241
3242        Catalog::with_debug(inner).await
3243    }
3244
3245    // Execute all builtin functions with all combinations of arguments from interesting datums.
3246    #[mz_ore::test(tokio::test)]
3247    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
3248    async fn test_smoketest_all_builtins() {
3249        fn inner(catalog: Catalog) -> Vec<mz_ore::task::JoinHandle<()>> {
3250            let catalog = Arc::new(catalog);
3251            let conn_catalog = catalog.for_system_session();
3252
3253            let resolve_type_oid = |item: &str| conn_catalog.state().get_system_type(item).oid();
3254            let mut handles = Vec::new();
3255
3256            // Extracted during planning; always panics when executed.
3257            let ignore_names = BTreeSet::from([
3258                "avg",
3259                "avg_internal_v1",
3260                "bool_and",
3261                "bool_or",
3262                "has_table_privilege", // > 3 s each
3263                "has_type_privilege",  // > 3 s each
3264                "mod",
3265                "mz_panic",
3266                "mz_sleep",
3267                "pow",
3268                "stddev_pop",
3269                "stddev_samp",
3270                "stddev",
3271                "var_pop",
3272                "var_samp",
3273                "variance",
3274            ]);
3275
3276            let fns = BUILTINS::funcs()
3277                .map(|func| (&func.name, func.inner))
3278                .chain(OP_IMPLS.iter());
3279
3280            for (name, func) in fns {
3281                if ignore_names.contains(name) {
3282                    continue;
3283                }
3284                let Func::Scalar(impls) = func else {
3285                    continue;
3286                };
3287
3288                'outer: for imp in impls {
3289                    let details = imp.details();
3290                    let mut styps = Vec::new();
3291                    for item in details.arg_typs.iter() {
3292                        let oid = resolve_type_oid(item);
3293                        let Ok(pgtyp) = mz_pgrepr::Type::from_oid(oid) else {
3294                            continue 'outer;
3295                        };
3296                        styps.push(SqlScalarType::try_from(&pgtyp).expect("must exist"));
3297                    }
3298                    let datums = styps
3299                        .iter()
3300                        .map(|styp| {
3301                            let mut datums = vec![Datum::Null];
3302                            datums.extend(styp.interesting_datums());
3303                            datums
3304                        })
3305                        .collect::<Vec<_>>();
3306                    // Skip nullary fns.
3307                    if datums.is_empty() {
3308                        continue;
3309                    }
3310
3311                    let return_oid = details
3312                        .return_typ
3313                        .map(resolve_type_oid)
3314                        .expect("must exist");
3315                    let return_styp = mz_pgrepr::Type::from_oid(return_oid)
3316                        .ok()
3317                        .map(|typ| SqlScalarType::try_from(&typ).expect("must exist"));
3318
3319                    let mut idxs = vec![0; datums.len()];
3320                    while idxs[0] < datums[0].len() {
3321                        let mut args = Vec::with_capacity(idxs.len());
3322                        for i in 0..(datums.len()) {
3323                            args.push(datums[i][idxs[i]]);
3324                        }
3325
3326                        let op = &imp.op;
3327                        let scalars = args
3328                            .iter()
3329                            .enumerate()
3330                            .map(|(i, datum)| {
3331                                CoercibleScalarExpr::Coerced(HirScalarExpr::literal(
3332                                    datum.clone(),
3333                                    styps[i].clone(),
3334                                ))
3335                            })
3336                            .collect();
3337
3338                        let call_name = format!(
3339                            "{name}({}) (oid: {})",
3340                            args.iter()
3341                                .map(|d| d.to_string())
3342                                .collect::<Vec<_>>()
3343                                .join(", "),
3344                            imp.oid
3345                        );
3346                        let catalog = Arc::clone(&catalog);
3347                        let call_name_fn = call_name.clone();
3348                        let return_styp = return_styp.clone();
3349                        let handle = task::spawn_blocking(
3350                            || call_name,
3351                            move || {
3352                                smoketest_fn(
3353                                    name,
3354                                    call_name_fn,
3355                                    op,
3356                                    imp,
3357                                    args,
3358                                    catalog,
3359                                    scalars,
3360                                    return_styp,
3361                                )
3362                            },
3363                        );
3364                        handles.push(handle);
3365
3366                        // Advance to the next datum combination.
3367                        for i in (0..datums.len()).rev() {
3368                            idxs[i] += 1;
3369                            if idxs[i] >= datums[i].len() {
3370                                if i == 0 {
3371                                    break;
3372                                }
3373                                idxs[i] = 0;
3374                                continue;
3375                            } else {
3376                                break;
3377                            }
3378                        }
3379                    }
3380                }
3381            }
3382            handles
3383        }
3384
3385        let handles = Catalog::with_debug(|catalog| async { inner(catalog) }).await;
3386        for handle in handles {
3387            handle.await;
3388        }
3389    }
3390
3391    fn smoketest_fn(
3392        name: &&str,
3393        call_name: String,
3394        op: &Operation<HirScalarExpr>,
3395        imp: &FuncImpl<HirScalarExpr>,
3396        args: Vec<Datum<'_>>,
3397        catalog: Arc<Catalog>,
3398        scalars: Vec<CoercibleScalarExpr>,
3399        return_styp: Option<SqlScalarType>,
3400    ) {
3401        let conn_catalog = catalog.for_system_session();
3402        let pcx = PlanContext::zero();
3403        let scx = StatementContext::new(Some(&pcx), &conn_catalog);
3404        let qcx = QueryContext::root(&scx, QueryLifetime::OneShot);
3405        let ecx = ExprContext {
3406            qcx: &qcx,
3407            name: "smoketest",
3408            scope: &Scope::empty(),
3409            relation_type: &SqlRelationType::empty(),
3410            allow_aggregates: false,
3411            allow_subqueries: false,
3412            allow_parameters: false,
3413            allow_windows: false,
3414        };
3415        let arena = RowArena::new();
3416        let mut session = Session::<Timestamp>::dummy();
3417        session
3418            .start_transaction(to_datetime(0), None, None)
3419            .expect("must succeed");
3420        let prep_style = ExprPrepOneShot {
3421            logical_time: EvalTime::Time(Timestamp::MIN),
3422            session: &session,
3423            catalog_state: &catalog.state,
3424        };
3425
3426        // Execute the function as much as possible, ensuring no panics occur, but
3427        // otherwise ignoring eval errors. We also do various other checks.
3428        let res = (op.0)(&ecx, scalars, &imp.params, vec![]);
3429        if let Ok(hir) = res {
3430            if let Ok(mut mir) = hir.lower_uncorrelated(catalog.system_config()) {
3431                // Populate unmaterialized functions.
3432                prep_style.prep_scalar_expr(&mut mir).expect("must succeed");
3433
3434                if let Ok(eval_result_datum) = mir.eval(&[], &arena) {
3435                    if let Some(return_styp) = return_styp {
3436                        let mir_typ = mir.typ(&[]);
3437                        // MIR type inference should be consistent with the type
3438                        // we get from the catalog.
3439                        soft_assert_eq_or_log!(
3440                            mir_typ.scalar_type,
3441                            (&return_styp).into(),
3442                            "MIR type did not match the catalog type (cast elimination/repr type error)"
3443                        );
3444                        // The following will check not just that the scalar type
3445                        // is ok, but also catches if the function returned a null
3446                        // but the MIR type inference said "non-nullable".
3447                        if !eval_result_datum.is_instance_of(&mir_typ) {
3448                            panic!(
3449                                "{call_name}: expected return type of {return_styp:?}, got {eval_result_datum}"
3450                            );
3451                        }
3452                        // Check the consistency of `introduces_nulls` and
3453                        // `propagates_nulls` with `MirScalarExpr::typ`.
3454                        if let Some((introduces_nulls, propagates_nulls)) =
3455                            call_introduces_propagates_nulls(&mir)
3456                        {
3457                            if introduces_nulls {
3458                                // If the function introduces_nulls, then the return
3459                                // type should always be nullable, regardless of
3460                                // the nullability of the input types.
3461                                assert!(
3462                                    mir_typ.nullable,
3463                                    "fn named `{}` called on args `{:?}` (lowered to `{}`) yielded mir_typ.nullable: {}",
3464                                    name, args, mir, mir_typ.nullable
3465                                );
3466                            } else {
3467                                let any_input_null = args.iter().any(|arg| arg.is_null());
3468                                if !any_input_null {
3469                                    assert!(
3470                                        !mir_typ.nullable,
3471                                        "fn named `{}` called on args `{:?}` (lowered to `{}`) yielded mir_typ.nullable: {}",
3472                                        name, args, mir, mir_typ.nullable
3473                                    );
3474                                } else if propagates_nulls {
3475                                    // propagates_nulls means the optimizer short-circuits
3476                                    // all-null inputs, so the output must be nullable.
3477                                    assert!(
3478                                        mir_typ.nullable,
3479                                        "fn named `{}` called on args `{:?}` (lowered to `{}`) yielded mir_typ.nullable: {}",
3480                                        name, args, mir, mir_typ.nullable
3481                                    );
3482                                }
3483                                // When propagates_nulls is false, the output may still
3484                                // be nullable if a non-nullable parameter received a null
3485                                // input (per-position null rejection). The is_instance_of
3486                                // check above ensures type consistency.
3487                            }
3488                        }
3489                        // Check that `MirScalarExpr::reduce` yields the same result
3490                        // as the real evaluation.
3491                        let mut reduced = mir.clone();
3492                        reduced.reduce(&[]);
3493                        match reduced {
3494                            MirScalarExpr::Literal(reduce_result, ctyp) => {
3495                                match reduce_result {
3496                                    Ok(reduce_result_row) => {
3497                                        let reduce_result_datum = reduce_result_row.unpack_first();
3498                                        assert_eq!(
3499                                            reduce_result_datum,
3500                                            eval_result_datum,
3501                                            "eval/reduce datum mismatch: fn named `{}` called on args `{:?}` (lowered to `{}`) evaluated to `{}` with typ `{:?}`, but reduced to `{}` with typ `{:?}`",
3502                                            name,
3503                                            args,
3504                                            mir,
3505                                            eval_result_datum,
3506                                            mir_typ.scalar_type,
3507                                            reduce_result_datum,
3508                                            ctyp.scalar_type
3509                                        );
3510                                        // Let's check that the types also match.
3511                                        // (We are not checking nullability here,
3512                                        // because it's ok when we know a more
3513                                        // precise nullability after actually
3514                                        // evaluating a function than before.)
3515                                        assert_eq!(
3516                                            ctyp.scalar_type,
3517                                            mir_typ.scalar_type,
3518                                            "eval/reduce type mismatch: fn named `{}` called on args `{:?}` (lowered to `{}`) evaluated to `{}` with typ `{:?}`, but reduced to `{}` with typ `{:?}`",
3519                                            name,
3520                                            args,
3521                                            mir,
3522                                            eval_result_datum,
3523                                            mir_typ.scalar_type,
3524                                            reduce_result_datum,
3525                                            ctyp.scalar_type
3526                                        );
3527                                    }
3528                                    Err(..) => {} // It's ok, we might have given invalid args to the function
3529                                }
3530                            }
3531                            _ => unreachable!(
3532                                "all args are literals, so should have reduced to a literal"
3533                            ),
3534                        }
3535                    }
3536                }
3537            }
3538        }
3539    }
3540
3541    /// If the given MirScalarExpr
3542    ///  - is a function call, and
3543    ///  - all arguments are literals
3544    /// then it returns whether the called function (introduces_nulls, propagates_nulls).
3545    fn call_introduces_propagates_nulls(mir_func_call: &MirScalarExpr) -> Option<(bool, bool)> {
3546        match mir_func_call {
3547            MirScalarExpr::CallUnary { func, expr } => {
3548                if expr.is_literal() {
3549                    Some((func.introduces_nulls(), func.propagates_nulls()))
3550                } else {
3551                    None
3552                }
3553            }
3554            MirScalarExpr::CallBinary { func, expr1, expr2 } => {
3555                if expr1.is_literal() && expr2.is_literal() {
3556                    Some((func.introduces_nulls(), func.propagates_nulls()))
3557                } else {
3558                    None
3559                }
3560            }
3561            MirScalarExpr::CallVariadic { func, exprs } => {
3562                if exprs.iter().all(|arg| arg.is_literal()) {
3563                    Some((func.introduces_nulls(), func.propagates_nulls()))
3564                } else {
3565                    None
3566                }
3567            }
3568            _ => None,
3569        }
3570    }
3571
3572    // Make sure pg views don't use types that only exist in Materialize.
3573    #[mz_ore::test(tokio::test)]
3574    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
3575    async fn test_pg_views_forbidden_types() {
3576        Catalog::with_debug(|catalog| async move {
3577            let conn_catalog = catalog.for_system_session();
3578
3579            for view in BUILTINS::views().filter(|view| {
3580                view.schema == PG_CATALOG_SCHEMA || view.schema == INFORMATION_SCHEMA
3581            }) {
3582                let item = conn_catalog
3583                    .resolve_item(&PartialItemName {
3584                        database: None,
3585                        schema: Some(view.schema.to_string()),
3586                        item: view.name.to_string(),
3587                    })
3588                    .expect("unable to resolve view")
3589                    // TODO(alter_table)
3590                    .at_version(RelationVersionSelector::Latest);
3591                let full_name = conn_catalog.resolve_full_name(item.name());
3592                let desc = item.relation_desc().expect("invalid item type");
3593                for col_type in desc.iter_types() {
3594                    match &col_type.scalar_type {
3595                        typ @ SqlScalarType::UInt16
3596                        | typ @ SqlScalarType::UInt32
3597                        | typ @ SqlScalarType::UInt64
3598                        | typ @ SqlScalarType::MzTimestamp
3599                        | typ @ SqlScalarType::List { .. }
3600                        | typ @ SqlScalarType::Map { .. }
3601                        | typ @ SqlScalarType::MzAclItem => {
3602                            panic!("{typ:?} type found in {full_name}");
3603                        }
3604                        SqlScalarType::AclItem
3605                        | SqlScalarType::Bool
3606                        | SqlScalarType::Int16
3607                        | SqlScalarType::Int32
3608                        | SqlScalarType::Int64
3609                        | SqlScalarType::Float32
3610                        | SqlScalarType::Float64
3611                        | SqlScalarType::Numeric { .. }
3612                        | SqlScalarType::Date
3613                        | SqlScalarType::Time
3614                        | SqlScalarType::Timestamp { .. }
3615                        | SqlScalarType::TimestampTz { .. }
3616                        | SqlScalarType::Interval
3617                        | SqlScalarType::PgLegacyChar
3618                        | SqlScalarType::Bytes
3619                        | SqlScalarType::String
3620                        | SqlScalarType::Char { .. }
3621                        | SqlScalarType::VarChar { .. }
3622                        | SqlScalarType::Jsonb
3623                        | SqlScalarType::Uuid
3624                        | SqlScalarType::Array(_)
3625                        | SqlScalarType::Record { .. }
3626                        | SqlScalarType::Oid
3627                        | SqlScalarType::RegProc
3628                        | SqlScalarType::RegType
3629                        | SqlScalarType::RegClass
3630                        | SqlScalarType::Int2Vector
3631                        | SqlScalarType::Range { .. }
3632                        | SqlScalarType::PgLegacyName => {}
3633                    }
3634                }
3635            }
3636            catalog.expire().await;
3637        })
3638        .await
3639    }
3640
3641    // Make sure objects reside in the `mz_introspection` schema iff they depend on per-replica
3642    // introspection relations.
3643    #[mz_ore::test(tokio::test)]
3644    #[cfg_attr(miri, ignore)] // unsupported operation: returning ready events from epoll_wait is not yet implemented
3645    async fn test_mz_introspection_builtins() {
3646        Catalog::with_debug(|catalog| async move {
3647            let conn_catalog = catalog.for_system_session();
3648
3649            let introspection_schema_id = catalog.get_mz_introspection_schema_id();
3650            let introspection_schema_spec = SchemaSpecifier::Id(introspection_schema_id);
3651
3652            for entry in catalog.entries() {
3653                let schema_spec = entry.name().qualifiers.schema_spec;
3654                let introspection_deps = catalog.introspection_dependencies(entry.id);
3655                if introspection_deps.is_empty() {
3656                    assert!(
3657                        schema_spec != introspection_schema_spec,
3658                        "entry does not depend on introspection sources but is in \
3659                         `mz_introspection`: {}",
3660                        conn_catalog.resolve_full_name(entry.name()),
3661                    );
3662                } else {
3663                    assert!(
3664                        schema_spec == introspection_schema_spec,
3665                        "entry depends on introspection sources but is not in \
3666                         `mz_introspection`: {}",
3667                        conn_catalog.resolve_full_name(entry.name()),
3668                    );
3669                }
3670            }
3671        })
3672        .await
3673    }
3674
3675    #[mz_ore::test(tokio::test)]
3676    #[cfg_attr(miri, ignore)] //  unsupported operation: can't call foreign function `TLS_client_method` on OS `linux`
3677    async fn test_multi_subscriber_catalog() {
3678        let persist_client = PersistClient::new_for_tests().await;
3679        let bootstrap_args = test_bootstrap_args();
3680        let organization_id = Uuid::new_v4();
3681        let db_name = "DB";
3682
3683        let mut writer_catalog = Catalog::open_debug_catalog(
3684            persist_client.clone(),
3685            organization_id.clone(),
3686            &bootstrap_args,
3687        )
3688        .await
3689        .expect("open_debug_catalog");
3690        let mut read_only_catalog = Catalog::open_debug_read_only_catalog(
3691            persist_client.clone(),
3692            organization_id.clone(),
3693            &bootstrap_args,
3694        )
3695        .await
3696        .expect("open_debug_read_only_catalog");
3697        assert_err!(writer_catalog.resolve_database(db_name));
3698        assert_err!(read_only_catalog.resolve_database(db_name));
3699
3700        let commit_ts = writer_catalog.current_upper().await;
3701        writer_catalog
3702            .transact(
3703                None,
3704                commit_ts,
3705                None,
3706                vec![Op::CreateDatabase {
3707                    name: db_name.to_string(),
3708                    owner_id: MZ_SYSTEM_ROLE_ID,
3709                }],
3710            )
3711            .await
3712            .expect("failed to transact");
3713
3714        let write_db = writer_catalog
3715            .resolve_database(db_name)
3716            .expect("resolve_database");
3717        read_only_catalog
3718            .sync_to_current_updates()
3719            .await
3720            .expect("sync_to_current_updates");
3721        let read_db = read_only_catalog
3722            .resolve_database(db_name)
3723            .expect("resolve_database");
3724
3725        assert_eq!(write_db, read_db);
3726
3727        let writer_catalog_fencer =
3728            Catalog::open_debug_catalog(persist_client, organization_id, &bootstrap_args)
3729                .await
3730                .expect("open_debug_catalog for fencer");
3731        let fencer_db = writer_catalog_fencer
3732            .resolve_database(db_name)
3733            .expect("resolve_database for fencer");
3734        assert_eq!(fencer_db, read_db);
3735
3736        let write_fence_err = writer_catalog
3737            .sync_to_current_updates()
3738            .await
3739            .expect_err("sync_to_current_updates for fencer");
3740        assert!(matches!(
3741            write_fence_err,
3742            CatalogError::Durable(DurableCatalogError::Fence(FenceError::Epoch { .. }))
3743        ));
3744        let read_fence_err = read_only_catalog
3745            .sync_to_current_updates()
3746            .await
3747            .expect_err("sync_to_current_updates after fencer");
3748        assert!(matches!(
3749            read_fence_err,
3750            CatalogError::Durable(DurableCatalogError::Fence(FenceError::Epoch { .. }))
3751        ));
3752
3753        writer_catalog.expire().await;
3754        read_only_catalog.expire().await;
3755        writer_catalog_fencer.expire().await;
3756    }
3757}