tests: add infra and test for storcon leadership transfer (#8587)

## Problem
https://github.com/neondatabase/neon/pull/8588 implemented the mechanism
for storage controller
leadership transfers. However, there's no tests that exercise the
behaviour.

## Summary of changes
1. Teach `neon_local` how to handle multiple storage controller
instances. Each storage controller
instance gets its own subdirectory (`storage_controller_1, ...`).
`storage_controller start|stop` subcommands
have also been extended to optionally accept an instance id.
2. Add a storage controller proxy test fixture. It's a basic HTTP server
that forwards requests from pageserver
and test env to the currently configured storage controller.
3. Add a test which exercises storage controller leadership transfer.
4. Finally fix a couple bugs that the test surfaced
This commit is contained in:
Vlad Lazar
2024-08-16 13:05:04 +01:00
committed by GitHub
parent 7fdc3ea162
commit 3f91ea28d9
12 changed files with 841 additions and 251 deletions

View File

@@ -379,7 +379,7 @@ where
}
}
fn process_has_stopped(pid: Pid) -> anyhow::Result<bool> {
pub(crate) fn process_has_stopped(pid: Pid) -> anyhow::Result<bool> {
match kill(pid, None) {
// Process exists, keep waiting
Ok(_) => Ok(false),

View File

@@ -15,7 +15,9 @@ use control_plane::local_env::{
};
use control_plane::pageserver::PageServerNode;
use control_plane::safekeeper::SafekeeperNode;
use control_plane::storage_controller::StorageController;
use control_plane::storage_controller::{
NeonStorageControllerStartArgs, NeonStorageControllerStopArgs, StorageController,
};
use control_plane::{broker, local_env};
use pageserver_api::config::{
DEFAULT_HTTP_LISTEN_PORT as DEFAULT_PAGESERVER_HTTP_PORT,
@@ -1052,6 +1054,36 @@ fn get_start_timeout(args: &ArgMatches) -> &Duration {
humantime_duration.as_ref()
}
fn storage_controller_start_args(args: &ArgMatches) -> NeonStorageControllerStartArgs {
let maybe_instance_id = args.get_one::<u8>("instance-id");
let base_port = args.get_one::<u16>("base-port");
if maybe_instance_id.is_some() && base_port.is_none() {
panic!("storage-controller start specificied instance-id but did not provide base-port");
}
let start_timeout = args
.get_one::<humantime::Duration>("start-timeout")
.expect("invalid value for start-timeout");
NeonStorageControllerStartArgs {
instance_id: maybe_instance_id.copied().unwrap_or(1),
base_port: base_port.copied(),
start_timeout: *start_timeout,
}
}
fn storage_controller_stop_args(args: &ArgMatches) -> NeonStorageControllerStopArgs {
let maybe_instance_id = args.get_one::<u8>("instance-id");
let immediate = args.get_one::<String>("stop-mode").map(|s| s.as_str()) == Some("immediate");
NeonStorageControllerStopArgs {
instance_id: maybe_instance_id.copied().unwrap_or(1),
immediate,
}
}
async fn handle_pageserver(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> Result<()> {
match sub_match.subcommand() {
Some(("start", subcommand_args)) => {
@@ -1113,19 +1145,14 @@ async fn handle_storage_controller(
let svc = StorageController::from_env(env);
match sub_match.subcommand() {
Some(("start", start_match)) => {
if let Err(e) = svc.start(get_start_timeout(start_match)).await {
if let Err(e) = svc.start(storage_controller_start_args(start_match)).await {
eprintln!("start failed: {e}");
exit(1);
}
}
Some(("stop", stop_match)) => {
let immediate = stop_match
.get_one::<String>("stop-mode")
.map(|s| s.as_str())
== Some("immediate");
if let Err(e) = svc.stop(immediate).await {
if let Err(e) = svc.stop(storage_controller_stop_args(stop_match)).await {
eprintln!("stop failed: {}", e);
exit(1);
}
@@ -1228,7 +1255,12 @@ async fn handle_start_all(
// Only start the storage controller if the pageserver is configured to need it
if env.control_plane_api.is_some() {
let storage_controller = StorageController::from_env(env);
if let Err(e) = storage_controller.start(retry_timeout).await {
if let Err(e) = storage_controller
.start(NeonStorageControllerStartArgs::with_default_instance_id(
(*retry_timeout).into(),
))
.await
{
eprintln!("storage_controller start failed: {:#}", e);
try_stop_all(env, true).await;
exit(1);
@@ -1358,10 +1390,21 @@ async fn try_stop_all(env: &local_env::LocalEnv, immediate: bool) {
eprintln!("neon broker stop failed: {e:#}");
}
if env.control_plane_api.is_some() {
// Stop all storage controller instances. In the most common case there's only one,
// but iterate though the base data directory in order to discover the instances.
let storcon_instances = env
.storage_controller_instances()
.await
.expect("Must inspect data dir");
for (instance_id, _instance_dir_path) in storcon_instances {
let storage_controller = StorageController::from_env(env);
if let Err(e) = storage_controller.stop(immediate).await {
eprintln!("storage controller stop failed: {e:#}");
let stop_args = NeonStorageControllerStopArgs {
instance_id,
immediate,
};
if let Err(e) = storage_controller.stop(stop_args).await {
eprintln!("Storage controller instance {instance_id} stop failed: {e:#}");
}
}
}
@@ -1501,6 +1544,18 @@ fn cli() -> Command {
.action(ArgAction::SetTrue)
.required(false);
let instance_id = Arg::new("instance-id")
.long("instance-id")
.help("Identifier used to distinguish storage controller instances (default 1)")
.value_parser(value_parser!(u8))
.required(false);
let base_port = Arg::new("base-port")
.long("base-port")
.help("Base port for the storage controller instance idenfified by instance-id (defaults to pagserver cplane api)")
.value_parser(value_parser!(u16))
.required(false);
Command::new("Neon CLI")
.arg_required_else_help(true)
.version(GIT_VERSION)
@@ -1609,9 +1664,12 @@ fn cli() -> Command {
.arg_required_else_help(true)
.about("Manage storage_controller")
.subcommand(Command::new("start").about("Start storage controller")
.arg(timeout_arg.clone()))
.arg(timeout_arg.clone())
.arg(instance_id.clone())
.arg(base_port))
.subcommand(Command::new("stop").about("Stop storage controller")
.arg(stop_mode_arg.clone()))
.arg(stop_mode_arg.clone())
.arg(instance_id))
)
.subcommand(
Command::new("safekeeper")

View File

@@ -156,6 +156,11 @@ pub struct NeonStorageControllerConf {
#[serde(with = "humantime_serde")]
pub max_warming_up: Duration,
pub start_as_candidate: bool,
/// Database url used when running multiple storage controller instances
pub database_url: Option<SocketAddr>,
/// Threshold for auto-splitting a tenant into shards
pub split_threshold: Option<u64>,
@@ -174,6 +179,8 @@ impl Default for NeonStorageControllerConf {
Self {
max_offline: Self::DEFAULT_MAX_OFFLINE_INTERVAL,
max_warming_up: Self::DEFAULT_MAX_WARMING_UP_INTERVAL,
start_as_candidate: false,
database_url: None,
split_threshold: None,
max_secondary_lag_bytes: None,
}
@@ -392,6 +399,36 @@ impl LocalEnv {
}
}
/// Inspect the base data directory and extract the instance id and instance directory path
/// for all storage controller instances
pub async fn storage_controller_instances(&self) -> std::io::Result<Vec<(u8, PathBuf)>> {
let mut instances = Vec::default();
let dir = std::fs::read_dir(self.base_data_dir.clone())?;
for dentry in dir {
let dentry = dentry?;
let is_dir = dentry.metadata()?.is_dir();
let filename = dentry.file_name().into_string().unwrap();
let parsed_instance_id = match filename.strip_prefix("storage_controller_") {
Some(suffix) => suffix.parse::<u8>().ok(),
None => None,
};
let is_instance_dir = is_dir && parsed_instance_id.is_some();
if !is_instance_dir {
continue;
}
instances.push((
parsed_instance_id.expect("Checked previously"),
dentry.path(),
));
}
Ok(instances)
}
pub fn register_branch_mapping(
&mut self,
branch_name: String,

View File

@@ -3,6 +3,8 @@ use crate::{
local_env::{LocalEnv, NeonStorageControllerConf},
};
use camino::{Utf8Path, Utf8PathBuf};
use hyper::Uri;
use nix::unistd::Pid;
use pageserver_api::{
controller_api::{
NodeConfigureRequest, NodeDescribeResponse, NodeRegisterRequest, TenantCreateRequest,
@@ -18,7 +20,7 @@ use pageserver_client::mgmt_api::ResponseErrorMessageExt;
use postgres_backend::AuthType;
use reqwest::Method;
use serde::{de::DeserializeOwned, Deserialize, Serialize};
use std::{fs, str::FromStr, time::Duration};
use std::{fs, net::SocketAddr, path::PathBuf, str::FromStr, sync::OnceLock};
use tokio::process::Command;
use tracing::instrument;
use url::Url;
@@ -29,12 +31,14 @@ use utils::{
pub struct StorageController {
env: LocalEnv,
listen: String,
private_key: Option<Vec<u8>>,
public_key: Option<String>,
postgres_port: u16,
client: reqwest::Client,
config: NeonStorageControllerConf,
// The listen addresses is learned when starting the storage controller,
// hence the use of OnceLock to init it at the right time.
listen: OnceLock<SocketAddr>,
}
const COMMAND: &str = "storage_controller";
@@ -43,6 +47,36 @@ const STORAGE_CONTROLLER_POSTGRES_VERSION: u32 = 16;
const DB_NAME: &str = "storage_controller";
pub struct NeonStorageControllerStartArgs {
pub instance_id: u8,
pub base_port: Option<u16>,
pub start_timeout: humantime::Duration,
}
impl NeonStorageControllerStartArgs {
pub fn with_default_instance_id(start_timeout: humantime::Duration) -> Self {
Self {
instance_id: 1,
base_port: None,
start_timeout,
}
}
}
pub struct NeonStorageControllerStopArgs {
pub instance_id: u8,
pub immediate: bool,
}
impl NeonStorageControllerStopArgs {
pub fn with_default_instance_id(immediate: bool) -> Self {
Self {
instance_id: 1,
immediate,
}
}
}
#[derive(Serialize, Deserialize)]
pub struct AttachHookRequest {
pub tenant_shard_id: TenantShardId,
@@ -67,23 +101,6 @@ pub struct InspectResponse {
impl StorageController {
pub fn from_env(env: &LocalEnv) -> Self {
// Makes no sense to construct this if pageservers aren't going to use it: assume
// pageservers have control plane API set
let listen_url = env.control_plane_api.clone().unwrap();
let listen = format!(
"{}:{}",
listen_url.host_str().unwrap(),
listen_url.port().unwrap()
);
// Convention: NeonEnv in python tests reserves the next port after the control_plane_api
// port, for use by our captive postgres.
let postgres_port = listen_url
.port()
.expect("Control plane API setting should always have a port")
+ 1;
// Assume all pageservers have symmetric auth configuration: this service
// expects to use one JWT token to talk to all of them.
let ps_conf = env
@@ -126,20 +143,28 @@ impl StorageController {
Self {
env: env.clone(),
listen,
private_key,
public_key,
postgres_port,
client: reqwest::ClientBuilder::new()
.build()
.expect("Failed to construct http client"),
config: env.storage_controller.clone(),
listen: OnceLock::default(),
}
}
fn pid_file(&self) -> Utf8PathBuf {
Utf8PathBuf::from_path_buf(self.env.base_data_dir.join("storage_controller.pid"))
.expect("non-Unicode path")
fn storage_controller_instance_dir(&self, instance_id: u8) -> PathBuf {
self.env
.base_data_dir
.join(format!("storage_controller_{}", instance_id))
}
fn pid_file(&self, instance_id: u8) -> Utf8PathBuf {
Utf8PathBuf::from_path_buf(
self.storage_controller_instance_dir(instance_id)
.join("storage_controller.pid"),
)
.expect("non-Unicode path")
}
/// PIDFile for the postgres instance used to store storage controller state
@@ -184,9 +209,9 @@ impl StorageController {
}
/// Readiness check for our postgres process
async fn pg_isready(&self, pg_bin_dir: &Utf8Path) -> anyhow::Result<bool> {
async fn pg_isready(&self, pg_bin_dir: &Utf8Path, postgres_port: u16) -> anyhow::Result<bool> {
let bin_path = pg_bin_dir.join("pg_isready");
let args = ["-h", "localhost", "-p", &format!("{}", self.postgres_port)];
let args = ["-h", "localhost", "-p", &format!("{}", postgres_port)];
let exitcode = Command::new(bin_path).args(args).spawn()?.wait().await?;
Ok(exitcode.success())
@@ -199,8 +224,8 @@ impl StorageController {
/// who just want to run `cargo neon_local` without knowing about diesel.
///
/// Returns the database url
pub async fn setup_database(&self) -> anyhow::Result<String> {
let database_url = format!("postgresql://localhost:{}/{DB_NAME}", self.postgres_port);
pub async fn setup_database(&self, postgres_port: u16) -> anyhow::Result<String> {
let database_url = format!("postgresql://localhost:{}/{DB_NAME}", postgres_port);
let pg_bin_dir = self.get_pg_bin_dir().await?;
let createdb_path = pg_bin_dir.join("createdb");
@@ -209,7 +234,7 @@ impl StorageController {
"-h",
"localhost",
"-p",
&format!("{}", self.postgres_port),
&format!("{}", postgres_port),
DB_NAME,
])
.output()
@@ -230,13 +255,14 @@ impl StorageController {
pub async fn connect_to_database(
&self,
postgres_port: u16,
) -> anyhow::Result<(
tokio_postgres::Client,
tokio_postgres::Connection<tokio_postgres::Socket, tokio_postgres::tls::NoTlsStream>,
)> {
tokio_postgres::Config::new()
.host("localhost")
.port(self.postgres_port)
.port(postgres_port)
// The user is the ambient operating system user name.
// That is an impurity which we want to fix in => TODO https://github.com/neondatabase/neon/issues/8400
//
@@ -252,72 +278,115 @@ impl StorageController {
.map_err(anyhow::Error::new)
}
pub async fn start(&self, retry_timeout: &Duration) -> anyhow::Result<()> {
// Start a vanilla Postgres process used by the storage controller for persistence.
let pg_data_path = Utf8PathBuf::from_path_buf(self.env.base_data_dir.clone())
.unwrap()
.join("storage_controller_db");
let pg_bin_dir = self.get_pg_bin_dir().await?;
let pg_lib_dir = self.get_pg_lib_dir().await?;
let pg_log_path = pg_data_path.join("postgres.log");
pub async fn start(&self, start_args: NeonStorageControllerStartArgs) -> anyhow::Result<()> {
let instance_dir = self.storage_controller_instance_dir(start_args.instance_id);
if let Err(err) = tokio::fs::create_dir(&instance_dir).await {
if err.kind() != std::io::ErrorKind::AlreadyExists {
panic!("Failed to create instance dir {instance_dir:?}");
}
}
if !tokio::fs::try_exists(&pg_data_path).await? {
// Initialize empty database
let initdb_path = pg_bin_dir.join("initdb");
let mut child = Command::new(&initdb_path)
.envs(vec![
("LD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
("DYLD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
])
.args(["-D", pg_data_path.as_ref()])
.spawn()
.expect("Failed to spawn initdb");
let status = child.wait().await?;
if !status.success() {
anyhow::bail!("initdb failed with status {status}");
let (listen, postgres_port) = {
if let Some(base_port) = start_args.base_port {
(
format!("127.0.0.1:{base_port}"),
self.config
.database_url
.expect("--base-port requires NeonStorageControllerConf::database_url")
.port(),
)
} else {
let listen_url = self.env.control_plane_api.clone().unwrap();
let listen = format!(
"{}:{}",
listen_url.host_str().unwrap(),
listen_url.port().unwrap()
);
(listen, listen_url.port().unwrap() + 1)
}
};
// Write a minimal config file:
// - Specify the port, since this is chosen dynamically
// - Switch off fsync, since we're running on lightweight test environments and when e.g. scale testing
// the storage controller we don't want a slow local disk to interfere with that.
//
// NB: it's important that we rewrite this file on each start command so we propagate changes
// from `LocalEnv`'s config file (`.neon/config`).
tokio::fs::write(
&pg_data_path.join("postgresql.conf"),
format!("port = {}\nfsync=off\n", self.postgres_port),
)
.await?;
let socket_addr = listen
.parse()
.expect("listen address is a valid socket address");
self.listen
.set(socket_addr)
.expect("StorageController::listen is only set here");
println!("Starting storage controller database...");
let db_start_args = [
"-w",
"-D",
pg_data_path.as_ref(),
"-l",
pg_log_path.as_ref(),
"start",
];
// Do we remove the pid file on stop?
let pg_started = self.is_postgres_running().await?;
let pg_lib_dir = self.get_pg_lib_dir().await?;
background_process::start_process(
"storage_controller_db",
&self.env.base_data_dir,
pg_bin_dir.join("pg_ctl").as_std_path(),
db_start_args,
vec![
("LD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
("DYLD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
],
background_process::InitialPidFile::Create(self.postgres_pid_file()),
retry_timeout,
|| self.pg_isready(&pg_bin_dir),
)
.await?;
if !pg_started {
// Start a vanilla Postgres process used by the storage controller for persistence.
let pg_data_path = Utf8PathBuf::from_path_buf(self.env.base_data_dir.clone())
.unwrap()
.join("storage_controller_db");
let pg_bin_dir = self.get_pg_bin_dir().await?;
let pg_log_path = pg_data_path.join("postgres.log");
// Run migrations on every startup, in case something changed.
let database_url = self.setup_database().await?;
if !tokio::fs::try_exists(&pg_data_path).await? {
// Initialize empty database
let initdb_path = pg_bin_dir.join("initdb");
let mut child = Command::new(&initdb_path)
.envs(vec![
("LD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
("DYLD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
])
.args(["-D", pg_data_path.as_ref()])
.spawn()
.expect("Failed to spawn initdb");
let status = child.wait().await?;
if !status.success() {
anyhow::bail!("initdb failed with status {status}");
}
};
// Write a minimal config file:
// - Specify the port, since this is chosen dynamically
// - Switch off fsync, since we're running on lightweight test environments and when e.g. scale testing
// the storage controller we don't want a slow local disk to interfere with that.
//
// NB: it's important that we rewrite this file on each start command so we propagate changes
// from `LocalEnv`'s config file (`.neon/config`).
tokio::fs::write(
&pg_data_path.join("postgresql.conf"),
format!("port = {}\nfsync=off\n", postgres_port),
)
.await?;
println!("Starting storage controller database...");
let db_start_args = [
"-w",
"-D",
pg_data_path.as_ref(),
"-l",
pg_log_path.as_ref(),
"start",
];
background_process::start_process(
"storage_controller_db",
&self.env.base_data_dir,
pg_bin_dir.join("pg_ctl").as_std_path(),
db_start_args,
vec![
("LD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
("DYLD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
],
background_process::InitialPidFile::Create(self.postgres_pid_file()),
&start_args.start_timeout,
|| self.pg_isready(&pg_bin_dir, postgres_port),
)
.await?;
// Run migrations on every startup, in case something changed.
self.setup_database(postgres_port).await?;
}
let database_url = format!("postgresql://localhost:{}/{DB_NAME}", postgres_port);
// We support running a startup SQL script to fiddle with the database before we launch storcon.
// This is used by the test suite.
@@ -339,7 +408,7 @@ impl StorageController {
}
}
};
let (mut client, conn) = self.connect_to_database().await?;
let (mut client, conn) = self.connect_to_database(postgres_port).await?;
let conn = tokio::spawn(conn);
let tx = client.build_transaction();
let tx = tx.start().await?;
@@ -348,9 +417,20 @@ impl StorageController {
drop(client);
conn.await??;
let listen = self
.listen
.get()
.expect("cell is set earlier in this function");
let address_for_peers = Uri::builder()
.scheme("http")
.authority(format!("{}:{}", listen.ip(), listen.port()))
.path_and_query("")
.build()
.unwrap();
let mut args = vec![
"-l",
&self.listen,
&listen.to_string(),
"--dev",
"--database-url",
&database_url,
@@ -358,10 +438,17 @@ impl StorageController {
&humantime::Duration::from(self.config.max_offline).to_string(),
"--max-warming-up-interval",
&humantime::Duration::from(self.config.max_warming_up).to_string(),
"--address-for-peers",
&address_for_peers.to_string(),
]
.into_iter()
.map(|s| s.to_string())
.collect::<Vec<_>>();
if self.config.start_as_candidate {
args.push("--start-as-candidate".to_string());
}
if let Some(private_key) = &self.private_key {
let claims = Claims::new(None, Scope::PageServerApi);
let jwt_token =
@@ -394,15 +481,15 @@ impl StorageController {
background_process::start_process(
COMMAND,
&self.env.base_data_dir,
&instance_dir,
&self.env.storage_controller_bin(),
args,
vec![
("LD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
("DYLD_LIBRARY_PATH".to_owned(), pg_lib_dir.to_string()),
],
background_process::InitialPidFile::Create(self.pid_file()),
retry_timeout,
background_process::InitialPidFile::Create(self.pid_file(start_args.instance_id)),
&start_args.start_timeout,
|| async {
match self.ready().await {
Ok(_) => Ok(true),
@@ -415,8 +502,35 @@ impl StorageController {
Ok(())
}
pub async fn stop(&self, immediate: bool) -> anyhow::Result<()> {
background_process::stop_process(immediate, COMMAND, &self.pid_file())?;
pub async fn stop(&self, stop_args: NeonStorageControllerStopArgs) -> anyhow::Result<()> {
background_process::stop_process(
stop_args.immediate,
COMMAND,
&self.pid_file(stop_args.instance_id),
)?;
let storcon_instances = self.env.storage_controller_instances().await?;
for (instance_id, instanced_dir_path) in storcon_instances {
if instance_id == stop_args.instance_id {
continue;
}
let pid_file = instanced_dir_path.join("storage_controller.pid");
let pid = tokio::fs::read_to_string(&pid_file)
.await
.map_err(|err| {
anyhow::anyhow!("Failed to read storcon pid file at {pid_file:?}: {err}")
})?
.parse::<i32>()
.expect("pid is valid i32");
let other_proc_alive = !background_process::process_has_stopped(Pid::from_raw(pid))?;
if other_proc_alive {
// There is another storage controller instance running, so we return
// and leave the database running.
return Ok(());
}
}
let pg_data_path = self.env.base_data_dir.join("storage_controller_db");
let pg_bin_dir = self.get_pg_bin_dir().await?;
@@ -429,27 +543,51 @@ impl StorageController {
.wait()
.await?;
if !stop_status.success() {
let pg_status_args = ["-D", &pg_data_path.to_string_lossy(), "status"];
let status_exitcode = Command::new(pg_bin_dir.join("pg_ctl"))
.args(pg_status_args)
.spawn()?
.wait()
.await?;
// pg_ctl status returns this exit code if postgres is not running: in this case it is
// fine that stop failed. Otherwise it is an error that stop failed.
const PG_STATUS_NOT_RUNNING: i32 = 3;
if Some(PG_STATUS_NOT_RUNNING) == status_exitcode.code() {
println!("Storage controller database is already stopped");
return Ok(());
} else {
anyhow::bail!("Failed to stop storage controller database: {stop_status}")
match self.is_postgres_running().await {
Ok(false) => {
println!("Storage controller database is already stopped");
return Ok(());
}
Ok(true) => {
anyhow::bail!("Failed to stop storage controller database");
}
Err(err) => {
anyhow::bail!("Failed to stop storage controller database: {err}");
}
}
}
Ok(())
}
async fn is_postgres_running(&self) -> anyhow::Result<bool> {
let pg_data_path = self.env.base_data_dir.join("storage_controller_db");
let pg_bin_dir = self.get_pg_bin_dir().await?;
let pg_status_args = ["-D", &pg_data_path.to_string_lossy(), "status"];
let status_exitcode = Command::new(pg_bin_dir.join("pg_ctl"))
.args(pg_status_args)
.spawn()?
.wait()
.await?;
// pg_ctl status returns this exit code if postgres is not running: in this case it is
// fine that stop failed. Otherwise it is an error that stop failed.
const PG_STATUS_NOT_RUNNING: i32 = 3;
const PG_NO_DATA_DIR: i32 = 4;
const PG_STATUS_RUNNING: i32 = 0;
match status_exitcode.code() {
Some(PG_STATUS_NOT_RUNNING) => Ok(false),
Some(PG_NO_DATA_DIR) => Ok(false),
Some(PG_STATUS_RUNNING) => Ok(true),
Some(code) => Err(anyhow::anyhow!(
"pg_ctl status returned unexpected status code: {:?}",
code
)),
None => Err(anyhow::anyhow!("pg_ctl status returned no status code")),
}
}
fn get_claims_for_path(path: &str) -> anyhow::Result<Option<Claims>> {
let category = match path.find('/') {
Some(idx) => &path[..idx],
@@ -475,15 +613,31 @@ impl StorageController {
RQ: Serialize + Sized,
RS: DeserializeOwned + Sized,
{
// The configured URL has the /upcall path prefix for pageservers to use: we will strip that out
// for general purpose API access.
let listen_url = self.env.control_plane_api.clone().unwrap();
let url = Url::from_str(&format!(
"http://{}:{}/{path}",
listen_url.host_str().unwrap(),
listen_url.port().unwrap()
))
.unwrap();
// In the special case of the `storage_controller start` subcommand, we wish
// to use the API endpoint of the newly started storage controller in order
// to pass the readiness check. In this scenario [`Self::listen`] will be set
// (see [`Self::start`]).
//
// Otherwise, we infer the storage controller api endpoint from the configured
// control plane API.
let url = if let Some(socket_addr) = self.listen.get() {
Url::from_str(&format!(
"http://{}:{}/{path}",
socket_addr.ip().to_canonical(),
socket_addr.port()
))
.unwrap()
} else {
// The configured URL has the /upcall path prefix for pageservers to use: we will strip that out
// for general purpose API access.
let listen_url = self.env.control_plane_api.clone().unwrap();
Url::from_str(&format!(
"http://{}:{}/{path}",
listen_url.host_str().unwrap(),
listen_url.port().unwrap()
))
.unwrap()
};
let mut builder = self.client.request(method, url);
if let Some(body) = body {