diff --git a/src/adapter/src/catalog.rs b/src/adapter/src/catalog.rs index 0da27071c6fe..d3b64a80877b 100644 --- a/src/adapter/src/catalog.rs +++ b/src/adapter/src/catalog.rs @@ -95,7 +95,8 @@ pub mod builtin; pub mod storage; pub const SYSTEM_CONN_ID: ConnectionId = 0; -const SYSTEM_USER: &str = "mz_system"; +pub const SYSTEM_USER: &str = "mz_system"; +pub const HTTP_DEFAULT_USER: &str = "mz_http_default_user"; const CREATE_SQL_TODO: &str = "TODO"; /// A `Catalog` keeps track of the SQL objects known to the planner. diff --git a/src/adapter/src/catalog/builtin.rs b/src/adapter/src/catalog/builtin.rs index 85dbe12e7123..e8cfd6877fd9 100644 --- a/src/adapter/src/catalog/builtin.rs +++ b/src/adapter/src/catalog/builtin.rs @@ -32,6 +32,8 @@ use mz_compute_client::logging::{ComputeLog, DifferentialLog, LogVariant, Timely use mz_repr::{RelationDesc, ScalarType}; use mz_sql::catalog::{CatalogType, CatalogTypeDetails, NameReference, TypeReference}; +use crate::catalog::{HTTP_DEFAULT_USER, SYSTEM_USER}; + pub const MZ_TEMP_SCHEMA: &str = "mz_temp"; pub const MZ_CATALOG_SCHEMA: &str = "mz_catalog"; pub const PG_CATALOG_SCHEMA: &str = "pg_catalog"; @@ -2105,7 +2107,11 @@ AS SELECT FROM mz_catalog.mz_roles r", }; -pub const MZ_SYSTEM: BuiltinRole = BuiltinRole { name: "mz_system" }; +pub const MZ_SYSTEM: BuiltinRole = BuiltinRole { name: SYSTEM_USER }; + +pub const MZ_HTTP_DEFAULT_USER: BuiltinRole = BuiltinRole { + name: HTTP_DEFAULT_USER, +}; pub static BUILTINS_STATIC: Lazy>> = Lazy::new(|| { let mut builtins = vec![ @@ -2289,7 +2295,8 @@ pub static BUILTINS_STATIC: Lazy>> = Lazy::new(|| { builtins }); -pub static BUILTIN_ROLES: Lazy> = Lazy::new(|| vec![MZ_SYSTEM]); +pub static BUILTIN_ROLES: Lazy> = + Lazy::new(|| vec![MZ_SYSTEM, MZ_HTTP_DEFAULT_USER]); #[allow(non_snake_case)] pub mod BUILTINS { diff --git a/src/adapter/src/client.rs b/src/adapter/src/client.rs index ee863d0338af..449880235c15 100644 --- a/src/adapter/src/client.rs +++ b/src/adapter/src/client.rs @@ -20,6 +20,7 @@ use mz_ore::thread::JoinOnDropHandle; use mz_repr::{Datum, GlobalId, Row, ScalarType}; use mz_sql::ast::{Raw, Statement}; +use crate::catalog::SYSTEM_USER; use crate::command::{ Canceled, Command, ExecuteResponse, Response, SimpleExecuteResponse, SimpleResult, StartupResponse, @@ -67,6 +68,15 @@ impl Handle { } } +/// Distinguishes between external and internal clients. +#[derive(Debug, Clone)] +pub enum ClientType { + /// Client that has connected over an external port. + External, + /// Client that has connected over an internal port. + Internal, +} + /// A coordinator client. /// /// A coordinator client is a simple handle to a communication channel with the @@ -78,13 +88,15 @@ impl Handle { pub struct Client { cmd_tx: mpsc::UnboundedSender, id_alloc: Arc>, + client_type: ClientType, } impl Client { - pub(crate) fn new(cmd_tx: mpsc::UnboundedSender) -> Client { + pub(crate) fn new(cmd_tx: mpsc::UnboundedSender, client_type: ClientType) -> Client { Client { cmd_tx, id_alloc: Arc::new(IdAllocator::new(1, 1 << 16)), + client_type, } } @@ -103,7 +115,7 @@ impl Client { /// a system user. pub async fn system_execute(&self, stmts: &str) -> Result { let conn_client = self.new_conn()?; - let session = Session::new(conn_client.conn_id(), "mz_system".into()); + let session = Session::new(conn_client.conn_id(), SYSTEM_USER.into()); let (mut session_client, _) = conn_client.startup(session, false).await?; session_client.simple_execute(stmts).await } @@ -152,6 +164,8 @@ impl ConnClient { session: Session, create_user_if_not_exists: bool, ) -> Result<(SessionClient, StartupResponse), AdapterError> { + self.validate_user(session.user())?; + // Cancellation works by creating a watch channel (which remembers only // the last value sent to it) and sharing it between the coordinator and // connection. The coordinator will send a canceled message on it if a @@ -186,6 +200,15 @@ impl ConnClient { } } + /// Validate that the user is allowed to login. + fn validate_user(&self, user: &str) -> Result<(), AdapterError> { + if user == SYSTEM_USER && !matches!(self.inner.client_type, ClientType::Internal) { + return Err(AdapterError::UnauthorizedLogin(user.to_string())); + } + + Ok(()) + } + /// Cancels the query currently running on another connection. pub async fn cancel_request(&mut self, conn_id: ConnectionId, secret_key: u32) { self.inner diff --git a/src/adapter/src/coord.rs b/src/adapter/src/coord.rs index c515336efc87..97750afcc685 100644 --- a/src/adapter/src/coord.rs +++ b/src/adapter/src/coord.rs @@ -77,7 +77,6 @@ use chrono::{DateTime, Utc}; use derivative::Derivative; use futures::StreamExt; use itertools::Itertools; -use mz_ore::tracing::OpenTelemetryContext; use rand::seq::SliceRandom; use timely::progress::{Antichain, Timestamp as TimelyTimestamp}; use tokio::runtime::Handle as TokioHandle; @@ -94,6 +93,7 @@ use mz_ore::metrics::MetricsRegistry; use mz_ore::now::NowFn; use mz_ore::stack; use mz_ore::thread::JoinHandleExt; +use mz_ore::tracing::OpenTelemetryContext; use mz_persist_client::usage::StorageUsageClient; use mz_repr::{Datum, Diff, GlobalId, Row, Timestamp}; use mz_secrets::SecretsController; @@ -112,7 +112,7 @@ use crate::catalog::{ self, storage, BuiltinMigrationMetadata, BuiltinTableUpdate, Catalog, CatalogItem, ClusterReplicaSizeMap, Sink, SinkConnectionState, StorageHostSizeMap, }; -use crate::client::{Client, ConnectionId, Handle}; +use crate::client::{Client, ClientType, ConnectionId, Handle}; use crate::command::{Canceled, Command, ExecuteResponse}; use crate::coord::appends::{AdvanceLocalInput, Deferred, PendingWriteTxn}; use crate::coord::id_bundle::CollectionIdBundle; @@ -840,7 +840,7 @@ pub async fn serve( connection_context, storage_usage_client, }: Config, -) -> Result<(Handle, Client), AdapterError> { +) -> Result<(Handle, Client, Client), AdapterError> { let (cmd_tx, cmd_rx) = mpsc::unbounded_channel(); let (internal_cmd_tx, internal_cmd_rx) = mpsc::unbounded_channel(); let (strict_serializable_reads_tx, strict_serializable_reads_rx) = mpsc::unbounded_channel(); @@ -957,8 +957,9 @@ pub async fn serve( start_instant, _thread: thread.join_on_drop(), }; - let client = Client::new(cmd_tx); - Ok((handle, client)) + let external_client = Client::new(cmd_tx.clone(), ClientType::External); + let internal_client = Client::new(cmd_tx, ClientType::Internal); + Ok((handle, external_client, internal_client)) } Err(e) => Err(e), } diff --git a/src/adapter/src/error.rs b/src/adapter/src/error.rs index e67c0366cb7e..dc50ce6ead62 100644 --- a/src/adapter/src/error.rs +++ b/src/adapter/src/error.rs @@ -126,6 +126,8 @@ pub enum AdapterError { TailOnlyTransaction, /// An error occurred in the MIR stage of the optimizer. Transform(TransformError), + /// Unauthorized to login using the provided user. + UnauthorizedLogin(String), /// The specified function cannot be called UncallableFunction { func: UnmaterializableFunc, @@ -364,6 +366,9 @@ impl fmt::Display for AdapterError { f.write_str("TAIL in transactions must be the only read statement") } AdapterError::Transform(e) => e.fmt(f), + AdapterError::UnauthorizedLogin(user) => { + write!(f, "unauthorized login to user '{user}'") + } AdapterError::UncallableFunction { func, context } => { write!(f, "cannot call {} in {}", func, context) } diff --git a/src/adapter/src/session.rs b/src/adapter/src/session.rs index 20eba502602c..f9714ed776aa 100644 --- a/src/adapter/src/session.rs +++ b/src/adapter/src/session.rs @@ -27,19 +27,20 @@ use mz_sql::ast::{Raw, Statement, TransactionAccessMode}; use mz_sql::plan::{Params, PlanContext, StatementDesc}; use mz_sql_parser::ast::TransactionIsolationLevel; +use crate::catalog::SYSTEM_USER; use crate::client::ConnectionId; use crate::coord::peek::PeekResponseUnary; use crate::coord::CoordTimestamp; use crate::error::AdapterError; use crate::session::vars::IsolationLevel; -pub(crate) mod vars; - pub use self::vars::{ ClientSeverity, SessionVars, Var, DEFAULT_DATABASE_NAME, SERVER_MAJOR_VERSION, SERVER_MINOR_VERSION, SERVER_PATCH_VERSION, }; +pub(crate) mod vars; + const DUMMY_CONNECTION_ID: ConnectionId = 0; /// A session holds per-connection state. @@ -67,7 +68,7 @@ impl Session { /// Dummy sessions are intended for use when executing queries on behalf of /// the system itself, rather than on behalf of a user. pub fn dummy() -> Session { - Self::new_internal(DUMMY_CONNECTION_ID, "mz_system".into()) + Self::new_internal(DUMMY_CONNECTION_ID, SYSTEM_USER.into()) } fn new_internal(conn_id: ConnectionId, user: String) -> Session { diff --git a/src/environmentd/src/http.rs b/src/environmentd/src/http.rs index 6240ca453410..0da119b93787 100644 --- a/src/environmentd/src/http.rs +++ b/src/environmentd/src/http.rs @@ -34,9 +34,7 @@ use http::header::{AUTHORIZATION, CONTENT_TYPE}; use http::{Request, StatusCode}; use hyper::server::conn::AddrIncoming; use hyper_openssl::MaybeHttpsStream; -use mz_adapter::SessionClient; -use mz_ore::metrics::MetricsRegistry; -use mz_ore::tracing::OpenTelemetryEnableCallback; +use mz_adapter::catalog::HTTP_DEFAULT_USER; use openssl::nid::Nid; use openssl::ssl::{Ssl, SslContext}; use openssl::x509::X509; @@ -48,7 +46,10 @@ use tower_http::cors::{AllowOrigin, Any, CorsLayer}; use tracing::{error, warn}; use mz_adapter::session::Session; +use mz_adapter::SessionClient; use mz_frontegg_auth::{FronteggAuthentication, FronteggError}; +use mz_ore::metrics::MetricsRegistry; +use mz_ore::tracing::OpenTelemetryEnableCallback; use crate::BUILD_INFO; @@ -57,8 +58,6 @@ mod memory; mod root; mod sql; -const SYSTEM_USER: &str = "mz_system"; - #[derive(Debug, Clone)] pub struct Config { pub tls: Option, @@ -278,7 +277,7 @@ async fn auth( let user = match frontegg { // If no Frontegg authentication, we can use the cert's username if // present, otherwise the system user. - None => user.unwrap_or_else(|| SYSTEM_USER.to_string()), + None => user.unwrap_or_else(|| HTTP_DEFAULT_USER.to_string()), // If we require Frontegg auth, fetch credentials from the HTTP auth // header. Basic auth comes with a username/password, where the password // is the client+secret pair. Bearer auth is an existing JWT that must diff --git a/src/environmentd/src/lib.rs b/src/environmentd/src/lib.rs index ef615ae2a1b1..6e0a119f61a6 100644 --- a/src/environmentd/src/lib.rs +++ b/src/environmentd/src/lib.rs @@ -26,6 +26,7 @@ use tokio::net::TcpListener; use tokio::sync::oneshot; use tokio_stream::wrappers::TcpListenerStream; use tower_http::cors::AllowOrigin; +use tracing::{error, info}; use mz_adapter::catalog::storage::BootstrapArgs; use mz_adapter::catalog::{ClusterReplicaSizeMap, StorageHostSizeMap}; @@ -39,7 +40,6 @@ use mz_ore::tracing::OpenTelemetryEnableCallback; use mz_persist_client::usage::StorageUsageClient; use mz_secrets::SecretsController; use mz_storage::types::connections::ConnectionContext; -use tracing::{error, info}; use crate::tcp_connection::ConnectionHandler; @@ -259,22 +259,23 @@ pub async fn serve(config: Config) -> Result { // Initialize controller. let controller = mz_controller::Controller::new(config.controller).await; // Initialize adapter. - let (adapter_handle, adapter_client) = mz_adapter::serve(mz_adapter::Config { - dataflow_client: controller, - storage: adapter_storage, - unsafe_mode: config.unsafe_mode, - build_info: &BUILD_INFO, - metrics_registry: config.metrics_registry.clone(), - now: config.now, - secrets_controller: config.secrets_controller, - cluster_replica_sizes: config.cluster_replica_sizes, - storage_host_sizes: config.storage_host_sizes, - default_storage_host_size: config.default_storage_host_size, - availability_zones: config.availability_zones, - connection_context: config.connection_context, - storage_usage_client, - }) - .await?; + let (adapter_handle, external_adapter_client, internal_adapter_client) = + mz_adapter::serve(mz_adapter::Config { + dataflow_client: controller, + storage: adapter_storage, + unsafe_mode: config.unsafe_mode, + build_info: &BUILD_INFO, + metrics_registry: config.metrics_registry.clone(), + now: config.now, + secrets_controller: config.secrets_controller, + cluster_replica_sizes: config.cluster_replica_sizes, + storage_host_sizes: config.storage_host_sizes, + default_storage_host_size: config.default_storage_host_size, + availability_zones: config.availability_zones, + connection_context: config.connection_context, + storage_usage_client, + }) + .await?; // TODO(benesch): replace both `TCPListenerStream`s below with // `_listener.incoming()` if that is @@ -291,7 +292,7 @@ pub async fn serve(config: Config) -> Result { task::spawn(|| "pgwire_server", { let pgwire_server = mz_pgwire::Server::new(mz_pgwire::Config { tls: pgwire_tls, - adapter_client: adapter_client.clone(), + adapter_client: external_adapter_client.clone(), frontegg: config.frontegg.clone(), }); @@ -311,7 +312,7 @@ pub async fn serve(config: Config) -> Result { task::spawn(|| "internal_pgwire_server", { let internal_pgwire_server = mz_pgwire::Server::new(mz_pgwire::Config { tls: None, - adapter_client: adapter_client.clone(), + adapter_client: internal_adapter_client.clone(), frontegg: None, }); let mut incoming = TcpListenerStream::new(internal_sql_listener); @@ -330,7 +331,7 @@ pub async fn serve(config: Config) -> Result { let http_server = http::Server::new(http::Config { tls: http_tls, frontegg: config.frontegg, - adapter_client, + adapter_client: external_adapter_client, allowed_origin: config.cors_allowed_origin, }); let mut incoming = TcpListenerStream::new(http_listener); diff --git a/src/environmentd/tests/auth.rs b/src/environmentd/tests/auth.rs index 5cf9f8f418e5..a197c6898f6e 100644 --- a/src/environmentd/tests/auth.rs +++ b/src/environmentd/tests/auth.rs @@ -30,7 +30,6 @@ use hyper::service::{make_service_fn, service_fn}; use hyper::{body, Body, Request, Response, Server, StatusCode, Uri}; use hyper_openssl::HttpsConnector; use jsonwebtoken::{self, DecodingKey, EncodingKey}; -use mz_ore::now::SYSTEM_TIME; use openssl::asn1::Asn1Time; use openssl::error::ErrorStack; use openssl::hash::MessageDigest; @@ -51,6 +50,7 @@ use tempfile::TempDir; use tokio::runtime::Runtime; use uuid::Uuid; +use mz_adapter::catalog::HTTP_DEFAULT_USER; use mz_environmentd::TlsMode; use mz_frontegg_auth::{ ApiTokenArgs, ApiTokenResponse, Claims, FronteggAuthentication, FronteggConfig, RefreshToken, @@ -58,6 +58,7 @@ use mz_frontegg_auth::{ }; use mz_ore::assert_contains; use mz_ore::now::NowFn; +use mz_ore::now::SYSTEM_TIME; use mz_ore::retry::Retry; use mz_ore::task::RuntimeExt; @@ -988,7 +989,7 @@ fn test_auth() -> Result<(), Box> { assert: Assert::Success, }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTP, headers: &no_headers, configure: Box::new(|_| Ok(())), @@ -1016,7 +1017,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|_| Ok(())), @@ -1053,7 +1054,7 @@ fn test_auth() -> Result<(), Box> { // Test that specifying an mzcloud header does nothing and uses the default // user. TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &frontegg_header_basic, configure: Box::new(|b| Ok(b.set_verify(SslVerifyMode::NONE))), @@ -1075,7 +1076,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTP, headers: &no_headers, configure: Box::new(|_| Ok(())), @@ -1098,7 +1099,7 @@ fn test_auth() -> Result<(), Box> { assert: Assert::Success, }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|b| Ok(b.set_verify(SslVerifyMode::NONE))), @@ -1143,7 +1144,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTP, headers: &no_headers, configure: Box::new(|_| Ok(())), @@ -1164,7 +1165,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|b| Ok(b.set_verify(SslVerifyMode::NONE))), @@ -1188,7 +1189,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|b| { @@ -1216,7 +1217,7 @@ fn test_auth() -> Result<(), Box> { TestCase::Http { // In verify-ca mode, the HTTP interface ignores the // certificate's user. - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|b| { @@ -1280,7 +1281,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTP, headers: &no_headers, configure: Box::new(|_| Ok(())), @@ -1301,7 +1302,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|b| Ok(b.set_verify(SslVerifyMode::NONE))), @@ -1325,7 +1326,7 @@ fn test_auth() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &no_headers, configure: Box::new(|b| { @@ -1450,7 +1451,7 @@ fn test_auth_intermediate_ca() -> Result<(), Box> { })), }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &HeaderMap::new(), configure: Box::new(|b| b.set_ca_file(ca.ca_cert_path())), @@ -1480,7 +1481,7 @@ fn test_auth_intermediate_ca() -> Result<(), Box> { assert: Assert::Success, }, TestCase::Http { - user: "mz_system", + user: HTTP_DEFAULT_USER, scheme: Scheme::HTTPS, headers: &HeaderMap::new(), configure: Box::new(|b| b.set_ca_file(ca.ca_cert_path())), diff --git a/src/environmentd/tests/sql.rs b/src/environmentd/tests/sql.rs index 2c301a9db6f6..511fddcda861 100644 --- a/src/environmentd/tests/sql.rs +++ b/src/environmentd/tests/sql.rs @@ -13,7 +13,6 @@ //! scripts. The tests here are simply too complicated to be easily expressed //! in testdrive, e.g., because they depend on the current time. -use anyhow::anyhow; use std::error::Error; use std::net::{Ipv4Addr, SocketAddr}; use std::sync::Arc; @@ -21,12 +20,12 @@ use std::sync::Mutex; use std::time::{Duration, Instant}; use std::{env, thread}; +use anyhow::anyhow; use axum::response::IntoResponse; use axum::response::Response; use axum::{routing, Json, Router}; use chrono::{DateTime, Utc}; use http::StatusCode; -use mz_ore::retry::Retry; use postgres::Row; use regex::Regex; use serde_json::json; @@ -36,8 +35,10 @@ use tokio_postgres::config::Host; use tokio_postgres::Client; use tracing::info; +use mz_adapter::catalog::SYSTEM_USER; use mz_ore::assert_contains; use mz_ore::now::{EpochMillis, NowFn, NOW_ZERO, SYSTEM_TIME}; +use mz_ore::retry::Retry; use mz_ore::task::{self, AbortOnDropHandle, JoinHandleExt}; use crate::util::{MzTimestamp, PostgresErrorExt, KAFKA_ADDRS}; @@ -1385,6 +1386,27 @@ fn test_linearizability() -> Result<(), Box> { Ok(()) } +#[test] +fn test_system_user() -> Result<(), Box> { + mz_ore::test::init_logging(); + + let config = util::Config::default(); + let server = util::start_server(config)?; + + assert!(server + .pg_config() + .user(SYSTEM_USER) + .connect(postgres::NoTls) + .is_err()); + assert!(server + .pg_config_internal() + .user(SYSTEM_USER) + .connect(postgres::NoTls) + .is_ok()); + + Ok(()) +} + /// Group commit will block writes until the current time has advanced. This can make /// performing inserts while using deterministic time difficult. This is a helper /// method to perform writes and advance the current time. diff --git a/src/environmentd/tests/util.rs b/src/environmentd/tests/util.rs index 41de1248dbb6..47abe888f119 100644 --- a/src/environmentd/tests/util.rs +++ b/src/environmentd/tests/util.rs @@ -227,6 +227,16 @@ impl Server { config } + pub fn pg_config_internal(&self) -> postgres::Config { + let local_addr = self.inner.internal_sql_local_addr(); + let mut config = postgres::Config::new(); + config + .host(&Ipv4Addr::LOCALHOST.to_string()) + .port(local_addr.port()) + .user("materialize"); + config + } + pub fn pg_config_async(&self) -> tokio_postgres::Config { let local_addr = self.inner.sql_local_addr(); let mut config = tokio_postgres::Config::new(); diff --git a/src/pgwire/src/message.rs b/src/pgwire/src/message.rs index 81b168e698f3..fe45c79f8e36 100644 --- a/src/pgwire/src/message.rs +++ b/src/pgwire/src/message.rs @@ -404,6 +404,7 @@ impl ErrorResponse { AdapterError::SqlCatalog(_) => SqlState::INTERNAL_ERROR, AdapterError::TailOnlyTransaction => SqlState::INVALID_TRANSACTION_STATE, AdapterError::Transform(_) => SqlState::INTERNAL_ERROR, + AdapterError::UnauthorizedLogin(_) => SqlState::INSUFFICIENT_PRIVILEGE, AdapterError::UncallableFunction { .. } => SqlState::FEATURE_NOT_SUPPORTED, AdapterError::UnknownCursor(_) => SqlState::INVALID_CURSOR_NAME, AdapterError::UnknownParameter(_) => SqlState::UNDEFINED_OBJECT, diff --git a/test/sqllogictest/id_reuse.slt b/test/sqllogictest/id_reuse.slt index f2a0e95c489b..49d19ad91623 100644 --- a/test/sqllogictest/id_reuse.slt +++ b/test/sqllogictest/id_reuse.slt @@ -68,6 +68,7 @@ query TT rowsort SELECT id, name FROM mz_roles ---- s1 mz_system +s2 mz_http_default_user u1 materialize u2 foo @@ -81,6 +82,7 @@ query TT rowsort SELECT id, name FROM mz_roles ---- s1 mz_system +s2 mz_http_default_user u1 materialize u3 bar diff --git a/test/sqllogictest/pg_catalog_roles.slt b/test/sqllogictest/pg_catalog_roles.slt index 4c189eede178..b0a7d60581df 100644 --- a/test/sqllogictest/pg_catalog_roles.slt +++ b/test/sqllogictest/pg_catalog_roles.slt @@ -13,4 +13,5 @@ query IT SELECT oid, rolname FROM pg_roles ORDER BY oid ---- 20007 mz_system -20008 materialize +20008 mz_http_default_user +20009 materialize diff --git a/test/testdrive/roles.td b/test/testdrive/roles.td index c9069b47609a..f5b0042fe433 100644 --- a/test/testdrive/roles.td +++ b/test/testdrive/roles.td @@ -12,6 +12,7 @@ $ set-sql-timeout duration=1s # Verify initial roles. > SELECT id, name FROM mz_roles s1 mz_system +s2 mz_http_default_user u1 materialize # Verify that invalid options are rejected. @@ -29,6 +30,7 @@ contains:conflicting or redundant options > CREATE USER fms SUPERUSER > SELECT id, name FROM mz_roles s1 mz_system +s2 mz_http_default_user u1 materialize u2 rj u3 fms @@ -39,6 +41,7 @@ u3 fms contains:unknown role 'bad' > SELECT id, name FROM mz_roles s1 mz_system +s2 mz_http_default_user u1 materialize u2 rj u3 fms @@ -47,17 +50,20 @@ u3 fms > DROP ROLE IF EXISTS rj, fms, bad > SELECT id, name FROM mz_roles s1 mz_system +s2 mz_http_default_user u1 materialize # Verify that the single name version of DROP ROLE works too. > CREATE ROLE nlb LOGIN SUPERUSER > SELECT id, name FROM mz_roles s1 mz_system +s2 mz_http_default_user u1 materialize u4 nlb > DROP ROLE nlb > SELECT id, name FROM mz_roles s1 mz_system +s2 mz_http_default_user u1 materialize > DROP ROLE IF EXISTS nlb diff --git a/test/upgrade/check-from-current_source-role.td b/test/upgrade/check-from-current_source-role.td index 1e38d7a0542a..289a655f66b1 100644 --- a/test/upgrade/check-from-current_source-role.td +++ b/test/upgrade/check-from-current_source-role.td @@ -9,6 +9,7 @@ > SELECT name FROM mz_roles; mz_system +mz_http_default_user materialize superuser_login "space role"