diff --git a/compute_tools/src/catalog.rs b/compute_tools/src/catalog.rs index 2f6f82dd39..08ae8bf44d 100644 --- a/compute_tools/src/catalog.rs +++ b/compute_tools/src/catalog.rs @@ -1,4 +1,3 @@ -use compute_api::responses::CatalogObjects; use futures::Stream; use postgres::NoTls; use std::{path::Path, process::Stdio, result::Result, sync::Arc}; @@ -13,7 +12,8 @@ use tokio_util::codec::{BytesCodec, FramedRead}; use tracing::warn; use crate::compute::ComputeNode; -use crate::pg_helpers::{get_existing_dbs_async, get_existing_roles_async}; +use crate::pg_helpers::{get_existing_dbs_async, get_existing_roles_async, postgres_conf_for_db}; +use compute_api::responses::CatalogObjects; pub async fn get_dbs_and_roles(compute: &Arc) -> anyhow::Result { let connstr = compute.connstr.clone(); @@ -43,6 +43,8 @@ pub enum SchemaDumpError { DatabaseDoesNotExist, #[error("Failed to execute pg_dump.")] IO(#[from] std::io::Error), + #[error("Unexpected error.")] + Unexpected, } // It uses the pg_dump utility to dump the schema of the specified database. @@ -60,11 +62,38 @@ pub async fn get_database_schema( let pgbin = &compute.pgbin; let basepath = Path::new(pgbin).parent().unwrap(); let pgdump = basepath.join("pg_dump"); - let mut connstr = compute.connstr.clone(); - connstr.set_path(dbname); + + // Replace the DB in the connection string and disable it to parts. + // This is the only option to handle DBs with special characters. + let conf = + postgres_conf_for_db(&compute.connstr, dbname).map_err(|_| SchemaDumpError::Unexpected)?; + let host = conf + .get_hosts() + .first() + .ok_or(SchemaDumpError::Unexpected)?; + let host = match host { + tokio_postgres::config::Host::Tcp(ip) => ip.to_string(), + #[cfg(unix)] + tokio_postgres::config::Host::Unix(path) => path.to_string_lossy().to_string(), + }; + let port = conf + .get_ports() + .first() + .ok_or(SchemaDumpError::Unexpected)?; + let user = conf.get_user().ok_or(SchemaDumpError::Unexpected)?; + let dbname = conf.get_dbname().ok_or(SchemaDumpError::Unexpected)?; + let mut cmd = Command::new(pgdump) + // XXX: this seems to be the only option to deal with DBs with `=` in the name + // See + .env("PGDATABASE", dbname) + .arg("--host") + .arg(host) + .arg("--port") + .arg(port.to_string()) + .arg("--username") + .arg(user) .arg("--schema-only") - .arg(connstr.as_str()) .stdout(Stdio::piped()) .stderr(Stdio::piped()) .kill_on_drop(true) diff --git a/compute_tools/src/compute.rs b/compute_tools/src/compute.rs index 4f67425ba8..1a026a4014 100644 --- a/compute_tools/src/compute.rs +++ b/compute_tools/src/compute.rs @@ -34,9 +34,8 @@ use utils::measured_stream::MeasuredReader; use nix::sys::signal::{kill, Signal}; use remote_storage::{DownloadError, RemotePath}; use tokio::spawn; -use url::Url; -use crate::installed_extensions::get_installed_extensions_sync; +use crate::installed_extensions::get_installed_extensions; use crate::local_proxy; use crate::pg_helpers::*; use crate::spec::*; @@ -816,30 +815,32 @@ impl ComputeNode { Ok(()) } - async fn get_maintenance_client(url: &Url) -> Result { - let mut connstr = url.clone(); + async fn get_maintenance_client( + conf: &tokio_postgres::Config, + ) -> Result { + let mut conf = conf.clone(); - connstr - .query_pairs_mut() - .append_pair("application_name", "apply_config"); + conf.application_name("apply_config"); - let (client, conn) = match tokio_postgres::connect(connstr.as_str(), NoTls).await { + let (client, conn) = match conf.connect(NoTls).await { + // If connection fails, it may be the old node with `zenith_admin` superuser. + // + // In this case we need to connect with old `zenith_admin` name + // and create new user. We cannot simply rename connected user, + // but we can create a new one and grant it all privileges. Err(e) => match e.code() { Some(&SqlState::INVALID_PASSWORD) | Some(&SqlState::INVALID_AUTHORIZATION_SPECIFICATION) => { - // connect with zenith_admin if cloud_admin could not authenticate + // Connect with zenith_admin if cloud_admin could not authenticate info!( "cannot connect to postgres: {}, retrying with `zenith_admin` username", e ); - let mut zenith_admin_connstr = connstr.clone(); - - zenith_admin_connstr - .set_username("zenith_admin") - .map_err(|_| anyhow::anyhow!("invalid connstr"))?; + let mut zenith_admin_conf = postgres::config::Config::from(conf.clone()); + zenith_admin_conf.user("zenith_admin"); let mut client = - Client::connect(zenith_admin_connstr.as_str(), NoTls) + zenith_admin_conf.connect(NoTls) .context("broken cloud_admin credential: tried connecting with cloud_admin but could not authenticate, and zenith_admin does not work either")?; // Disable forwarding so that users don't get a cloud_admin role @@ -853,8 +854,8 @@ impl ComputeNode { drop(client); - // reconnect with connstring with expected name - tokio_postgres::connect(connstr.as_str(), NoTls).await? + // Reconnect with connstring with expected name + conf.connect(NoTls).await? } _ => return Err(e.into()), }, @@ -885,7 +886,7 @@ impl ComputeNode { pub fn apply_spec_sql( &self, spec: Arc, - url: Arc, + conf: Arc, concurrency: usize, ) -> Result<()> { let rt = tokio::runtime::Builder::new_multi_thread() @@ -897,7 +898,7 @@ impl ComputeNode { rt.block_on(async { // Proceed with post-startup configuration. Note, that order of operations is important. - let client = Self::get_maintenance_client(&url).await?; + let client = Self::get_maintenance_client(&conf).await?; let spec = spec.clone(); let databases = get_existing_dbs_async(&client).await?; @@ -931,7 +932,7 @@ impl ComputeNode { RenameAndDeleteDatabases, CreateAndAlterDatabases, ] { - debug!("Applying phase {:?}", &phase); + info!("Applying phase {:?}", &phase); apply_operations( spec.clone(), ctx.clone(), @@ -942,6 +943,7 @@ impl ComputeNode { .await?; } + info!("Applying RunInEachDatabase phase"); let concurrency_token = Arc::new(tokio::sync::Semaphore::new(concurrency)); let db_processes = spec @@ -955,7 +957,7 @@ impl ComputeNode { let spec = spec.clone(); let ctx = ctx.clone(); let jwks_roles = jwks_roles.clone(); - let mut url = url.as_ref().clone(); + let mut conf = conf.as_ref().clone(); let concurrency_token = concurrency_token.clone(); let db = db.clone(); @@ -964,14 +966,14 @@ impl ComputeNode { match &db { DB::SystemDB => {} DB::UserDB(db) => { - url.set_path(db.name.as_str()); + conf.dbname(db.name.as_str()); } } - let url = Arc::new(url); + let conf = Arc::new(conf); let fut = Self::apply_spec_sql_db( spec.clone(), - url, + conf, ctx.clone(), jwks_roles.clone(), concurrency_token.clone(), @@ -1017,7 +1019,7 @@ impl ComputeNode { /// semaphore. The caller has to make sure the semaphore isn't exhausted. async fn apply_spec_sql_db( spec: Arc, - url: Arc, + conf: Arc, ctx: Arc>, jwks_roles: Arc>, concurrency_token: Arc, @@ -1046,7 +1048,7 @@ impl ComputeNode { // that database. || async { if client_conn.is_none() { - let db_client = Self::get_maintenance_client(&url).await?; + let db_client = Self::get_maintenance_client(&conf).await?; client_conn.replace(db_client); } let client = client_conn.as_ref().unwrap(); @@ -1061,34 +1063,16 @@ impl ComputeNode { Ok::<(), anyhow::Error>(()) } - /// Do initial configuration of the already started Postgres. - #[instrument(skip_all)] - pub fn apply_config(&self, compute_state: &ComputeState) -> Result<()> { - // If connection fails, - // it may be the old node with `zenith_admin` superuser. - // - // In this case we need to connect with old `zenith_admin` name - // and create new user. We cannot simply rename connected user, - // but we can create a new one and grant it all privileges. - let mut url = self.connstr.clone(); - url.query_pairs_mut() - .append_pair("application_name", "apply_config"); - - let url = Arc::new(url); - let spec = Arc::new( - compute_state - .pspec - .as_ref() - .expect("spec must be set") - .spec - .clone(), - ); - - // Choose how many concurrent connections to use for applying the spec changes. - // If the cluster is not currently Running we don't have to deal with user connections, + /// Choose how many concurrent connections to use for applying the spec changes. + pub fn max_service_connections( + &self, + compute_state: &ComputeState, + spec: &ComputeSpec, + ) -> usize { + // If the cluster is in Init state we don't have to deal with user connections, // and can thus use all `max_connections` connection slots. However, that's generally not // very efficient, so we generally still limit it to a smaller number. - let max_concurrent_connections = if compute_state.status != ComputeStatus::Running { + if compute_state.status == ComputeStatus::Init { // If the settings contain 'max_connections', use that as template if let Some(config) = spec.cluster.settings.find("max_connections") { config.parse::().ok() @@ -1144,10 +1128,29 @@ impl ComputeNode { .map(|val| if val > 1 { val - 1 } else { 1 }) .last() .unwrap_or(3) - }; + } + } + + /// Do initial configuration of the already started Postgres. + #[instrument(skip_all)] + pub fn apply_config(&self, compute_state: &ComputeState) -> Result<()> { + let mut conf = tokio_postgres::Config::from_str(self.connstr.as_str()).unwrap(); + conf.application_name("apply_config"); + + let conf = Arc::new(conf); + let spec = Arc::new( + compute_state + .pspec + .as_ref() + .expect("spec must be set") + .spec + .clone(), + ); + + let max_concurrent_connections = self.max_service_connections(compute_state, &spec); // Merge-apply spec & changes to PostgreSQL state. - self.apply_spec_sql(spec.clone(), url.clone(), max_concurrent_connections)?; + self.apply_spec_sql(spec.clone(), conf.clone(), max_concurrent_connections)?; if let Some(ref local_proxy) = &spec.clone().local_proxy_config { info!("configuring local_proxy"); @@ -1156,12 +1159,11 @@ impl ComputeNode { // Run migrations separately to not hold up cold starts thread::spawn(move || { - let mut connstr = url.as_ref().clone(); - connstr - .query_pairs_mut() - .append_pair("application_name", "migrations"); + let conf = conf.as_ref().clone(); + let mut conf = postgres::config::Config::from(conf); + conf.application_name("migrations"); - let mut client = Client::connect(connstr.as_str(), NoTls)?; + let mut client = conf.connect(NoTls)?; handle_migrations(&mut client).context("apply_config handle_migrations") }); @@ -1222,21 +1224,28 @@ impl ComputeNode { let pgdata_path = Path::new(&self.pgdata); let postgresql_conf_path = pgdata_path.join("postgresql.conf"); config::write_postgres_conf(&postgresql_conf_path, &spec, None)?; - // temporarily reset max_cluster_size in config + + // TODO(ololobus): We need a concurrency during reconfiguration as well, + // but DB is already running and used by user. We can easily get out of + // `max_connections` limit, and the current code won't handle that. + // let compute_state = self.state.lock().unwrap().clone(); + // let max_concurrent_connections = self.max_service_connections(&compute_state, &spec); + let max_concurrent_connections = 1; + + // Temporarily reset max_cluster_size in config // to avoid the possibility of hitting the limit, while we are reconfiguring: - // creating new extensions, roles, etc... + // creating new extensions, roles, etc. config::with_compute_ctl_tmp_override(pgdata_path, "neon.max_cluster_size=-1", || { self.pg_reload_conf()?; if spec.mode == ComputeMode::Primary { - let mut url = self.connstr.clone(); - url.query_pairs_mut() - .append_pair("application_name", "apply_config"); - let url = Arc::new(url); + let mut conf = tokio_postgres::Config::from_str(self.connstr.as_str()).unwrap(); + conf.application_name("apply_config"); + let conf = Arc::new(conf); let spec = Arc::new(spec.clone()); - self.apply_spec_sql(spec, url, 1)?; + self.apply_spec_sql(spec, conf, max_concurrent_connections)?; } Ok(()) @@ -1362,7 +1371,17 @@ impl ComputeNode { let connstr = self.connstr.clone(); thread::spawn(move || { - get_installed_extensions_sync(connstr).context("get_installed_extensions") + let res = get_installed_extensions(&connstr); + match res { + Ok(extensions) => { + info!( + "[NEON_EXT_STAT] {}", + serde_json::to_string(&extensions) + .expect("failed to serialize extensions list") + ); + } + Err(err) => error!("could not get installed extensions: {err:?}"), + } }); } diff --git a/compute_tools/src/http/api.rs b/compute_tools/src/http/api.rs index 8a047634df..a6c6cff20a 100644 --- a/compute_tools/src/http/api.rs +++ b/compute_tools/src/http/api.rs @@ -296,7 +296,12 @@ async fn routes(req: Request, compute: &Arc) -> Response render_json(Body::from(serde_json::to_string(&res).unwrap())), Err(e) => render_json_error( diff --git a/compute_tools/src/installed_extensions.rs b/compute_tools/src/installed_extensions.rs index 79d8b2ca04..f473c29a55 100644 --- a/compute_tools/src/installed_extensions.rs +++ b/compute_tools/src/installed_extensions.rs @@ -2,17 +2,16 @@ use compute_api::responses::{InstalledExtension, InstalledExtensions}; use metrics::proto::MetricFamily; use std::collections::HashMap; use std::collections::HashSet; -use tracing::info; -use url::Url; use anyhow::Result; use postgres::{Client, NoTls}; -use tokio::task; use metrics::core::Collector; use metrics::{register_uint_gauge_vec, UIntGaugeVec}; use once_cell::sync::Lazy; +use crate::pg_helpers::postgres_conf_for_db; + /// We don't reuse get_existing_dbs() just for code clarity /// and to make database listing query here more explicit. /// @@ -42,75 +41,51 @@ fn list_dbs(client: &mut Client) -> Result> { /// /// Same extension can be installed in multiple databases with different versions, /// we only keep the highest and lowest version across all databases. -pub async fn get_installed_extensions(connstr: Url) -> Result { - let mut connstr = connstr.clone(); +pub fn get_installed_extensions(connstr: &url::Url) -> Result { + let mut client = Client::connect(connstr.as_str(), NoTls)?; + let databases: Vec = list_dbs(&mut client)?; - task::spawn_blocking(move || { - let mut client = Client::connect(connstr.as_str(), NoTls)?; - let databases: Vec = list_dbs(&mut client)?; + let mut extensions_map: HashMap = HashMap::new(); + for db in databases.iter() { + let config = postgres_conf_for_db(connstr, db)?; + let mut db_client = config.connect(NoTls)?; + let extensions: Vec<(String, String)> = db_client + .query( + "SELECT extname, extversion FROM pg_catalog.pg_extension;", + &[], + )? + .iter() + .map(|row| (row.get("extname"), row.get("extversion"))) + .collect(); - let mut extensions_map: HashMap = HashMap::new(); - for db in databases.iter() { - connstr.set_path(db); - let mut db_client = Client::connect(connstr.as_str(), NoTls)?; - let extensions: Vec<(String, String)> = db_client - .query( - "SELECT extname, extversion FROM pg_catalog.pg_extension;", - &[], - )? - .iter() - .map(|row| (row.get("extname"), row.get("extversion"))) - .collect(); + for (extname, v) in extensions.iter() { + let version = v.to_string(); - for (extname, v) in extensions.iter() { - let version = v.to_string(); + // increment the number of databases where the version of extension is installed + INSTALLED_EXTENSIONS + .with_label_values(&[extname, &version]) + .inc(); - // increment the number of databases where the version of extension is installed - INSTALLED_EXTENSIONS - .with_label_values(&[extname, &version]) - .inc(); - - extensions_map - .entry(extname.to_string()) - .and_modify(|e| { - e.versions.insert(version.clone()); - // count the number of databases where the extension is installed - e.n_databases += 1; - }) - .or_insert(InstalledExtension { - extname: extname.to_string(), - versions: HashSet::from([version.clone()]), - n_databases: 1, - }); - } + extensions_map + .entry(extname.to_string()) + .and_modify(|e| { + e.versions.insert(version.clone()); + // count the number of databases where the extension is installed + e.n_databases += 1; + }) + .or_insert(InstalledExtension { + extname: extname.to_string(), + versions: HashSet::from([version.clone()]), + n_databases: 1, + }); } + } - let res = InstalledExtensions { - extensions: extensions_map.values().cloned().collect(), - }; + let res = InstalledExtensions { + extensions: extensions_map.values().cloned().collect(), + }; - Ok(res) - }) - .await? -} - -// Gather info about installed extensions -pub fn get_installed_extensions_sync(connstr: Url) -> Result<()> { - let rt = tokio::runtime::Builder::new_current_thread() - .enable_all() - .build() - .expect("failed to create runtime"); - let result = rt - .block_on(crate::installed_extensions::get_installed_extensions( - connstr, - )) - .expect("failed to get installed extensions"); - - info!( - "[NEON_EXT_STAT] {}", - serde_json::to_string(&result).expect("failed to serialize extensions list") - ); - Ok(()) + Ok(res) } static INSTALLED_EXTENSIONS: Lazy = Lazy::new(|| { diff --git a/compute_tools/src/pg_helpers.rs b/compute_tools/src/pg_helpers.rs index 4a1e5ee0e8..e03b410699 100644 --- a/compute_tools/src/pg_helpers.rs +++ b/compute_tools/src/pg_helpers.rs @@ -6,6 +6,7 @@ use std::io::{BufRead, BufReader}; use std::os::unix::fs::PermissionsExt; use std::path::Path; use std::process::Child; +use std::str::FromStr; use std::thread::JoinHandle; use std::time::{Duration, Instant}; @@ -13,8 +14,10 @@ use anyhow::{bail, Result}; use futures::StreamExt; use ini::Ini; use notify::{RecursiveMode, Watcher}; +use postgres::config::Config; use tokio::io::AsyncBufReadExt; use tokio::time::timeout; +use tokio_postgres; use tokio_postgres::NoTls; use tracing::{debug, error, info, instrument}; @@ -542,3 +545,11 @@ async fn handle_postgres_logs_async(stderr: tokio::process::ChildStderr) -> Resu Ok(()) } + +/// `Postgres::config::Config` handles database names with whitespaces +/// and special characters properly. +pub fn postgres_conf_for_db(connstr: &url::Url, dbname: &str) -> Result { + let mut conf = Config::from_str(connstr.as_str())?; + conf.dbname(dbname); + Ok(conf) +} diff --git a/test_runner/fixtures/endpoint/http.py b/test_runner/fixtures/endpoint/http.py index db3723b7cc..1cd9158c68 100644 --- a/test_runner/fixtures/endpoint/http.py +++ b/test_runner/fixtures/endpoint/http.py @@ -1,5 +1,7 @@ from __future__ import annotations +import urllib.parse + import requests from requests.adapters import HTTPAdapter @@ -20,7 +22,9 @@ class EndpointHttpClient(requests.Session): return res.json() def database_schema(self, database: str): - res = self.get(f"http://localhost:{self.port}/database_schema?database={database}") + res = self.get( + f"http://localhost:{self.port}/database_schema?database={urllib.parse.quote(database, safe='')}" + ) res.raise_for_status() return res.text diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index a45a311dc2..1f4d2aa5ec 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -3934,6 +3934,35 @@ class Endpoint(PgProtocol, LogUtils): log.info(json.dumps(dict(data_dict, **kwargs))) json.dump(dict(data_dict, **kwargs), file, indent=4) + def respec_deep(self, **kwargs: Any) -> None: + """ + Update the endpoint.json file taking into account nested keys. + It does one level deep update. Should enough for most cases. + Distinct method from respec() to do not break existing functionality. + NOTE: This method also updates the spec.json file, not endpoint.json. + We need it because neon_local also writes to spec.json, so intended + use-case is i) start endpoint with some config, ii) respec_deep(), + iii) call reconfigure() to apply the changes. + """ + config_path = os.path.join(self.endpoint_path(), "spec.json") + with open(config_path) as f: + data_dict: dict[str, Any] = json.load(f) + + log.info("Current compute spec: %s", json.dumps(data_dict, indent=4)) + + for key, value in kwargs.items(): + if isinstance(value, dict): + if key not in data_dict: + data_dict[key] = value + else: + data_dict[key] = {**data_dict[key], **value} + else: + data_dict[key] = value + + with open(config_path, "w") as file: + log.info("Updating compute spec to: %s", json.dumps(data_dict, indent=4)) + json.dump(data_dict, file, indent=4) + # Please note: Migrations only run if pg_skip_catalog_updates is false def wait_for_migrations(self, num_migrations: int = 11): with self.cursor() as cur: diff --git a/test_runner/regress/test_compute_catalog.py b/test_runner/regress/test_compute_catalog.py index d43c71ceac..b3719a45ed 100644 --- a/test_runner/regress/test_compute_catalog.py +++ b/test_runner/regress/test_compute_catalog.py @@ -3,13 +3,60 @@ from __future__ import annotations import requests from fixtures.neon_fixtures import NeonEnv +TEST_DB_NAMES = [ + { + "name": "neondb", + "owner": "cloud_admin", + }, + { + "name": "db with spaces", + "owner": "cloud_admin", + }, + { + "name": "db with%20spaces ", + "owner": "cloud_admin", + }, + { + "name": "db with whitespaces ", + "owner": "cloud_admin", + }, + { + "name": "injective db with spaces'; SELECT pg_sleep(10);", + "owner": "cloud_admin", + }, + { + "name": "db with #pound-sign and &ersands=true", + "owner": "cloud_admin", + }, + { + "name": "db with emoji 🌍", + "owner": "cloud_admin", + }, +] + def test_compute_catalog(neon_simple_env: NeonEnv): + """ + Create a bunch of databases with tricky names and test that we can list them + and dump via API. + """ env = neon_simple_env - endpoint = env.endpoints.create_start("main", config_lines=["log_min_messages=debug1"]) - client = endpoint.http_client() + endpoint = env.endpoints.create_start("main") + # Update the spec.json file to include new databases + # and reconfigure the endpoint to create some test databases. + endpoint.respec_deep( + **{ + "skip_pg_catalog_updates": False, + "cluster": { + "databases": TEST_DB_NAMES, + }, + } + ) + endpoint.reconfigure() + + client = endpoint.http_client() objects = client.dbs_and_roles() # Assert that 'cloud_admin' role exists in the 'roles' list @@ -22,9 +69,24 @@ def test_compute_catalog(neon_simple_env: NeonEnv): db["name"] == "postgres" for db in objects["databases"] ), "The 'postgres' database is missing" - ddl = client.database_schema(database="postgres") + # Check other databases + for test_db in TEST_DB_NAMES: + db = next((db for db in objects["databases"] if db["name"] == test_db["name"]), None) + assert db is not None, f"The '{test_db['name']}' database is missing" + assert ( + db["owner"] == test_db["owner"] + ), f"The '{test_db['name']}' database has incorrect owner" - assert "-- PostgreSQL database dump" in ddl + ddl = client.database_schema(database=test_db["name"]) + + # Check that it looks like a valid PostgreSQL dump + assert "-- PostgreSQL database dump" in ddl + + # Check that it doesn't contain health_check and migration traces. + # They are only created in system `postgres` database, so by checking + # that we ensure that we dump right databases. + assert "health_check" not in ddl, f"The '{test_db['name']}' database contains health_check" + assert "migration" not in ddl, f"The '{test_db['name']}' database contains migrations data" try: client.database_schema(database="nonexistentdb") @@ -33,3 +95,44 @@ def test_compute_catalog(neon_simple_env: NeonEnv): assert ( e.response.status_code == 404 ), f"Expected 404 status code, but got {e.response.status_code}" + + +def test_compute_create_databases(neon_simple_env: NeonEnv): + """ + Test that compute_ctl can create and work with databases with special + characters (whitespaces, %, tabs, etc.) in the name. + """ + env = neon_simple_env + + # Create and start endpoint so that neon_local put all the generated + # stuff into the spec.json file. + endpoint = env.endpoints.create_start("main") + + # Update the spec.json file to include new databases + # and reconfigure the endpoint to apply the changes. + endpoint.respec_deep( + **{ + "skip_pg_catalog_updates": False, + "cluster": { + "databases": TEST_DB_NAMES, + }, + } + ) + endpoint.reconfigure() + + for db in TEST_DB_NAMES: + # Check that database has a correct name in the system catalog + with endpoint.cursor() as cursor: + cursor.execute("SELECT datname FROM pg_database WHERE datname = %s", (db["name"],)) + catalog_db = cursor.fetchone() + assert catalog_db is not None + assert len(catalog_db) == 1 + assert catalog_db[0] == db["name"] + + # Check that we can connect to this database without any issues + with endpoint.cursor(dbname=db["name"]) as cursor: + cursor.execute("SELECT * FROM current_database()") + curr_db = cursor.fetchone() + assert curr_db is not None + assert len(curr_db) == 1 + assert curr_db[0] == db["name"]