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

@@ -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 {