Compare commits

...

45 Commits

Author SHA1 Message Date
John Spray
3f1e5b38d8 DNM demo 2023-12-22 18:11:37 +00:00
John Spray
56fddf1ea6 f 2023-12-22 16:21:55 +00:00
John Spray
2a13306c71 DNM demo scripts 2023-12-22 15:32:50 +00:00
John Spray
54386a537d f dirty hacks 2023-12-22 15:32:21 +00:00
John Spray
b4b5d20ddc DNM control_plane: pinning for John's workstation 2023-12-22 15:31:57 +00:00
John Spray
fa058efa58 pageserver: only upload initdb from shard 0 2023-12-22 14:04:08 +00:00
John Spray
cd1e438169 demo snapshot 2023-12-22 13:58:29 +00:00
John Spray
f67080410f f status command 2023-12-22 13:58:22 +00:00
John Spray
82ccd449d7 pageserver: fix active tenant lookup hitting secondaries with sharding 2023-12-22 13:58:00 +00:00
John Spray
d4c4ee6a14 f DNM demo 2023-12-22 13:40:22 +00:00
John Spray
758bb24445 DNM dirty hacks 2023-12-22 13:40:04 +00:00
John Spray
ed11fc0ec8 DNM: hack small buffer size into compute 2023-12-22 13:39:48 +00:00
John Spray
61044aa7f0 DNM demo script 2023-12-22 11:54:15 +00:00
John Spray
35de775ed5 pageserver: log details on shard routing error 2023-12-22 11:54:15 +00:00
John Spray
95505e5ac1 neon_local: add shard split command 2023-12-22 11:54:15 +00:00
John Spray
dda046bbcd pageserver: don't delete ancestor shard layers 2023-12-22 11:54:15 +00:00
John Spray
7b2019fdfe fixup! compute_tools: enable passing through stripe size 2023-12-22 11:54:15 +00:00
John Spray
b57848236f pgxn: fix stripe calculation 2023-12-22 11:54:15 +00:00
Konstantin Knizhnik
218a8a7461 Support sharding at compute side
refer #5508
2023-12-21 17:30:06 +00:00
John Spray
e745391c72 f neon_local reconfigure 2023-12-21 17:21:10 +00:00
John Spray
6e56f88b79 control_plane: improve debug of pageserver_connstr 2023-12-21 17:21:10 +00:00
John Spray
3ccf5abc9d control plane: improve handling of stripe size 2023-12-21 17:21:10 +00:00
John Spray
bc90272e47 tests: support initial stream size + migration 2023-12-21 17:21:10 +00:00
John Spray
aa1252d687 compute_tools: enable passing through stripe size 2023-12-21 17:21:10 +00:00
John Spray
727eef05b3 tests: make more fixtures/helpers shard-aware 2023-12-21 17:21:10 +00:00
John Spray
a5813e2516 tests: add test_sharding_smoke 2023-12-21 17:21:10 +00:00
John Spray
3c1d8e7239 tests: enable using timeout with CLIs 2023-12-21 17:21:10 +00:00
John Spray
3835a51429 control_plane: rebase fixes 2023-12-21 17:21:10 +00:00
John Spray
495c3d70f3 clippy 2023-12-21 17:19:01 +00:00
John Spray
d1af9d480e tests: enable s3 scrubber in pg_regress tests 2023-12-21 17:19:01 +00:00
John Spray
14b0acbda7 neon_local: improved timeline creation and 'branch' 2023-12-21 17:19:01 +00:00
John Spray
583375e6f6 tests: use sharding in test_pageserver_chaos 2023-12-21 17:19:01 +00:00
John Spray
1e542b3187 neon_local: always get endpoint pageserver from attachment service 2023-12-21 17:19:01 +00:00
John Spray
37db221a60 tests: enable sharding for tests in test_pg_regress.py 2023-12-21 17:19:01 +00:00
John Spray
9055985d72 tests: adapt helpers for sharding 2023-12-21 17:19:01 +00:00
John Spray
fc2f9fa3fe pageserver: implement shard splitting 2023-12-21 17:19:01 +00:00
John Spray
9cd72caabf neon_local: add tenant status command 2023-12-21 17:18:58 +00:00
John Spray
d1a0a0941a neon_local: add stripe size arg 2023-12-21 17:03:37 +00:00
John Spray
347bd012b3 neon_local: attachment service status, refactors 2023-12-21 17:03:37 +00:00
John Spray
58f64339f3 neon_local: implement Locate API for attachment service 2023-12-21 17:03:37 +00:00
John Spray
78e673fbb3 neon_local: use attachment service to locate pagservers for endpoints 2023-12-21 17:03:37 +00:00
John Spray
3d573be816 neon_local: use attachment service for tenant creation 2023-12-21 17:03:37 +00:00
John Spray
bdc4a7512b pageserver: refactor creation API (add ShardParams) 2023-12-21 17:03:37 +00:00
John Spray
24d0395f66 neon_local: update various TenantId uses to TenantShardId 2023-12-21 17:03:37 +00:00
John Spray
7bbfc160aa DNM: script for sharding demo 2023-12-21 17:03:14 +00:00
35 changed files with 2755 additions and 677 deletions

View File

@@ -69,6 +69,12 @@ pub fn write_postgres_conf(
)?;
}
writeln!(file, "shared_buffers=8MB")?;
if let Some(stripe_size) = &spec.shard_stripe_size {
writeln!(file, "neon.stripe_size={}", stripe_size)?;
}
match spec.mode {
ComputeMode::Primary => {}
ComputeMode::Static(lsn) => {

View File

@@ -1,8 +1,18 @@
use crate::{background_process, local_env::LocalEnv};
use anyhow::anyhow;
use camino::Utf8PathBuf;
use serde::{Deserialize, Serialize};
use hyper::{Method, StatusCode};
use pageserver_api::{
models::{
ShardParameters, TenantCreateRequest, TenantShardSplitRequest, TenantShardSplitResponse,
TimelineCreateRequest, TimelineInfo,
},
shard::TenantShardId,
};
use postgres_connection::parse_host_port;
use serde::{de::DeserializeOwned, Deserialize, Serialize};
use std::{path::PathBuf, process::Child};
use tracing::instrument;
use utils::id::{NodeId, TenantId};
pub struct AttachmentService {
@@ -16,7 +26,7 @@ const COMMAND: &str = "attachment_service";
#[derive(Serialize, Deserialize)]
pub struct AttachHookRequest {
pub tenant_id: TenantId,
pub tenant_shard_id: TenantShardId,
pub node_id: Option<NodeId>,
}
@@ -27,7 +37,7 @@ pub struct AttachHookResponse {
#[derive(Serialize, Deserialize)]
pub struct InspectRequest {
pub tenant_id: TenantId,
pub tenant_shard_id: TenantShardId,
}
#[derive(Serialize, Deserialize)]
@@ -35,6 +45,46 @@ pub struct InspectResponse {
pub attachment: Option<(u32, NodeId)>,
}
#[derive(Serialize, Deserialize)]
pub struct TenantCreateResponseShard {
pub node_id: NodeId,
pub generation: u32,
}
#[derive(Serialize, Deserialize)]
pub struct TenantCreateResponse {
pub shards: Vec<TenantCreateResponseShard>,
}
#[derive(Serialize, Deserialize)]
pub struct NodeRegisterRequest {
pub node_id: NodeId,
pub listen_pg_addr: String,
pub listen_pg_port: u16,
pub listen_http_addr: String,
pub listen_http_port: u16,
}
#[derive(Serialize, Deserialize, Debug)]
pub struct TenantLocateResponseShard {
pub shard_id: TenantShardId,
pub node_id: NodeId,
pub listen_pg_addr: String,
pub listen_pg_port: u16,
pub listen_http_addr: String,
pub listen_http_port: u16,
}
#[derive(Serialize, Deserialize)]
pub struct TenantLocateResponse {
pub shards: Vec<TenantLocateResponseShard>,
pub shard_params: ShardParameters,
}
impl AttachmentService {
pub fn from_env(env: &LocalEnv) -> Self {
let path = env.base_data_dir.join("attachments.json");
@@ -67,31 +117,87 @@ impl AttachmentService {
pub async fn start(&self) -> anyhow::Result<Child> {
let path_str = self.path.to_string_lossy();
background_process::start_process(
let result = background_process::start_process(
COMMAND,
&self.env.base_data_dir,
&self.env.attachment_service_bin(),
["-l", &self.listen, "-p", &path_str],
[],
background_process::InitialPidFile::Create(self.pid_file()),
// TODO: a real status check
|| async move { anyhow::Ok(true) },
|| async {
match self.status().await {
Ok(_) => Ok(true),
Err(_) => Ok(false),
}
},
)
.await
.await;
for ps_conf in &self.env.pageservers {
let (pg_host, pg_port) =
parse_host_port(&ps_conf.listen_pg_addr).expect("Unable to parse listen_pg_addr");
let (http_host, http_port) = parse_host_port(&ps_conf.listen_http_addr)
.expect("Unable to parse listen_http_addr");
self.node_register(NodeRegisterRequest {
node_id: ps_conf.id,
listen_pg_addr: pg_host.to_string(),
listen_pg_port: pg_port.unwrap_or(5432),
listen_http_addr: http_host.to_string(),
listen_http_port: http_port.unwrap_or(80),
})
.await?;
}
result
}
pub fn stop(&self, immediate: bool) -> anyhow::Result<()> {
background_process::stop_process(immediate, COMMAND, &self.pid_file())
}
/// Simple HTTP request wrapper for calling into attachment service
async fn dispatch<RQ, RS>(
&self,
method: hyper::Method,
path: String,
body: Option<RQ>,
) -> anyhow::Result<RS>
where
RQ: Serialize + Sized,
RS: DeserializeOwned + Sized,
{
let url = self
.env
.control_plane_api
.clone()
.unwrap()
.join(&path)
.unwrap();
let mut builder = self.client.request(method, url);
if let Some(body) = body {
builder = builder.json(&body)
}
let response = builder.send().await?;
if response.status() != StatusCode::OK {
return Err(anyhow!(
"Unexpected status {} on {}",
response.status(),
path
));
}
Ok(response.json().await?)
}
/// Call into the attach_hook API, for use before handing out attachments to pageservers
#[instrument(skip(self))]
pub async fn attach_hook(
&self,
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
pageserver_id: NodeId,
) -> anyhow::Result<Option<u32>> {
use hyper::StatusCode;
let url = self
.env
.control_plane_api
@@ -101,7 +207,7 @@ impl AttachmentService {
.unwrap();
let request = AttachHookRequest {
tenant_id,
tenant_shard_id,
node_id: Some(pageserver_id),
};
@@ -114,9 +220,11 @@ impl AttachmentService {
Ok(response.gen)
}
pub async fn inspect(&self, tenant_id: TenantId) -> anyhow::Result<Option<(u32, NodeId)>> {
use hyper::StatusCode;
#[instrument(skip(self))]
pub async fn inspect(
&self,
tenant_shard_id: TenantShardId,
) -> anyhow::Result<Option<(u32, NodeId)>> {
let url = self
.env
.control_plane_api
@@ -125,7 +233,7 @@ impl AttachmentService {
.join("inspect")
.unwrap();
let request = InspectRequest { tenant_id };
let request = InspectRequest { tenant_shard_id };
let response = self.client.post(url).json(&request).send().await?;
if response.status() != StatusCode::OK {
@@ -135,4 +243,59 @@ impl AttachmentService {
let response = response.json::<InspectResponse>().await?;
Ok(response.attachment)
}
#[instrument(skip(self))]
pub async fn tenant_create(
&self,
req: TenantCreateRequest,
) -> anyhow::Result<TenantCreateResponse> {
self.dispatch(Method::POST, "tenant".to_string(), Some(req))
.await
}
#[instrument(skip(self))]
pub async fn tenant_locate(&self, tenant_id: TenantId) -> anyhow::Result<TenantLocateResponse> {
self.dispatch::<(), _>(Method::GET, format!("tenant/{tenant_id}/locate"), None)
.await
}
#[instrument(skip(self), fields(%tenant_id, %new_shard_count))]
pub async fn tenant_split(
&self,
tenant_id: TenantId,
new_shard_count: u8,
) -> anyhow::Result<TenantShardSplitResponse> {
self.dispatch(
Method::PUT,
format!("tenant/{tenant_id}/shard_split"),
Some(TenantShardSplitRequest { new_shard_count }),
)
.await
}
#[instrument(skip_all, fields(node_id=%req.node_id))]
pub async fn node_register(&self, req: NodeRegisterRequest) -> anyhow::Result<()> {
self.dispatch::<_, ()>(Method::POST, "node".to_string(), Some(req))
.await
}
#[instrument(skip(self))]
pub async fn status(&self) -> anyhow::Result<()> {
self.dispatch::<(), ()>(Method::GET, "status".to_string(), None)
.await
}
#[instrument(skip_all, fields(%tenant_id, timeline_id=%req.new_timeline_id))]
pub async fn tenant_timeline_create(
&self,
tenant_id: TenantId,
req: TimelineCreateRequest,
) -> anyhow::Result<TimelineInfo> {
self.dispatch(
Method::POST,
format!("tenant/{tenant_id}/timeline"),
Some(req),
)
.await
}
}

View File

@@ -6,14 +6,22 @@
///
use anyhow::anyhow;
use clap::Parser;
use hex::FromHex;
use hyper::StatusCode;
use hyper::{Body, Request, Response};
use pageserver_api::shard::TenantShardId;
use hyper::{Method, StatusCode};
use pageserver_api::models::{
LocationConfig, LocationConfigMode, ShardParameters, TenantConfig, TenantCreateRequest,
TenantLocationConfigRequest, TenantShardSplitRequest, TenantShardSplitResponse,
TimelineCreateRequest, TimelineInfo,
};
use pageserver_api::shard::{ShardCount, ShardIdentity, ShardNumber, TenantShardId};
use reqwest::Client;
use serde::{Deserialize, Serialize};
use std::collections::{BTreeMap, HashMap};
use std::path::{Path, PathBuf};
use std::{collections::HashMap, sync::Arc};
use std::sync::Arc;
use utils::http::endpoint::request_span;
use utils::http::request::parse_request_param;
use utils::id::TenantId;
use utils::logging::{self, LogFormat};
use utils::signals::{ShutdownSignals, Signal};
@@ -24,7 +32,7 @@ use utils::{
json::{json_request, json_response},
RequestExt, RouterBuilder,
},
id::{NodeId, TenantId},
id::NodeId,
tcp_listener,
};
@@ -34,7 +42,9 @@ use pageserver_api::control_api::{
};
use control_plane::attachment_service::{
AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse,
AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse, NodeRegisterRequest,
TenantCreateResponse, TenantCreateResponseShard, TenantLocateResponse,
TenantLocateResponseShard,
};
#[derive(Parser)]
@@ -50,50 +60,71 @@ struct Cli {
path: PathBuf,
}
// The persistent state of each Tenant
#[derive(Serialize, Deserialize, Clone)]
/// Our latest knowledge of how this tenant is configured in the outside world.
///
/// Meaning:
/// * No instance of this type exists for a node: we are certain that we have nothing configured on that
/// node for this shard.
/// * Instance exists with conf==None: we *might* have some state on that node, but we don't know
/// what it is (e.g. we failed partway through configuring it)
/// * Instance exists with conf==Some: this tells us what we last successfully configured on this node,
/// and that configuration will still be present unless something external interfered.
#[derive(Serialize, Deserialize)]
struct ObservedStateLocation {
/// If None, it means we do not know the status of this shard's location on this node, but
/// we know that we might have some state on this node.
conf: Option<LocationConfig>,
}
#[derive(Serialize, Deserialize, Default)]
struct ObservedState {
locations: HashMap<NodeId, ObservedStateLocation>,
}
#[derive(Serialize, Deserialize)]
struct TenantState {
tenant_shard_id: TenantShardId,
shard: ShardIdentity,
// Currently attached pageserver
pageserver: Option<NodeId>,
// Latest generation number: next time we attach, increment this
// and use the incremented number when attaching
generation: u32,
observed: ObservedState,
config: TenantConfig,
}
fn to_hex_map<S, V>(input: &HashMap<TenantId, V>, serializer: S) -> Result<S::Ok, S::Error>
where
S: serde::Serializer,
V: Clone + Serialize,
{
let transformed = input.iter().map(|(k, v)| (hex::encode(k), v.clone()));
#[derive(Serialize, Deserialize, Clone)]
struct NodeState {
id: NodeId,
transformed
.collect::<HashMap<String, V>>()
.serialize(serializer)
listen_http_addr: String,
listen_http_port: u16,
listen_pg_addr: String,
listen_pg_port: u16,
}
fn from_hex_map<'de, D, V>(deserializer: D) -> Result<HashMap<TenantId, V>, D::Error>
where
D: serde::de::Deserializer<'de>,
V: Deserialize<'de>,
{
let hex_map = HashMap::<String, V>::deserialize(deserializer)?;
hex_map
.into_iter()
.map(|(k, v)| {
TenantId::from_hex(k)
.map(|k| (k, v))
.map_err(serde::de::Error::custom)
})
.collect()
impl NodeState {
fn base_url(&self) -> String {
format!(
"http://{}:{}/v1",
self.listen_http_addr, self.listen_http_port
)
}
}
// Top level state available to all HTTP handlers
#[derive(Serialize, Deserialize)]
struct PersistentState {
#[serde(serialize_with = "to_hex_map", deserialize_with = "from_hex_map")]
tenants: HashMap<TenantId, TenantState>,
tenants: BTreeMap<TenantShardId, TenantState>,
pageservers: HashMap<NodeId, NodeState>,
#[serde(skip)]
path: PathBuf,
@@ -127,7 +158,8 @@ impl PersistentState {
{
tracing::info!("Will create state file at {}", path.display());
Self {
tenants: HashMap::new(),
tenants: BTreeMap::new(),
pageservers: HashMap::new(),
path: path.to_owned(),
}
}
@@ -160,6 +192,126 @@ fn get_state(request: &Request<Body>) -> &State {
.as_ref()
}
impl TenantState {
async fn location_config(
&self,
node: &NodeState,
config: LocationConfig,
) -> anyhow::Result<()> {
let configure_request = TenantLocationConfigRequest {
tenant_shard_id: self.tenant_shard_id,
config,
};
let client = Client::new();
let response = client
.request(
Method::PUT,
format!(
"{}/tenant/{}/location_config",
node.base_url(),
self.tenant_shard_id
),
)
.json(&configure_request)
.send()
.await?;
response.error_for_status()?;
Ok(())
}
async fn timeline_create(
&self,
node: &NodeState,
req: &TimelineCreateRequest,
) -> anyhow::Result<TimelineInfo> {
let client = Client::new();
let response = client
.request(
Method::POST,
format!(
"{}/tenant/{}/timeline",
node.base_url(),
self.tenant_shard_id
),
)
.json(req)
.send()
.await?;
response.error_for_status_ref()?;
Ok(response.json().await?)
}
fn schedule(&mut self, scheduler: &mut Scheduler) -> Result<(), ScheduleError> {
if self.pageserver.is_some() {
return Ok(());
}
self.pageserver = Some(scheduler.schedule_shard()?);
Ok(())
}
async fn reconcile(
&mut self,
pageservers: &HashMap<NodeId, NodeState>,
) -> Result<(), ReconcileError> {
let wanted_conf = LocationConfig {
mode: LocationConfigMode::AttachedSingle,
generation: Some(self.generation),
secondary_conf: None,
shard_number: self.shard.number.0,
shard_count: self.shard.count.0,
shard_stripe_size: self.shard.stripe_size.0,
tenant_conf: self.config.clone(),
};
match self.pageserver {
Some(node_id) => {
match self.observed.locations.get(&node_id) {
Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {
// Nothing to do
tracing::info!("Observed configuration already correct.")
}
Some(_) | None => {
// If there is no observed configuration, or if its value does not equal our intent, then we must call out to the pageserver.
tracing::info!("Observed configuration requires update.");
let node = pageservers
.get(&node_id)
.expect("Pageserver may not be removed while referenced");
self.location_config(node, wanted_conf).await?;
}
}
}
None => {
// Detach everything
for node_id in self.observed.locations.keys() {
let node = pageservers
.get(node_id)
.expect("Pageserver may not be removed while referenced");
self.location_config(
node,
LocationConfig {
mode: LocationConfigMode::Detached,
generation: None,
secondary_conf: None,
shard_number: self.shard.number.0,
shard_count: self.shard.count.0,
shard_stripe_size: self.shard.stripe_size.0,
tenant_conf: self.config.clone(),
},
)
.await?;
}
}
}
Ok(())
}
}
/// Pageserver calls into this on startup, to learn which tenants it should attach
async fn handle_re_attach(mut req: Request<Body>) -> Result<Response<Body>, ApiError> {
let reattach_req = json_request::<ReAttachRequest>(&mut req).await?;
@@ -174,8 +326,7 @@ async fn handle_re_attach(mut req: Request<Body>) -> Result<Response<Body>, ApiE
if state.pageserver == Some(reattach_req.node_id) {
state.generation += 1;
response.tenants.push(ReAttachResponseTenant {
// TODO(sharding): make this shard-aware
id: TenantShardId::unsharded(*t),
id: *t,
gen: state.generation,
});
}
@@ -198,8 +349,7 @@ async fn handle_validate(mut req: Request<Body>) -> Result<Response<Body>, ApiEr
};
for req_tenant in validate_req.tenants {
// TODO(sharding): make this shard-aware
if let Some(tenant_state) = locked.tenants.get(&req_tenant.id.tenant_id) {
if let Some(tenant_state) = locked.tenants.get(&req_tenant.id) {
let valid = tenant_state.generation == req_tenant.gen;
tracing::info!(
"handle_validate: {}(gen {}): valid={valid} (latest {})",
@@ -227,30 +377,34 @@ async fn handle_attach_hook(mut req: Request<Body>) -> Result<Response<Body>, Ap
let tenant_state = locked
.tenants
.entry(attach_req.tenant_id)
.entry(attach_req.tenant_shard_id)
.or_insert_with(|| TenantState {
tenant_shard_id: attach_req.tenant_shard_id,
pageserver: attach_req.node_id,
generation: 0,
shard: ShardIdentity::unsharded(),
observed: ObservedState::default(),
config: TenantConfig::default(),
});
if let Some(attaching_pageserver) = attach_req.node_id.as_ref() {
tenant_state.generation += 1;
tracing::info!(
tenant_id = %attach_req.tenant_id,
tenant_id = %attach_req.tenant_shard_id,
ps_id = %attaching_pageserver,
generation = %tenant_state.generation,
"issuing",
);
} else if let Some(ps_id) = tenant_state.pageserver {
tracing::info!(
tenant_id = %attach_req.tenant_id,
tenant_id = %attach_req.tenant_shard_id,
%ps_id,
generation = %tenant_state.generation,
"dropping",
);
} else {
tracing::info!(
tenant_id = %attach_req.tenant_id,
tenant_id = %attach_req.tenant_shard_id,
"no-op: tenant already has no pageserver");
}
tenant_state.pageserver = attach_req.node_id;
@@ -258,7 +412,7 @@ async fn handle_attach_hook(mut req: Request<Body>) -> Result<Response<Body>, Ap
tracing::info!(
"handle_attach_hook: tenant {} set generation {}, pageserver {}",
attach_req.tenant_id,
attach_req.tenant_shard_id,
tenant_state.generation,
attach_req.node_id.unwrap_or(utils::id::NodeId(0xfffffff))
);
@@ -278,7 +432,7 @@ async fn handle_inspect(mut req: Request<Body>) -> Result<Response<Body>, ApiErr
let state = get_state(&req).inner.clone();
let locked = state.write().await;
let tenant_state = locked.tenants.get(&inspect_req.tenant_id);
let tenant_state = locked.tenants.get(&inspect_req.tenant_shard_id);
json_response(
StatusCode::OK,
@@ -288,13 +442,510 @@ async fn handle_inspect(mut req: Request<Body>) -> Result<Response<Body>, ApiErr
)
}
/// Scenarios in which we cannot find a suitable location for a tenant shard
#[derive(thiserror::Error, Debug)]
enum ScheduleError {
#[error("No pageservers found")]
NoPageservers,
}
impl From<ScheduleError> for ApiError {
fn from(value: ScheduleError) -> Self {
ApiError::Conflict(format!("Scheduling error: {}", value))
}
}
#[derive(thiserror::Error, Debug)]
enum ReconcileError {
#[error(transparent)]
Other(#[from] anyhow::Error),
}
impl From<ReconcileError> for ApiError {
fn from(value: ReconcileError) -> Self {
ApiError::Conflict(format!("Reconciliation error: {}", value))
}
}
struct Scheduler {
tenant_counts: HashMap<NodeId, usize>,
}
impl Scheduler {
fn new(persistent_state: &PersistentState) -> Self {
let mut tenant_counts = HashMap::new();
for node_id in persistent_state.pageservers.keys() {
tenant_counts.insert(*node_id, 0);
}
for tenant in persistent_state.tenants.values() {
if let Some(ps) = tenant.pageserver {
let entry = tenant_counts.entry(ps).or_insert(0);
*entry += 1;
}
}
Self { tenant_counts }
}
fn schedule_shard(&mut self) -> Result<NodeId, ScheduleError> {
if self.tenant_counts.is_empty() {
return Err(ScheduleError::NoPageservers);
}
let mut tenant_counts: Vec<(NodeId, usize)> =
self.tenant_counts.iter().map(|(k, v)| (*k, *v)).collect();
tenant_counts.sort_by_key(|i| i.1);
for (node_id, count) in &tenant_counts {
tracing::info!("tenant_counts[{node_id}]={count}");
}
let node_id = tenant_counts.first().unwrap().0;
tracing::info!("scheduler selected node {node_id}");
*self.tenant_counts.get_mut(&node_id).unwrap() += 1;
Ok(node_id)
}
}
async fn handle_tenant_create(mut req: Request<Body>) -> Result<Response<Body>, ApiError> {
let create_req = json_request::<TenantCreateRequest>(&mut req).await?;
let state = get_state(&req).inner.clone();
let mut locked = state.write().await;
tracing::info!(
"Creating tenant {}, shard_count={:?}, have {} pageservers",
create_req.new_tenant_id,
create_req.shard_parameters.count,
locked.pageservers.len()
);
// This service expects to handle sharding itself: it is an error to try and directly create
// a particular shard here.
let tenant_id = if create_req.new_tenant_id.shard_count > ShardCount(1) {
return Err(ApiError::BadRequest(anyhow::anyhow!(
"Attempted to create a specific shard, this API is for creating the whole tenant"
)));
} else {
create_req.new_tenant_id.tenant_id
};
// Shard count 0 is valid: it means create a single shard (ShardCount(0) means "unsharded")
let literal_shard_count = if create_req.shard_parameters.is_unsharded() {
1
} else {
create_req.shard_parameters.count.0
};
let mut response_shards = Vec::new();
let mut scheduler = Scheduler::new(&locked);
for i in 0..literal_shard_count {
let shard_number = ShardNumber(i);
let tenant_shard_id = TenantShardId {
tenant_id,
shard_number,
shard_count: create_req.shard_parameters.count,
};
tracing::info!("Creating shard {tenant_shard_id}...");
use std::collections::btree_map::Entry;
match locked.tenants.entry(tenant_shard_id) {
Entry::Occupied(mut entry) => {
tracing::info!("Tenant shard {tenant_shard_id} already exists while creating");
if entry.get_mut().pageserver.is_none() {
entry.get_mut().pageserver = Some(scheduler.schedule_shard().map_err(|e| {
ApiError::Conflict(format!(
"Failed to schedule shard {tenant_shard_id}: {e}"
))
})?);
}
response_shards.push(TenantCreateResponseShard {
node_id: entry
.get()
.pageserver
.expect("We just set pageserver if it was None"),
generation: entry.get().generation,
});
continue;
}
Entry::Vacant(entry) => {
let state = TenantState {
tenant_shard_id,
pageserver: Some(scheduler.schedule_shard().map_err(|e| {
ApiError::Conflict(format!(
"Failed to schedule shard {tenant_shard_id}: {e}"
))
})?),
generation: create_req.generation.unwrap_or(1),
shard: ShardIdentity::from_params(shard_number, &create_req.shard_parameters),
observed: ObservedState::default(),
config: create_req.config.clone(),
};
response_shards.push(TenantCreateResponseShard {
node_id: state
.pageserver
.expect("We just set pageserver if it was None"),
generation: state.generation,
});
entry.insert(state)
}
};
}
// Take a snapshot of pageservers
let pageservers = locked.pageservers.clone();
for (tenant_shard_id, shard) in locked
.tenants
.range_mut(TenantShardId::tenant_range(tenant_id))
{
shard.reconcile(&pageservers).await.map_err(|e| {
ApiError::Conflict(format!(
"Failed to reconcile tenant shard {}: {}",
tenant_shard_id, e
))
})?;
}
locked.save().await.map_err(ApiError::InternalServerError)?;
json_response(
StatusCode::OK,
TenantCreateResponse {
shards: response_shards,
},
)
}
async fn handle_tenant_timeline_create(mut req: Request<Body>) -> Result<Response<Body>, ApiError> {
let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?;
let mut create_req = json_request::<TimelineCreateRequest>(&mut req).await?;
let state = get_state(&req).inner.clone();
let mut locked = state.write().await;
tracing::info!(
"Creating timeline {}/{}, have {} pageservers",
tenant_id,
create_req.new_timeline_id,
locked.pageservers.len()
);
let mut scheduler = Scheduler::new(&locked);
// Take a snapshot of pageservers
let pageservers = locked.pageservers.clone();
let mut timeline_info = None;
for (_tenant_shard_id, shard) in locked
.tenants
.range_mut(TenantShardId::tenant_range(tenant_id))
{
shard.schedule(&mut scheduler)?;
shard.reconcile(&pageservers).await?;
let node_id = shard.pageserver.expect("We just scheduled successfully");
let node = pageservers
.get(&node_id)
.expect("Pageservers may not be deleted while referenced");
let shard_timeline_info = shard
.timeline_create(node, &create_req)
.await
.map_err(|e| ApiError::Conflict(format!("Failed to create timeline: {e}")))?;
if timeline_info.is_none() {
// If the caller specified an ancestor but no ancestor LSN, we are responsible for
// propagating the LSN chosen by the first shard to the other shards: it is important
// that all shards end up with the same ancestor_start_lsn.
if create_req.ancestor_timeline_id.is_some() && create_req.ancestor_start_lsn.is_none()
{
create_req.ancestor_start_lsn = shard_timeline_info.ancestor_lsn;
}
// We will return the TimelineInfo from the first shard
timeline_info = Some(shard_timeline_info);
}
}
json_response(StatusCode::OK, timeline_info)
}
async fn handle_tenant_locate(req: Request<Body>) -> Result<Response<Body>, ApiError> {
let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?;
let state = get_state(&req).inner.clone();
let mut locked = state.write().await;
tracing::info!("Locating shards for tenant {tenant_id}");
// Take a snapshot of pageservers
let pageservers = locked.pageservers.clone();
let mut result = Vec::new();
let mut shard_params: Option<ShardParameters> = None;
for (tenant_shard_id, shard) in locked
.tenants
.range_mut(TenantShardId::tenant_range(tenant_id))
{
let node_id = shard
.pageserver
.ok_or(ApiError::BadRequest(anyhow::anyhow!(
"Cannot locate a tenant that is not attached"
)))?;
let node = pageservers
.get(&node_id)
.expect("Pageservers may not be deleted while referenced");
result.push(TenantLocateResponseShard {
shard_id: *tenant_shard_id,
node_id,
listen_http_addr: node.listen_http_addr.clone(),
listen_http_port: node.listen_http_port,
listen_pg_addr: node.listen_pg_addr.clone(),
listen_pg_port: node.listen_pg_port,
});
match &shard_params {
None => {
shard_params = Some(ShardParameters {
stripe_size: Some(shard.shard.stripe_size),
count: shard.shard.count,
});
}
Some(params) => {
if params.stripe_size != Some(shard.shard.stripe_size) {
// This should never happen. We enforce at runtime because it's simpler than
// adding an extra per-tenant data structure to store the things that should be the same
return Err(ApiError::InternalServerError(anyhow::anyhow!(
"Inconsistent shard stripe size parameters!"
)));
}
}
}
}
if result.is_empty() {
return Err(ApiError::NotFound(
anyhow::anyhow!("No shards for this tenant ID found").into(),
));
}
let shard_params = shard_params.expect("result is non-empty, therefore this is set");
tracing::info!(
"Located tenant {} with params {:?} on shards {}",
tenant_id,
shard_params,
result
.iter()
.map(|s| format!("{:?}", s))
.collect::<Vec<_>>()
.join(",")
);
json_response(
StatusCode::OK,
TenantLocateResponse {
shards: result,
shard_params,
},
)
}
async fn handle_node_register(mut req: Request<Body>) -> Result<Response<Body>, ApiError> {
let register_req = json_request::<NodeRegisterRequest>(&mut req).await?;
let state = get_state(&req).inner.clone();
let mut locked = state.write().await;
locked.pageservers.insert(
register_req.node_id,
NodeState {
id: register_req.node_id,
listen_http_addr: register_req.listen_http_addr,
listen_http_port: register_req.listen_http_port,
listen_pg_addr: register_req.listen_pg_addr,
listen_pg_port: register_req.listen_pg_port,
},
);
tracing::info!(
"Registered pageserver {}, now have {} pageservers",
register_req.node_id,
locked.pageservers.len()
);
json_response(StatusCode::OK, ())
}
async fn handle_tenant_shard_split(mut req: Request<Body>) -> Result<Response<Body>, ApiError> {
let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?;
let split_req = json_request::<TenantShardSplitRequest>(&mut req).await?;
let state = get_state(&req).inner.clone();
let mut locked = state.write().await;
let pageservers = locked.pageservers.clone();
let mut replacements = HashMap::new();
for (tenant_shard_id, shard) in locked
.tenants
.range_mut(TenantShardId::tenant_range(tenant_id))
{
if tenant_shard_id.shard_count == ShardCount(split_req.new_shard_count) {
tracing::warn!(
"Tenant shard {} already has shard count {}",
tenant_shard_id,
split_req.new_shard_count
);
continue;
}
let node_id = shard
.pageserver
.ok_or(ApiError::BadRequest(anyhow::anyhow!(
"Cannot split a tenant that is not attached"
)))?;
let node = pageservers
.get(&node_id)
.expect("Pageservers may not be deleted while referenced");
let client = Client::new();
let response = client
.request(
Method::PUT,
format!("{}/tenant/{}/shard_split", node.base_url(), tenant_shard_id),
)
.json(&TenantShardSplitRequest {
new_shard_count: split_req.new_shard_count,
})
.send()
.await
.map_err(|e| {
ApiError::Conflict(format!("Failed to split {}: {}", tenant_shard_id, e))
})?;
// response.error_for_status().map_err(|e| {
// ApiError::Conflict(format!("Failed to split {}: {}", tenant_shard_id, e))
// })?;
response.error_for_status_ref().map_err(|e| {
ApiError::Conflict(format!("Failed to split {}: {}", tenant_shard_id, e))
})?;
let response: TenantShardSplitResponse = response.json().await.map_err(|e| {
ApiError::InternalServerError(anyhow::anyhow!(
"Malformed response from pageserver: {}",
e
))
})?;
tracing::info!(
"Split {} into {}",
tenant_shard_id,
response
.new_shards
.iter()
.map(|s| format!("{:?}", s))
.collect::<Vec<_>>()
.join(",")
);
replacements.insert(*tenant_shard_id, response.new_shards);
}
// Replace all the shards we just split with their children
let mut response = TenantShardSplitResponse {
new_shards: Vec::new(),
};
for (replaced, children) in replacements.into_iter() {
let (pageserver, generation, shard_ident, config) = {
let old_state = locked
.tenants
.remove(&replaced)
.expect("It was present, we just split it");
(
old_state.pageserver.unwrap(),
old_state.generation,
old_state.shard,
old_state.config.clone(),
)
};
locked.tenants.remove(&replaced);
for child in children {
let mut child_shard = shard_ident;
child_shard.number = child.shard_number;
child_shard.count = child.shard_count;
let mut child_observed: HashMap<NodeId, ObservedStateLocation> = HashMap::new();
child_observed.insert(
pageserver,
ObservedStateLocation {
conf: Some(LocationConfig {
mode: LocationConfigMode::AttachedSingle,
generation: Some(generation),
secondary_conf: None,
shard_number: child.shard_number.0,
shard_count: child.shard_count.0,
shard_stripe_size: shard_ident.stripe_size.0,
tenant_conf: config.clone(),
}),
},
);
locked.tenants.insert(
child,
TenantState {
tenant_shard_id: child,
shard: child_shard,
pageserver: Some(pageserver),
generation,
observed: ObservedState {
locations: child_observed,
},
config: config.clone(),
},
);
response.new_shards.push(child);
}
}
locked.save().await.map_err(ApiError::InternalServerError)?;
json_response(StatusCode::OK, response)
}
/// Status endpoint is just used for checking that our HTTP listener is up
async fn handle_status(_req: Request<Body>) -> Result<Response<Body>, ApiError> {
json_response(StatusCode::OK, ())
}
fn make_router(persistent_state: PersistentState) -> RouterBuilder<hyper::Body, ApiError> {
endpoint::make_router()
.data(Arc::new(State::new(persistent_state)))
.get("/status", |r| request_span(r, handle_status))
.post("/re-attach", |r| request_span(r, handle_re_attach))
.post("/validate", |r| request_span(r, handle_validate))
.post("/attach-hook", |r| request_span(r, handle_attach_hook))
.post("/inspect", |r| request_span(r, handle_inspect))
.post("/node", |r| request_span(r, handle_node_register))
.post("/tenant", |r| request_span(r, handle_tenant_create))
.post("/tenant/:tenant_id/timeline", |r| {
request_span(r, handle_tenant_timeline_create)
})
.get("/tenant/:tenant_id/locate", |r| {
request_span(r, handle_tenant_locate)
})
.put("/tenant/:tenant_id/shard_split", |r| {
request_span(r, handle_tenant_shard_split)
})
}
#[tokio::main]

View File

@@ -15,7 +15,10 @@ use control_plane::pageserver::{PageServerNode, PAGESERVER_REMOTE_STORAGE_DIR};
use control_plane::safekeeper::SafekeeperNode;
use control_plane::tenant_migration::migrate_tenant;
use control_plane::{broker, local_env};
use pageserver_api::models::TimelineInfo;
use pageserver_api::models::{
ShardParameters, TenantCreateRequest, TimelineCreateRequest, TimelineInfo,
};
use pageserver_api::shard::{ShardCount, ShardStripeSize, TenantShardId};
use pageserver_api::{
DEFAULT_HTTP_LISTEN_PORT as DEFAULT_PAGESERVER_HTTP_PORT,
DEFAULT_PG_LISTEN_PORT as DEFAULT_PAGESERVER_PG_PORT,
@@ -30,6 +33,7 @@ use std::path::PathBuf;
use std::process::exit;
use std::str::FromStr;
use storage_broker::DEFAULT_LISTEN_ADDR as DEFAULT_BROKER_ADDR;
use url::Host;
use utils::{
auth::{Claims, Scope},
id::{NodeId, TenantId, TenantTimelineId, TimelineId},
@@ -276,10 +280,10 @@ fn print_timeline(
/// Connects to the pageserver to query this information.
async fn get_timeline_infos(
env: &local_env::LocalEnv,
tenant_id: &TenantId,
tenant_shard_id: &TenantShardId,
) -> Result<HashMap<TimelineId, TimelineInfo>> {
Ok(get_default_pageserver(env)
.timeline_list(tenant_id)
.timeline_list(tenant_shard_id)
.await?
.into_iter()
.map(|timeline_info| (timeline_info.timeline_id, timeline_info))
@@ -297,6 +301,20 @@ fn get_tenant_id(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> anyhow::R
}
}
// Helper function to parse --tenant_id option, for commands that accept a shard suffix
fn get_tenant_shard_id(
sub_match: &ArgMatches,
env: &local_env::LocalEnv,
) -> anyhow::Result<TenantShardId> {
if let Some(tenant_id_from_arguments) = parse_tenant_shard_id(sub_match).transpose() {
tenant_id_from_arguments
} else if let Some(default_id) = env.default_tenant_id {
Ok(TenantShardId::unsharded(default_id))
} else {
anyhow::bail!("No tenant shard id. Use --tenant-id, or set a default tenant");
}
}
fn parse_tenant_id(sub_match: &ArgMatches) -> anyhow::Result<Option<TenantId>> {
sub_match
.get_one::<String>("tenant-id")
@@ -305,6 +323,14 @@ fn parse_tenant_id(sub_match: &ArgMatches) -> anyhow::Result<Option<TenantId>> {
.context("Failed to parse tenant id from the argument string")
}
fn parse_tenant_shard_id(sub_match: &ArgMatches) -> anyhow::Result<Option<TenantShardId>> {
sub_match
.get_one::<String>("tenant-id")
.map(|id_str| TenantShardId::from_str(id_str))
.transpose()
.context("Failed to parse tenant shard id from the argument string")
}
fn parse_timeline_id(sub_match: &ArgMatches) -> anyhow::Result<Option<TimelineId>> {
sub_match
.get_one::<String>("timeline-id")
@@ -393,47 +419,66 @@ async fn handle_tenant(
Some(("create", create_match)) => {
let tenant_conf: HashMap<_, _> = create_match
.get_many::<String>("config")
.map(|vals| vals.flat_map(|c| c.split_once(':')).collect())
.map(|vals: clap::parser::ValuesRef<'_, String>| {
vals.flat_map(|c| c.split_once(':')).collect()
})
.unwrap_or_default();
let shard_count: u8 = create_match
.get_one::<u8>("shard-count")
.cloned()
.unwrap_or(0);
let shard_stripe_size: Option<u32> =
create_match.get_one::<u32>("shard-stripe-size").cloned();
let tenant_conf = PageServerNode::parse_config(tenant_conf)?;
// If tenant ID was not specified, generate one
let tenant_id = parse_tenant_id(create_match)?.unwrap_or_else(TenantId::generate);
let generation = if env.control_plane_api.is_some() {
// We must register the tenant with the attachment service, so
// that when the pageserver restarts, it will be re-attached.
let attachment_service = AttachmentService::from_env(env);
attachment_service
.attach_hook(tenant_id, pageserver.conf.id)
.await?
} else {
None
};
pageserver
.tenant_create(tenant_id, generation, tenant_conf)
// We must register the tenant with the attachment service, so
// that when the pageserver restarts, it will be re-attached.
let attachment_service = AttachmentService::from_env(env);
attachment_service
.tenant_create(TenantCreateRequest {
// Note that ::unsharded here isn't actually because the tenant is unsharded, its because the
// attachment service expecfs a shard-naive tenant_id in this attribute, and the TenantCreateRequest
// type is used both in attachment service (for creating tenants) and in pageserver (for creating shards)
new_tenant_id: TenantShardId::unsharded(tenant_id),
generation: None,
shard_parameters: ShardParameters {
count: ShardCount(shard_count),
stripe_size: shard_stripe_size.map(ShardStripeSize),
},
config: tenant_conf,
})
.await?;
println!("tenant {tenant_id} successfully created on the pageserver");
// Create an initial timeline for the new tenant
let new_timeline_id = parse_timeline_id(create_match)?;
let new_timeline_id =
parse_timeline_id(create_match)?.unwrap_or(TimelineId::generate());
let pg_version = create_match
.get_one::<u32>("pg-version")
.copied()
.context("Failed to parse postgres version from the argument string")?;
let timeline_info = pageserver
.timeline_create(
// FIXME: passing None for ancestor_start_lsn is not kosher in a sharded world: we can't have
// different shards picking different start lsns. Maybe we have to teach attachment service
// to let shard 0 branch first and then propagate the chosen LSN to other shards.
attachment_service
.tenant_timeline_create(
tenant_id,
new_timeline_id,
None,
None,
Some(pg_version),
None,
TimelineCreateRequest {
new_timeline_id,
ancestor_timeline_id: None,
ancestor_start_lsn: None,
existing_initdb_timeline_id: None,
pg_version: Some(pg_version),
},
)
.await?;
let new_timeline_id = timeline_info.timeline_id;
let last_record_lsn = timeline_info.last_record_lsn;
env.register_branch_mapping(
DEFAULT_BRANCH_NAME.to_string(),
@@ -441,9 +486,7 @@ async fn handle_tenant(
new_timeline_id,
)?;
println!(
"Created an initial timeline '{new_timeline_id}' at Lsn {last_record_lsn} for tenant: {tenant_id}",
);
println!("Created an initial timeline '{new_timeline_id}' for tenant: {tenant_id}",);
if create_match.get_flag("set-default") {
println!("Setting tenant {tenant_id} as a default one");
@@ -470,12 +513,92 @@ async fn handle_tenant(
println!("tenant {tenant_id} successfully configured on the pageserver");
}
Some(("migrate", matches)) => {
let tenant_id = get_tenant_id(matches, env)?;
let tenant_shard_id = get_tenant_shard_id(matches, env)?;
let new_pageserver = get_pageserver(env, matches)?;
let new_pageserver_id = new_pageserver.conf.id;
migrate_tenant(env, tenant_id, new_pageserver).await?;
println!("tenant {tenant_id} migrated to {}", new_pageserver_id);
migrate_tenant(env, tenant_shard_id, new_pageserver).await?;
println!("tenant {tenant_shard_id} migrated to {}", new_pageserver_id);
}
Some(("split", matches)) => {
let tenant_id = get_tenant_id(matches, env)?;
let attachment_service = AttachmentService::from_env(env);
let old_shards = attachment_service.tenant_locate(tenant_id).await?.shards;
let new_shard_count = old_shards.len() * 2;
if old_shards.len() > 127 {
bail!("Cannot split further");
}
attachment_service
.tenant_split(tenant_id, new_shard_count as u8)
.await?;
println!("Split {}->{}", old_shards.len(), new_shard_count);
}
Some(("status", matches)) => {
let tenant_id = get_tenant_id(matches, env)?;
let mut shard_table = comfy_table::Table::new();
shard_table.set_header(["Shard", "Pageserver", "Physical Size"]);
let mut tenant_synthetic_size = None;
let attachment_service = AttachmentService::from_env(env);
for shard in attachment_service.tenant_locate(tenant_id).await?.shards {
let pageserver =
PageServerNode::from_env(env, env.get_pageserver_conf(shard.node_id)?);
let size = pageserver
.http_client
.tenant_details(shard.shard_id)
.await?
.tenant_info
.current_physical_size
.unwrap();
shard_table.add_row([
format!("{}", shard.shard_id.shard_slug()),
format!("{}", shard.node_id.0),
format!("{} MiB", size / (1024 * 1024)),
]);
if shard.shard_id.is_zero() {
tenant_synthetic_size =
Some(pageserver.tenant_synthetic_size(shard.shard_id).await?);
}
}
let Some(synthetic_size) = tenant_synthetic_size else {
bail!("Shard 0 not found")
};
let mut tenant_table = comfy_table::Table::new();
tenant_table.add_row(["Tenant ID".to_string(), tenant_id.to_string()]);
tenant_table.add_row([
"Synthetic size".to_string(),
format!("{} MiB", synthetic_size.size.unwrap_or(0) / (1024 * 1024)),
]);
println!("{tenant_table}");
println!("{shard_table}");
}
Some(("shard-split", matches)) => {
let tenant_id = get_tenant_id(matches, env)?;
let shard_count: u8 = matches.get_one::<u8>("shard-count").cloned().unwrap_or(0);
let attachment_service = AttachmentService::from_env(env);
let result = attachment_service
.tenant_split(tenant_id, shard_count)
.await?;
println!(
"Split tenant {} into shards {}",
tenant_id,
result
.new_shards
.iter()
.map(|s| format!("{:?}", s))
.collect::<Vec<_>>()
.join(",")
);
}
Some((sub_name, _)) => bail!("Unexpected tenant subcommand '{}'", sub_name),
@@ -489,8 +612,10 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local
match timeline_match.subcommand() {
Some(("list", list_match)) => {
let tenant_id = get_tenant_id(list_match, env)?;
let timelines = pageserver.timeline_list(&tenant_id).await?;
// TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the attachment service
// where shard 0 is attached, and query there.
let tenant_shard_id = get_tenant_shard_id(list_match, env)?;
let timelines = pageserver.timeline_list(&tenant_shard_id).await?;
print_timelines_tree(timelines, env.timeline_name_mappings())?;
}
Some(("create", create_match)) => {
@@ -505,18 +630,19 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local
.context("Failed to parse postgres version from the argument string")?;
let new_timeline_id_opt = parse_timeline_id(create_match)?;
let new_timeline_id = new_timeline_id_opt.unwrap_or(TimelineId::generate());
let timeline_info = pageserver
.timeline_create(
tenant_id,
new_timeline_id_opt,
None,
None,
Some(pg_version),
None,
)
let attachment_service = AttachmentService::from_env(env);
let create_req = TimelineCreateRequest {
new_timeline_id,
ancestor_timeline_id: None,
existing_initdb_timeline_id: None,
ancestor_start_lsn: None,
pg_version: Some(pg_version),
};
let timeline_info = attachment_service
.tenant_timeline_create(tenant_id, create_req)
.await?;
let new_timeline_id = timeline_info.timeline_id;
let last_record_lsn = timeline_info.last_record_lsn;
env.register_branch_mapping(new_branch_name.to_string(), tenant_id, new_timeline_id)?;
@@ -574,7 +700,6 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local
None,
pg_version,
ComputeMode::Primary,
DEFAULT_PAGESERVER_ID,
)?;
println!("Done");
}
@@ -598,17 +723,18 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local
.map(|lsn_str| Lsn::from_str(lsn_str))
.transpose()
.context("Failed to parse ancestor start Lsn from the request")?;
let timeline_info = pageserver
.timeline_create(
tenant_id,
None,
start_lsn,
Some(ancestor_timeline_id),
None,
None,
)
let new_timeline_id = TimelineId::generate();
let attachment_service = AttachmentService::from_env(env);
let create_req = TimelineCreateRequest {
new_timeline_id,
ancestor_timeline_id: Some(ancestor_timeline_id),
existing_initdb_timeline_id: None,
ancestor_start_lsn: start_lsn,
pg_version: None,
};
let timeline_info = attachment_service
.tenant_timeline_create(tenant_id, create_req)
.await?;
let new_timeline_id = timeline_info.timeline_id;
let last_record_lsn = timeline_info.last_record_lsn;
@@ -635,8 +761,10 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
match sub_name {
"list" => {
let tenant_id = get_tenant_id(sub_args, env)?;
let timeline_infos = get_timeline_infos(env, &tenant_id)
// TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the attachment service
// where shard 0 is attached, and query there.
let tenant_shard_id = get_tenant_shard_id(sub_args, env)?;
let timeline_infos = get_timeline_infos(env, &tenant_shard_id)
.await
.unwrap_or_else(|e| {
eprintln!("Failed to load timeline info: {}", e);
@@ -661,7 +789,7 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
for (endpoint_id, endpoint) in cplane
.endpoints
.iter()
.filter(|(_, endpoint)| endpoint.tenant_id == tenant_id)
.filter(|(_, endpoint)| endpoint.tenant_id == tenant_shard_id.tenant_id)
{
let lsn_str = match endpoint.mode {
ComputeMode::Static(lsn) => {
@@ -680,7 +808,10 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
};
let branch_name = timeline_name_mappings
.get(&TenantTimelineId::new(tenant_id, endpoint.timeline_id))
.get(&TenantTimelineId::new(
tenant_shard_id.tenant_id,
endpoint.timeline_id,
))
.map(|name| name.as_str())
.unwrap_or("?");
@@ -728,13 +859,6 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
.copied()
.unwrap_or(false);
let pageserver_id =
if let Some(id_str) = sub_args.get_one::<String>("endpoint-pageserver-id") {
NodeId(id_str.parse().context("while parsing pageserver id")?)
} else {
DEFAULT_PAGESERVER_ID
};
let mode = match (lsn, hot_standby) {
(Some(lsn), false) => ComputeMode::Static(lsn),
(None, true) => ComputeMode::Replica,
@@ -762,7 +886,6 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
http_port,
pg_version,
mode,
pageserver_id,
)?;
}
"start" => {
@@ -805,6 +928,22 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
endpoint.timeline_id,
)?;
let attachment_service = AttachmentService::from_env(env);
let locate_result = attachment_service.tenant_locate(endpoint.tenant_id).await?;
let pageservers = locate_result
.shards
.into_iter()
.map(|shard| {
(
Host::parse(&shard.listen_pg_addr)
.expect("Attachment service reported bad hostname"),
shard.listen_pg_port,
)
})
.collect::<Vec<_>>();
assert!(!pageservers.is_empty());
let stripe_size = locate_result.shard_params.stripe_size.map(|s| s.0 as usize);
let ps_conf = env.get_pageserver_conf(pageserver_id)?;
let auth_token = if matches!(ps_conf.pg_auth_type, AuthType::NeonJWT) {
let claims = Claims::new(Some(endpoint.tenant_id), Scope::Tenant);
@@ -816,7 +955,13 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
println!("Starting existing endpoint {endpoint_id}...");
endpoint
.start(&auth_token, safekeepers, remote_ext_config)
.start(
&auth_token,
safekeepers,
pageservers,
remote_ext_config,
stripe_size,
)
.await?;
}
"reconfigure" => {
@@ -827,15 +972,31 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re
.endpoints
.get(endpoint_id.as_str())
.with_context(|| format!("postgres endpoint {endpoint_id} is not found"))?;
let pageserver_id =
let pageservers =
if let Some(id_str) = sub_args.get_one::<String>("endpoint-pageserver-id") {
Some(NodeId(
id_str.parse().context("while parsing pageserver id")?,
))
let ps_id = NodeId(id_str.parse().context("while parsing pageserver id")?);
let pageserver = PageServerNode::from_env(env, env.get_pageserver_conf(ps_id)?);
vec![(
pageserver.pg_connection_config.host().clone(),
pageserver.pg_connection_config.port(),
)]
} else {
None
let attachment_service = AttachmentService::from_env(env);
attachment_service
.tenant_locate(endpoint.tenant_id)
.await?
.shards
.into_iter()
.map(|shard| {
(
Host::parse(&shard.listen_pg_addr)
.expect("Attachment service reported malformed host"),
shard.listen_pg_port,
)
})
.collect::<Vec<_>>()
};
endpoint.reconfigure(pageserver_id).await?;
endpoint.reconfigure(pageservers).await?;
}
"stop" => {
let endpoint_id = sub_args
@@ -1352,6 +1513,8 @@ fn cli() -> Command {
.arg(pg_version_arg.clone())
.arg(Arg::new("set-default").long("set-default").action(ArgAction::SetTrue).required(false)
.help("Use this tenant in future CLI commands where tenant_id is needed, but not specified"))
.arg(Arg::new("shard-count").value_parser(value_parser!(u8)).long("shard-count").action(ArgAction::Set).help("Number of shards in the new tenant (default 1)"))
.arg(Arg::new("shard-stripe-size").value_parser(value_parser!(u32)).long("shard-stripe-size").action(ArgAction::Set).help("Sharding stripe size in pages"))
)
.subcommand(Command::new("set-default").arg(tenant_id_arg.clone().required(true))
.about("Set a particular tenant as default in future CLI commands where tenant_id is needed, but not specified"))
@@ -1362,6 +1525,14 @@ fn cli() -> Command {
.about("Migrate a tenant from one pageserver to another")
.arg(tenant_id_arg.clone())
.arg(pageserver_id_arg.clone()))
.subcommand(Command::new("status")
.about("Human readable summary of the tenant's shards and attachment locations")
.arg(tenant_id_arg.clone()))
.subcommand(Command::new("shard-split")
.about("Increase the number of shards in the tenant")
.arg(tenant_id_arg.clone())
.arg(Arg::new("shard-count").value_parser(value_parser!(u8)).long("shard-count").action(ArgAction::Set).help("Number of shards in the new tenant (default 1)"))
)
)
.subcommand(
Command::new("pageserver")

View File

@@ -47,10 +47,11 @@ use std::time::Duration;
use anyhow::{anyhow, bail, Context, Result};
use compute_api::spec::RemoteExtSpec;
use serde::{Deserialize, Serialize};
use url::Host;
use utils::id::{NodeId, TenantId, TimelineId};
use crate::attachment_service::AttachmentService;
use crate::local_env::LocalEnv;
use crate::pageserver::PageServerNode;
use crate::postgresql_conf::PostgresConf;
use compute_api::responses::{ComputeState, ComputeStatus};
@@ -67,7 +68,6 @@ pub struct EndpointConf {
http_port: u16,
pg_version: u32,
skip_pg_catalog_updates: bool,
pageserver_id: NodeId,
}
//
@@ -119,19 +119,14 @@ impl ComputeControlPlane {
http_port: Option<u16>,
pg_version: u32,
mode: ComputeMode,
pageserver_id: NodeId,
) -> Result<Arc<Endpoint>> {
let pg_port = pg_port.unwrap_or_else(|| self.get_port());
let http_port = http_port.unwrap_or_else(|| self.get_port() + 1);
let pageserver =
PageServerNode::from_env(&self.env, self.env.get_pageserver_conf(pageserver_id)?);
let ep = Arc::new(Endpoint {
endpoint_id: endpoint_id.to_owned(),
pg_address: SocketAddr::new("127.0.0.1".parse().unwrap(), pg_port),
http_address: SocketAddr::new("127.0.0.1".parse().unwrap(), http_port),
env: self.env.clone(),
pageserver,
timeline_id,
mode,
tenant_id,
@@ -157,7 +152,6 @@ impl ComputeControlPlane {
pg_port,
pg_version,
skip_pg_catalog_updates: true,
pageserver_id,
})?,
)?;
std::fs::write(
@@ -216,7 +210,6 @@ pub struct Endpoint {
// These are not part of the endpoint as such, but the environment
// the endpoint runs in.
pub env: LocalEnv,
pageserver: PageServerNode,
// Optimizations
skip_pg_catalog_updates: bool,
@@ -239,15 +232,11 @@ impl Endpoint {
let conf: EndpointConf =
serde_json::from_slice(&std::fs::read(entry.path().join("endpoint.json"))?)?;
let pageserver =
PageServerNode::from_env(env, env.get_pageserver_conf(conf.pageserver_id)?);
Ok(Endpoint {
pg_address: SocketAddr::new("127.0.0.1".parse().unwrap(), conf.pg_port),
http_address: SocketAddr::new("127.0.0.1".parse().unwrap(), conf.http_port),
endpoint_id,
env: env.clone(),
pageserver,
timeline_id: conf.timeline_id,
mode: conf.mode,
tenant_id: conf.tenant_id,
@@ -464,11 +453,21 @@ impl Endpoint {
}
}
fn build_pageserver_connstr(pageservers: &[(Host, u16)]) -> String {
pageservers
.iter()
.map(|(host, port)| format!("postgresql://no_user@{host}:{port}"))
.collect::<Vec<_>>()
.join(",")
}
pub async fn start(
&self,
auth_token: &Option<String>,
safekeepers: Vec<NodeId>,
pageservers: Vec<(Host, u16)>,
remote_ext_config: Option<&String>,
shard_stripe_size: Option<usize>,
) -> Result<()> {
if self.status() == "running" {
anyhow::bail!("The endpoint is already running");
@@ -482,13 +481,9 @@ impl Endpoint {
std::fs::remove_dir_all(self.pgdata())?;
}
let pageserver_connstring = {
let config = &self.pageserver.pg_connection_config;
let (host, port) = (config.host(), config.port());
let pageserver_connstring = Self::build_pageserver_connstr(&pageservers);
assert!(!pageserver_connstring.is_empty());
// NOTE: avoid spaces in connection string, because it is less error prone if we forward it somewhere.
format!("postgresql://no_user@{host}:{port}")
};
let mut safekeeper_connstrings = Vec::new();
if self.mode == ComputeMode::Primary {
for sk_id in safekeepers {
@@ -537,6 +532,7 @@ impl Endpoint {
safekeeper_connstrings,
storage_auth_token: auth_token.clone(),
remote_extensions,
shard_stripe_size,
};
let spec_path = self.endpoint_path().join("spec.json");
std::fs::write(spec_path, serde_json::to_string_pretty(&spec)?)?;
@@ -549,8 +545,11 @@ impl Endpoint {
// Launch compute_ctl
println!("Starting postgres node at '{}'", self.connstr());
let mut cmd = Command::new(self.env.neon_distrib_dir.join("compute_ctl"));
cmd.args(["--http-port", &self.http_address.port().to_string()])
let mut cmd = Command::new("/usr/bin/taskset");
cmd.args(["-c".to_string(), "8-11".to_string()])
.args([self.env.neon_distrib_dir.join("compute_ctl")])
.args(["--http-port", &self.http_address.port().to_string()])
.args(["--pgdata", self.pgdata().to_str().unwrap()])
.args(["--connstr", &self.connstr()])
.args([
@@ -659,7 +658,7 @@ impl Endpoint {
}
}
pub async fn reconfigure(&self, pageserver_id: Option<NodeId>) -> Result<()> {
pub async fn reconfigure(&self, mut pageservers: Vec<(Host, u16)>) -> Result<()> {
let mut spec: ComputeSpec = {
let spec_path = self.endpoint_path().join("spec.json");
let file = std::fs::File::open(spec_path)?;
@@ -669,25 +668,27 @@ impl Endpoint {
let postgresql_conf = self.read_postgresql_conf()?;
spec.cluster.postgresql_conf = Some(postgresql_conf);
if let Some(pageserver_id) = pageserver_id {
let endpoint_config_path = self.endpoint_path().join("endpoint.json");
let mut endpoint_conf: EndpointConf = {
let file = std::fs::File::open(&endpoint_config_path)?;
serde_json::from_reader(file)?
};
endpoint_conf.pageserver_id = pageserver_id;
std::fs::write(
endpoint_config_path,
serde_json::to_string_pretty(&endpoint_conf)?,
)?;
let pageserver =
PageServerNode::from_env(&self.env, self.env.get_pageserver_conf(pageserver_id)?);
let ps_http_conf = &pageserver.pg_connection_config;
let (host, port) = (ps_http_conf.host(), ps_http_conf.port());
spec.pageserver_connstring = Some(format!("postgresql://no_user@{host}:{port}"));
// If we weren't given explicit pageservers, query the attachment service
if pageservers.is_empty() {
let attachment_service = AttachmentService::from_env(&self.env);
let locate_result = attachment_service.tenant_locate(self.tenant_id).await?;
pageservers = locate_result
.shards
.into_iter()
.map(|shard| {
(
Host::parse(&shard.listen_pg_addr)
.expect("Attachment service reported bad hostname"),
shard.listen_pg_port,
)
})
.collect::<Vec<_>>();
}
let pageserver_connstr = Self::build_pageserver_connstr(&pageservers);
assert!(!pageserver_connstr.is_empty());
spec.pageserver_connstring = Some(pageserver_connstr);
let client = reqwest::Client::new();
let response = client
.post(format!(

View File

@@ -1,5 +1,6 @@
//! Code to manage pageservers
//!
//!
//! In the local test environment, the pageserver stores its data directly in
//!
//! .neon/
@@ -12,12 +13,15 @@ use std::io::Write;
use std::num::NonZeroU64;
use std::path::PathBuf;
use std::process::{Child, Command};
use std::str::FromStr;
use std::time::Duration;
use anyhow::{bail, Context};
use camino::Utf8PathBuf;
use futures::SinkExt;
use pageserver_api::models::{self, LocationConfig, TenantInfo, TimelineInfo};
use pageserver_api::models::{
self, LocationConfig, ShardParameters, TenantHistorySize, TenantInfo, TimelineInfo,
};
use pageserver_api::shard::TenantShardId;
use pageserver_client::mgmt_api;
use postgres_backend::AuthType;
@@ -216,11 +220,19 @@ impl PageServerNode {
if update_config {
args.push(Cow::Borrowed("--update-config"));
}
let mut taskset_args = vec![
"-c".to_string(),
format!("{}", self.conf.id.0 - 1),
self.env.pageserver_bin().to_string_lossy().into(),
];
taskset_args.extend(args.into_iter().map(|a| a.to_string()));
background_process::start_process(
"pageserver",
&datadir,
&self.env.pageserver_bin(),
args.iter().map(Cow::as_ref),
&PathBuf::from_str("/usr/bin/taskset").unwrap(),
taskset_args,
self.pageserver_env_variables()?,
background_process::InitialPidFile::Expect(self.pid_file()),
|| async {
@@ -301,16 +313,8 @@ impl PageServerNode {
pub async fn tenant_list(&self) -> mgmt_api::Result<Vec<TenantInfo>> {
self.http_client.list_tenants().await
}
pub async fn tenant_create(
&self,
new_tenant_id: TenantId,
generation: Option<u32>,
settings: HashMap<&str, &str>,
) -> anyhow::Result<TenantId> {
let mut settings = settings.clone();
let config = models::TenantConfig {
pub fn parse_config(mut settings: HashMap<&str, &str>) -> anyhow::Result<models::TenantConfig> {
let result = models::TenantConfig {
checkpoint_distance: settings
.remove("checkpoint_distance")
.map(|x| x.parse::<u64>())
@@ -371,11 +375,26 @@ impl PageServerNode {
.context("Failed to parse 'gc_feedback' as bool")?,
heatmap_period: settings.remove("heatmap_period").map(|x| x.to_string()),
};
if !settings.is_empty() {
bail!("Unrecognized tenant settings: {settings:?}")
} else {
Ok(result)
}
}
pub async fn tenant_create(
&self,
new_tenant_id: TenantId,
generation: Option<u32>,
settings: HashMap<&str, &str>,
) -> anyhow::Result<TenantId> {
let config = Self::parse_config(settings.clone())?;
let request = models::TenantCreateRequest {
new_tenant_id: TenantShardId::unsharded(new_tenant_id),
generation,
config,
shard_parameters: ShardParameters::default(),
};
if !settings.is_empty() {
bail!("Unrecognized tenant settings: {settings:?}")
@@ -471,31 +490,32 @@ impl PageServerNode {
pub async fn location_config(
&self,
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
config: LocationConfig,
flush_ms: Option<Duration>,
) -> anyhow::Result<()> {
Ok(self
.http_client
.location_config(tenant_id, config, flush_ms)
.location_config(tenant_shard_id, config, flush_ms)
.await?)
}
pub async fn timeline_list(&self, tenant_id: &TenantId) -> anyhow::Result<Vec<TimelineInfo>> {
Ok(self.http_client.list_timelines(*tenant_id).await?)
pub async fn timeline_list(
&self,
tenant_shard_id: &TenantShardId,
) -> anyhow::Result<Vec<TimelineInfo>> {
Ok(self.http_client.list_timelines(*tenant_shard_id).await?)
}
pub async fn timeline_create(
&self,
tenant_id: TenantId,
new_timeline_id: Option<TimelineId>,
new_timeline_id: TimelineId,
ancestor_start_lsn: Option<Lsn>,
ancestor_timeline_id: Option<TimelineId>,
pg_version: Option<u32>,
existing_initdb_timeline_id: Option<TimelineId>,
) -> anyhow::Result<TimelineInfo> {
// If timeline ID was not specified, generate one
let new_timeline_id = new_timeline_id.unwrap_or(TimelineId::generate());
let req = models::TimelineCreateRequest {
new_timeline_id,
ancestor_start_lsn,
@@ -581,4 +601,14 @@ impl PageServerNode {
Ok(())
}
pub async fn tenant_synthetic_size(
&self,
tenant_shard_id: TenantShardId,
) -> anyhow::Result<TenantHistorySize> {
Ok(self
.http_client
.tenant_synthetic_size(tenant_shard_id)
.await?)
}
}

View File

@@ -11,19 +11,17 @@ use crate::{
use pageserver_api::models::{
LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig,
};
use pageserver_api::shard::TenantShardId;
use std::collections::HashMap;
use std::time::Duration;
use utils::{
id::{TenantId, TimelineId},
lsn::Lsn,
};
use utils::{id::TimelineId, lsn::Lsn};
/// Given an attached pageserver, retrieve the LSN for all timelines
async fn get_lsns(
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
pageserver: &PageServerNode,
) -> anyhow::Result<HashMap<TimelineId, Lsn>> {
let timelines = pageserver.timeline_list(&tenant_id).await?;
let timelines = pageserver.timeline_list(&tenant_shard_id).await?;
Ok(timelines
.into_iter()
.map(|t| (t.timeline_id, t.last_record_lsn))
@@ -33,12 +31,12 @@ async fn get_lsns(
/// Wait for the timeline LSNs on `pageserver` to catch up with or overtake
/// `baseline`.
async fn await_lsn(
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
pageserver: &PageServerNode,
baseline: HashMap<TimelineId, Lsn>,
) -> anyhow::Result<()> {
loop {
let latest = match get_lsns(tenant_id, pageserver).await {
let latest = match get_lsns(tenant_shard_id, pageserver).await {
Ok(l) => l,
Err(e) => {
println!(
@@ -86,7 +84,7 @@ async fn await_lsn(
/// - reconfigure compute endpoints to point to new attached pageserver
pub async fn migrate_tenant(
env: &LocalEnv,
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
dest_ps: PageServerNode,
) -> anyhow::Result<()> {
// Get a new generation
@@ -108,7 +106,7 @@ pub async fn migrate_tenant(
}
}
let previous = attachment_service.inspect(tenant_id).await?;
let previous = attachment_service.inspect(tenant_shard_id).await?;
let mut baseline_lsns = None;
if let Some((generation, origin_ps_id)) = &previous {
let origin_ps = PageServerNode::from_env(env, env.get_pageserver_conf(*origin_ps_id)?);
@@ -116,10 +114,12 @@ pub async fn migrate_tenant(
if origin_ps_id == &dest_ps.conf.id {
println!("🔁 Already attached to {origin_ps_id}, freshening...");
let gen = attachment_service
.attach_hook(tenant_id, dest_ps.conf.id)
.attach_hook(tenant_shard_id, dest_ps.conf.id)
.await?;
let dest_conf = build_location_config(LocationConfigMode::AttachedSingle, gen, None);
dest_ps.location_config(tenant_id, dest_conf, None).await?;
dest_ps
.location_config(tenant_shard_id, dest_conf, None)
.await?;
println!("✅ Migration complete");
return Ok(());
}
@@ -129,33 +129,35 @@ pub async fn migrate_tenant(
let stale_conf =
build_location_config(LocationConfigMode::AttachedStale, Some(*generation), None);
origin_ps
.location_config(tenant_id, stale_conf, Some(Duration::from_secs(10)))
.location_config(tenant_shard_id, stale_conf, Some(Duration::from_secs(10)))
.await?;
baseline_lsns = Some(get_lsns(tenant_id, &origin_ps).await?);
baseline_lsns = Some(get_lsns(tenant_shard_id, &origin_ps).await?);
}
let gen = attachment_service
.attach_hook(tenant_id, dest_ps.conf.id)
.attach_hook(tenant_shard_id, dest_ps.conf.id)
.await?;
let dest_conf = build_location_config(LocationConfigMode::AttachedMulti, gen, None);
println!("🔁 Attaching to pageserver {}", dest_ps.conf.id);
dest_ps.location_config(tenant_id, dest_conf, None).await?;
dest_ps
.location_config(tenant_shard_id, dest_conf, None)
.await?;
if let Some(baseline) = baseline_lsns {
println!("🕑 Waiting for LSN to catch up...");
await_lsn(tenant_id, &dest_ps, baseline).await?;
await_lsn(tenant_shard_id, &dest_ps, baseline).await?;
}
let cplane = ComputeControlPlane::load(env.clone())?;
for (endpoint_name, endpoint) in &cplane.endpoints {
if endpoint.tenant_id == tenant_id {
if endpoint.tenant_id == tenant_shard_id.tenant_id && endpoint.status() == "running" {
println!(
"🔁 Reconfiguring endpoint {} to use pageserver {}",
endpoint_name, dest_ps.conf.id
);
endpoint.reconfigure(Some(dest_ps.conf.id)).await?;
endpoint.reconfigure(vec![]).await?;
}
}
@@ -171,24 +173,29 @@ pub async fn migrate_tenant(
let found = other_ps_tenants
.into_iter()
.map(|t| t.id)
.any(|i| i.tenant_id == tenant_id);
.any(|i| i == tenant_shard_id);
if !found {
continue;
}
// Downgrade to a secondary location
let secondary_conf = build_location_config(
LocationConfigMode::Secondary,
None,
Some(LocationConfigSecondary { warm: true }),
);
// // Downgrade to a secondary location
// let secondary_conf = build_location_config(
// LocationConfigMode::Secondary,
// None,
// Some(LocationConfigSecondary { warm: true }),
// );
println!(
"💤 Switching to secondary mode on pageserver {}",
other_ps.conf.id
);
// println!(
// "💤 Switching to secondary mode on pageserver {}",
// other_ps.conf.id
// );
// other_ps
// .location_config(tenant_shard_id, secondary_conf, None)
// .await?;
let detached_conf = build_location_config(LocationConfigMode::Detached, None, None);
println!("💤 Detaching on pageserver {}", other_ps.conf.id);
other_ps
.location_config(tenant_id, secondary_conf, None)
.location_config(tenant_shard_id, detached_conf, None)
.await?;
}
@@ -197,7 +204,9 @@ pub async fn migrate_tenant(
dest_ps.conf.id
);
let dest_conf = build_location_config(LocationConfigMode::AttachedSingle, gen, None);
dest_ps.location_config(tenant_id, dest_conf, None).await?;
dest_ps
.location_config(tenant_shard_id, dest_conf, None)
.await?;
println!("✅ Migration complete");

61
demo_sharding.sh Normal file
View File

@@ -0,0 +1,61 @@
set -u
export RUST_LOG=INFO
INITIAL_SHARDS="${INITIAL_SHARDS:-1}"
PAGESERVERS=8
FINAL_SHARDS=8
STRIPE_SIZE=128
SCALE=100
export BUILD_ARGS=--release
ARGS="${BUILD_ARGS} -q"
RUST_LOG=info
TENANT_ID=1f359dd625e519a1a4e8d7509690f6fc
set -e
set -x
set +e
cargo neon $ARGS stop ; killall -9 storage_broker ; killall -9 safekeeper ; killall -9 pageserver ; killall -9 postgres ; killall -9 attachment_service ; rm -rf .neon
set -e
cargo build $ARGS --features=testing
cargo neon $ARGS init --num-pageservers=$PAGESERVERS && RUST_LOG=$RUST_LOG cargo neon $ARGS start && cargo neon $ARGS tenant create --shard-count=$INITIAL_SHARDS --shard-stripe-size=$STRIPE_SIZE --tenant-id=$TENANT_ID --timeline-id=3d34095be52fec4c44a92e774c573b57 --set-default
cargo neon $ARGS endpoint create && cargo neon $ARGS endpoint start ep-main
pgbench postgres -i -h 127.0.0.1 -p 55432 -U cloud_admin -s $SCALE
cargo neon $ARGS tenant status
# pgbench postgres -h 127.0.0.1 -p 55432 -U cloud_admin -T 600 -P 1 -c 32
#
# tmux
#Ctrl+b+% horizontal split
#Ctrl+b-o toggle panes
#alias neon="cargo neon --release -q"
# Pt1: baseline: one pageserver
#INITIAL_SHARDS=1 bash demo_sharding.sh
#neon tenant status
#taskset -c 12-15 pgbench postgres -h 127.0.0.1 -p 55432 -U cloud_admin -T 30 -P 1 -c 64
#taskset -c 12-15 pgbench postgres -h 127.0.0.1 -p 55432 -U cloud_admin -T 30 -P 1 -c 64 -S
# Pt2: four shards
#INITIAL_SHARDS=4 bash demo_sharding.sh
#neon tenant status
#taskset -c 12-15 pgbench postgres -h 127.0.0.1 -p 55432 -U cloud_admin -T 30 -P 1 -c 64
#taskset -c 12-15 pgbench postgres -h 127.0.0.1 -p 55432 -U cloud_admin -T 30 -P 1 -c 64 -S
# Pt3: 8 shards
#bash demo_split_8.sh
#taskset -c 12-15 pgbench postgres -h 127.0.0.1 -p 55432 -U cloud_admin -T 30 -P 1 -c 64 -S

16
demo_split_4.sh Normal file
View File

@@ -0,0 +1,16 @@
FINAL_SHARDS=4
TENANT_ID=1f359dd625e519a1a4e8d7509690f6fc
ARGS=--release -q
cargo neon $ARGS endpoint stop ep-main
cargo neon $ARGS tenant shard-split --shard-count=$FINAL_SHARDS
cargo neon $ARGS tenant status
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0004 --id=1
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0104 --id=2
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0204 --id=3
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0304 --id=4
cargo neon $ARGS tenant status
cargo neon $ARGS endpoint start ep-main

20
demo_split_8.sh Normal file
View File

@@ -0,0 +1,20 @@
FINAL_SHARDS=8
TENANT_ID=1f359dd625e519a1a4e8d7509690f6fc
ARGS="--release -q"
cargo neon $ARGS endpoint stop ep-main
cargo neon $ARGS tenant shard-split --shard-count=$FINAL_SHARDS
cargo neon $ARGS tenant status
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0008 --id=1
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0108 --id=2
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0208 --id=3
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0308 --id=4
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0408 --id=5
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0508 --id=6
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0608 --id=7
cargo neon $ARGS tenant migrate --tenant-id=$TENANT_ID-0708 --id=8
cargo neon $ARGS tenant status
cargo neon $ARGS endpoint start ep-main

View File

@@ -73,6 +73,9 @@ pub struct ComputeSpec {
// information about available remote extensions
pub remote_extensions: Option<RemoteExtSpec>,
// Stripe size for pageserver sharding, in pages
pub shard_stripe_size: Option<usize>,
}
/// Feature flag to signal `compute_ctl` to enable certain experimental functionality.

View File

@@ -18,7 +18,10 @@ use utils::{
lsn::Lsn,
};
use crate::{reltag::RelTag, shard::TenantShardId};
use crate::{
reltag::RelTag,
shard::{ShardCount, ShardStripeSize, TenantShardId},
};
use anyhow::bail;
use bytes::{Buf, BufMut, Bytes, BytesMut};
@@ -188,6 +191,41 @@ pub struct TimelineCreateRequest {
pub pg_version: Option<u32>,
}
#[derive(Serialize, Deserialize)]
pub struct TenantShardSplitRequest {
pub new_shard_count: u8,
}
#[derive(Serialize, Deserialize)]
pub struct TenantShardSplitResponse {
pub new_shards: Vec<TenantShardId>,
}
/// Parameters that apply to all shards in a tenant. Used during tenant creation.
#[derive(Serialize, Deserialize, Debug)]
#[serde(deny_unknown_fields)]
pub struct ShardParameters {
pub count: ShardCount,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub stripe_size: Option<ShardStripeSize>,
}
impl ShardParameters {
pub fn is_unsharded(&self) -> bool {
self.count == ShardCount(0)
}
}
impl Default for ShardParameters {
fn default() -> Self {
Self {
count: ShardCount(0),
stripe_size: None,
}
}
}
#[derive(Serialize, Deserialize, Debug)]
#[serde(deny_unknown_fields)]
pub struct TenantCreateRequest {
@@ -195,6 +233,12 @@ pub struct TenantCreateRequest {
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub generation: Option<u32>,
// If omitted, create a single shard with TenantShardId::unsharded()
#[serde(default)]
#[serde(skip_serializing_if = "ShardParameters::is_unsharded")]
pub shard_parameters: ShardParameters,
#[serde(flatten)]
pub config: TenantConfig, // as we have a flattened field, we should reject all unknown fields in it
}
@@ -217,7 +261,7 @@ impl std::ops::Deref for TenantCreateRequest {
/// An alternative representation of `pageserver::tenant::TenantConf` with
/// simpler types.
#[derive(Serialize, Deserialize, Debug, Default)]
#[derive(Serialize, Deserialize, Debug, Default, PartialEq, Eq, Clone)]
pub struct TenantConfig {
pub checkpoint_distance: Option<u64>,
pub checkpoint_timeout: Option<String>,
@@ -246,7 +290,7 @@ pub struct TenantConfig {
/// A flattened analog of a `pagesever::tenant::LocationMode`, which
/// lists out all possible states (and the virtual "Detached" state)
/// in a flat form rather than using rust-style enums.
#[derive(Serialize, Deserialize, Debug)]
#[derive(Serialize, Deserialize, Debug, PartialEq, Eq)]
pub enum LocationConfigMode {
AttachedSingle,
AttachedMulti,
@@ -255,14 +299,14 @@ pub enum LocationConfigMode {
Detached,
}
#[derive(Serialize, Deserialize, Debug)]
#[derive(Serialize, Deserialize, Debug, PartialEq, Eq)]
pub struct LocationConfigSecondary {
pub warm: bool,
}
/// An alternative representation of `pageserver::tenant::LocationConf`,
/// for use in external-facing APIs.
#[derive(Serialize, Deserialize, Debug)]
#[derive(Serialize, Deserialize, Debug, PartialEq, Eq)]
pub struct LocationConfig {
pub mode: LocationConfigMode,
/// If attaching, in what generation?
@@ -297,7 +341,7 @@ pub struct StatusResponse {
#[derive(Serialize, Deserialize, Debug)]
#[serde(deny_unknown_fields)]
pub struct TenantLocationConfigRequest {
pub tenant_id: TenantId,
pub tenant_shard_id: TenantShardId,
#[serde(flatten)]
pub config: LocationConfig, // as we have a flattened field, we should reject all unknown fields in it
}
@@ -671,6 +715,22 @@ pub struct PagestreamDbSizeResponse {
pub db_size: i64,
}
#[derive(Serialize, Deserialize, Debug)]
pub struct TenantPhysicalSizeResponse {
pub size: u64,
}
// XXX hack: this is a cut-down version of TenantHistorySize from the pageserver crate, omitting fields
// that require pageserver-internal types. It is sufficient to get the total size.
#[derive(Serialize, Deserialize, Debug)]
pub struct TenantHistorySize {
pub id: TenantId,
/// Size is a mixture of WAL and logical size, so the unit is bytes.
///
/// Will be none if `?inputs_only=true` was given.
pub size: Option<u64>,
}
impl PagestreamFeMessage {
pub fn serialize(&self) -> Bytes {
let mut bytes = BytesMut::new();

View File

@@ -1,6 +1,9 @@
use std::{ops::RangeInclusive, str::FromStr};
use crate::key::{is_rel_block_key, Key};
use crate::{
key::{is_rel_block_key, Key},
models::ShardParameters,
};
use hex::FromHex;
use serde::{Deserialize, Serialize};
use thiserror;
@@ -85,6 +88,16 @@ impl TenantShardId {
pub fn is_unsharded(&self) -> bool {
self.shard_number == ShardNumber(0) && self.shard_count == ShardCount(0)
}
/// Convenience for dropping the tenant_id and just getting the ShardIndex: this
/// is useful when logging from code that is already in a span that includes tenant ID, to
/// keep messages reasonably terse.
pub fn to_index(&self) -> ShardIndex {
ShardIndex {
shard_number: self.shard_number,
shard_count: self.shard_count,
}
}
}
/// Formatting helper
@@ -333,7 +346,7 @@ const DEFAULT_STRIPE_SIZE: ShardStripeSize = ShardStripeSize(256 * 1024 / 8);
pub struct ShardIdentity {
pub number: ShardNumber,
pub count: ShardCount,
stripe_size: ShardStripeSize,
pub stripe_size: ShardStripeSize,
layout: ShardLayout,
}
@@ -403,6 +416,17 @@ impl ShardIdentity {
}
}
/// For use when creating ShardIdentity instances for new shards, where a creation request
/// specifies the ShardParameters that apply to all shards.
pub fn from_params(number: ShardNumber, params: &ShardParameters) -> Self {
Self {
number,
count: params.count,
layout: LAYOUT_V1,
stripe_size: params.stripe_size.unwrap_or(DEFAULT_STRIPE_SIZE),
}
}
fn is_broken(&self) -> bool {
self.layout == LAYOUT_BROKEN
}

View File

@@ -1,4 +1,4 @@
use pageserver_api::models::*;
use pageserver_api::{models::*, shard::TenantShardId};
use reqwest::{IntoUrl, Method};
use utils::{
http::error::HttpErrorBody,
@@ -68,9 +68,9 @@ impl Client {
pub async fn tenant_details(
&self,
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
) -> Result<pageserver_api::models::TenantDetails> {
let uri = format!("{}/v1/tenant/{tenant_id}", self.mgmt_api_endpoint);
let uri = format!("{}/v1/tenant/{tenant_shard_id}", self.mgmt_api_endpoint);
self.get(uri)
.await?
.json()
@@ -80,9 +80,12 @@ impl Client {
pub async fn list_timelines(
&self,
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
) -> Result<Vec<pageserver_api::models::TimelineInfo>> {
let uri = format!("{}/v1/tenant/{tenant_id}/timeline", self.mgmt_api_endpoint);
let uri = format!(
"{}/v1/tenant/{tenant_shard_id}/timeline",
self.mgmt_api_endpoint
);
self.get(&uri)
.await?
.json()
@@ -166,14 +169,17 @@ impl Client {
pub async fn location_config(
&self,
tenant_id: TenantId,
tenant_shard_id: TenantShardId,
config: LocationConfig,
flush_ms: Option<std::time::Duration>,
) -> Result<()> {
let req_body = TenantLocationConfigRequest { tenant_id, config };
let req_body = TenantLocationConfigRequest {
tenant_shard_id,
config,
};
let path = format!(
"{}/v1/tenant/{}/location_config",
self.mgmt_api_endpoint, tenant_id
self.mgmt_api_endpoint, tenant_shard_id
);
let path = if let Some(flush_ms) = flush_ms {
format!("{}?flush_ms={}", path, flush_ms.as_millis())
@@ -199,4 +205,19 @@ impl Client {
.await
.map_err(Error::ReceiveBody)
}
pub async fn tenant_synthetic_size(
&self,
tenant_shard_id: TenantShardId,
) -> Result<TenantHistorySize> {
let uri = format!(
"{}/v1/tenant/{}/synthetic_size",
self.mgmt_api_endpoint, tenant_shard_id
);
self.get(&uri)
.await?
.json()
.await
.map_err(Error::ReceiveBody)
}
}

View File

@@ -2,6 +2,7 @@
use std::sync::Arc;
use pageserver_api::shard::TenantShardId;
use tokio::task::JoinSet;
use utils::id::{TenantId, TenantTimelineId};
@@ -31,7 +32,10 @@ pub async fn get_pageserver_tenant_timelines_unsharded(
async move {
(
tenant_id,
mgmt_api_client.tenant_details(tenant_id).await.unwrap(),
mgmt_api_client
.tenant_details(TenantShardId::unsharded(tenant_id))
.await
.unwrap(),
)
}
});

View File

@@ -15,10 +15,13 @@ use hyper::StatusCode;
use hyper::{Body, Request, Response, Uri};
use metrics::launch_timestamp::LaunchTimestamp;
use pageserver_api::models::TenantDetails;
use pageserver_api::models::TenantShardSplitRequest;
use pageserver_api::models::TenantShardSplitResponse;
use pageserver_api::models::{
DownloadRemoteLayersTaskSpawnRequest, LocationConfigMode, TenantAttachRequest,
TenantLoadRequest, TenantLocationConfigRequest,
};
use pageserver_api::shard::ShardCount;
use pageserver_api::shard::TenantShardId;
use remote_storage::GenericRemoteStorage;
use tenant_size_model::{SizeResult, StorageModel};
@@ -259,7 +262,7 @@ impl From<SetNewTenantConfigError> for ApiError {
SetNewTenantConfigError::GetTenant(tid) => {
ApiError::NotFound(anyhow!("tenant {}", tid).into())
}
e @ SetNewTenantConfigError::Persist(_) => {
e @ (SetNewTenantConfigError::Persist(_) | SetNewTenantConfigError::Other(_)) => {
ApiError::InternalServerError(anyhow::Error::new(e))
}
}
@@ -986,6 +989,25 @@ async fn tenant_size_handler(
)
}
async fn tenant_shard_split_handler(
mut request: Request<Body>,
_cancel: CancellationToken,
) -> Result<Response<Body>, ApiError> {
let req: TenantShardSplitRequest = json_request(&mut request).await?;
let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?;
let state = get_state(&request);
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn);
let new_shards = state
.tenant_manager
.shard_split(tenant_shard_id, ShardCount(req.new_shard_count), &ctx)
.await
.map_err(ApiError::InternalServerError)?;
json_response(StatusCode::OK, TenantShardSplitResponse { new_shards })
}
async fn layer_map_info_handler(
request: Request<Body>,
_cancel: CancellationToken,
@@ -1150,6 +1172,7 @@ async fn tenant_create_handler(
state.conf,
tenant_conf,
target_tenant_id,
request_data.shard_parameters,
generation,
state.tenant_resources(),
&ctx,
@@ -1823,6 +1846,9 @@ pub fn make_router(
.put("/v1/tenant/config", |r| {
api_handler(r, update_tenant_config_handler)
})
.put("/v1/tenant/:tenant_shard_id/shard_split", |r| {
api_handler(r, tenant_shard_split_handler)
})
.get("/v1/tenant/:tenant_shard_id/config", |r| {
api_handler(r, get_tenant_config_handler)
})

View File

@@ -405,13 +405,20 @@ impl PageServerHandler {
// shards (e.g. during splitting when the compute is not yet aware of the split), the tenant
// that we look up here may not be the one that serves all the actual requests: we will double
// check the mapping of key->shard later before calling into Timeline for getpage requests.
let tenant = mgr::get_active_tenant_with_timeout(
let tenant = match mgr::get_active_tenant_with_timeout(
tenant_id,
ShardSelector::First,
ACTIVE_TENANT_TIMEOUT,
&task_mgr::shutdown_token(),
)
.await?;
.await
{
Ok(t) => t,
Err(e) => {
tracing::warn!("Error at start of handle_pagerequests: {}", e);
return Err(e.into());
}
};
// Make request tracer if needed
let mut tracer = if tenant.get_trace_read_requests() {
@@ -426,9 +433,18 @@ impl PageServerHandler {
};
// Check that the timeline exists
let timeline = tenant
.get_timeline(timeline_id, true)
.map_err(|e| anyhow::anyhow!(e))?;
let timeline = match tenant.get_timeline(timeline_id, true) {
Ok(t) => t,
Err(e) => {
tracing::warn!("Error getting timeline: {}", e);
return Err(QueryError::Other(anyhow::anyhow!(e)));
}
};
tracing::info!(
"handle_pagerequests: got timeline {}",
timeline.tenant_shard_id
);
// Avoid starting new requests if the timeline has already started shutting down,
// and block timeline shutdown until this request is complete, or drops out due
@@ -815,6 +831,10 @@ impl PageServerHandler {
let key = rel_block_to_key(req.rel, req.blkno);
let page = if timeline.get_shard_identity().is_key_local(&key) {
tracing::debug!(
"handle_get_page_at_lsn: using shard {}",
timeline.tenant_shard_id
);
timeline
.get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest, ctx)
.await?
@@ -844,11 +864,18 @@ impl PageServerHandler {
// informed yet.
//
// https://github.com/neondatabase/neon/issues/6038
tracing::warn!("Page request routed to wrong shard: my identity {:?}, should go to shard{}, key {}",
timeline.get_shard_identity(), timeline.get_shard_identity().get_shard_number(&key).0, key);
return Err(anyhow::anyhow!("Request routed to wrong shard"));
}
Err(e) => return Err(e.into()),
};
tracing::debug!(
"handle_get_page_at_lsn: using shard {}",
timeline.tenant_shard_id
);
// Take a GateGuard for the duration of this request. If we were using our main Timeline object,
// the GateGuard was already held over the whole connection.
let _timeline_guard = timeline.gate.enter().map_err(|_| QueryError::Shutdown)?;

View File

@@ -17,6 +17,7 @@ use enumset::EnumSet;
use futures::stream::FuturesUnordered;
use futures::FutureExt;
use futures::StreamExt;
use pageserver_api::models::ShardParameters;
use pageserver_api::models::TimelineState;
use pageserver_api::shard::ShardIdentity;
use pageserver_api::shard::TenantShardId;
@@ -49,6 +50,7 @@ use self::metadata::TimelineMetadata;
use self::mgr::GetActiveTenantError;
use self::mgr::GetTenantError;
use self::mgr::TenantsMap;
use self::remote_timeline_client::upload::upload_index_part;
use self::remote_timeline_client::RemoteTimelineClient;
use self::timeline::uninit::TimelineExclusionError;
use self::timeline::uninit::TimelineUninitMark;
@@ -1531,6 +1533,7 @@ impl Tenant {
})?;
if active_only && !timeline.is_active() {
tracing::warn!("Timeline {} is not active", timeline.timeline_id);
Err(GetTimelineError::NotActive {
tenant_id: self.tenant_shard_id.tenant_id,
timeline_id,
@@ -2304,6 +2307,66 @@ impl Tenant {
pub(crate) fn get_generation(&self) -> Generation {
self.generation
}
pub(crate) async fn split_prepare(
&self,
child_shards: &Vec<TenantShardId>,
) -> anyhow::Result<()> {
let timelines = self.timelines.lock().unwrap().clone();
for timeline in timelines.values() {
let Some(tl_client) = &timeline.remote_client else {
anyhow::bail!("Remote storage is mandatory");
};
let Some(remote_storage) = &self.remote_storage else {
anyhow::bail!("Remote storage is mandatory");
};
// TODO: some higher level should enforce that timeline creation/deletion does not
// happen concurrently with splits. This is impossible to safely coordinate locally
// within one single pageserver's view of the world.
// Upload an index from the parent: this is partly to provide freshness for the
// child tenants that will copy it, and partly for general ease-of-debugging: there will
// always be a parent shard index in the same generation as we wrote the child shard index.
tl_client.schedule_index_upload_for_file_changes()?;
tl_client.wait_completion().await?;
// Shut down the timeline's remote client: this means that the indices we write
// for child shards will not be invalidated by the parent shard deleting layers.
tl_client.shutdown().await?;
// Download methods can still be used after shutdown, as they don't flow through the remote client's
// queue.
// TODO: create a way for remote timeline client to give us a copy of the last IndexPart it uploaded
// without having to download it again.
// TODO: carry a cancellation token in here
let result = tl_client
.download_index_file(CancellationToken::new())
.instrument(info_span!("download_index_file", tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug(), timeline_id=%timeline.timeline_id))
.await?;
let index_part = match result {
MaybeDeletedIndexPart::Deleted(_) => {
anyhow::bail!("Timeline deletion happened concurrently with split")
}
MaybeDeletedIndexPart::IndexPart(p) => p,
};
for child_shard in child_shards {
upload_index_part(
remote_storage,
child_shard,
&timeline.timeline_id,
self.generation,
&index_part,
&self.cancel,
)
.await?;
}
}
Ok(())
}
}
/// Given a Vec of timelines and their ancestors (timeline_id, ancestor_id),
@@ -2647,10 +2710,11 @@ impl Tenant {
}
}
// Legacy configs are implicitly in attached state
// Legacy configs are implicitly in attached state, and do not support sharding
Ok(LocationConf::attached_single(
tenant_conf,
Generation::none(),
&ShardParameters::default(),
))
} else {
// FIXME If the config file is not found, assume that we're attaching
@@ -3225,43 +3289,45 @@ impl Tenant {
// Upload the created data dir to S3
if let Some(storage) = &self.remote_storage {
let temp_path = timelines_path.join(format!(
"{INITDB_PATH}.upload-{timeline_id}.{TEMP_FILE_SUFFIX}"
));
if self.tenant_shard_id().is_zero() {
let temp_path = timelines_path.join(format!(
"{INITDB_PATH}.upload-{timeline_id}.{TEMP_FILE_SUFFIX}"
));
let (pgdata_zstd, tar_zst_size) =
import_datadir::create_tar_zst(&pgdata_path, &temp_path).await?;
backoff::retry(
|| async {
self::remote_timeline_client::upload_initdb_dir(
storage,
&self.tenant_shard_id.tenant_id,
&timeline_id,
pgdata_zstd.try_clone().await?,
tar_zst_size,
&self.cancel,
)
let (pgdata_zstd, tar_zst_size) =
import_datadir::create_tar_zst(&pgdata_path, &temp_path).await?;
backoff::retry(
|| async {
self::remote_timeline_client::upload_initdb_dir(
storage,
&self.tenant_shard_id.tenant_id,
&timeline_id,
pgdata_zstd.try_clone().await?,
tar_zst_size,
&self.cancel,
)
.await
},
|_| false,
3,
u32::MAX,
"persist_initdb_tar_zst",
backoff::Cancel::new(self.cancel.clone(), || anyhow::anyhow!("Cancelled")),
)
.await?;
tokio::fs::remove_file(&temp_path)
.await
},
|_| false,
3,
u32::MAX,
"persist_initdb_tar_zst",
backoff::Cancel::new(self.cancel.clone(), || anyhow::anyhow!("Cancelled")),
)
.await?;
tokio::fs::remove_file(&temp_path)
.await
.or_else(|e| {
if e.kind() == std::io::ErrorKind::NotFound {
// If something else already removed the file, ignore the error
Ok(())
} else {
Err(e)
}
})
.with_context(|| format!("tempfile removal {temp_path}"))?;
.or_else(|e| {
if e.kind() == std::io::ErrorKind::NotFound {
// If something else already removed the file, ignore the error
Ok(())
} else {
Err(e)
}
})
.with_context(|| format!("tempfile removal {temp_path}"))?;
}
}
}
let pgdata_lsn = import_datadir::get_lsn_from_controlfile(&pgdata_path)?.align();
@@ -3619,6 +3685,10 @@ impl Tenant {
Ok(())
}
pub(crate) fn get_tenant_conf(&self) -> TenantConfOpt {
self.tenant_conf.read().unwrap().tenant_conf
}
}
fn remove_timeline_and_uninit_mark(
@@ -4060,6 +4130,7 @@ pub(crate) mod harness {
AttachedTenantConf::try_from(LocationConf::attached_single(
TenantConfOpt::from(self.tenant_conf),
self.generation,
&ShardParameters::default(),
))
.unwrap(),
// This is a legacy/test code path: sharding isn't supported here.

View File

@@ -9,7 +9,7 @@
//! may lead to a data loss.
//!
use anyhow::bail;
use pageserver_api::models;
use pageserver_api::models::{self, ShardParameters};
use pageserver_api::shard::{ShardCount, ShardIdentity, ShardNumber, ShardStripeSize};
use serde::de::IntoDeserializer;
use serde::{Deserialize, Serialize};
@@ -24,7 +24,7 @@ pub mod defaults {
// which is good for now to trigger bugs.
// This parameter actually determines L0 layer file size.
pub const DEFAULT_CHECKPOINT_DISTANCE: u64 = 256 * 1024 * 1024;
pub const DEFAULT_CHECKPOINT_TIMEOUT: &str = "10 m";
pub const DEFAULT_CHECKPOINT_TIMEOUT: &str = "10 s";
// Target file size, when creating image and delta layers.
// This parameter determines L1 layer file size.
@@ -165,14 +165,17 @@ impl LocationConf {
/// For use when loading from a legacy configuration: presence of a tenant
/// implies it is in AttachmentMode::Single, which used to be the only
/// possible state. This function should eventually be removed.
pub(crate) fn attached_single(tenant_conf: TenantConfOpt, generation: Generation) -> Self {
pub(crate) fn attached_single(
tenant_conf: TenantConfOpt,
generation: Generation,
shard_params: &ShardParameters,
) -> Self {
Self {
mode: LocationMode::Attached(AttachedLocationConfig {
generation,
attach_mode: AttachmentMode::Single,
}),
// Legacy configuration loads are always from tenants created before sharding existed.
shard: ShardIdentity::unsharded(),
shard: ShardIdentity::from_params(ShardNumber(0), shard_params),
tenant_conf,
}
}

View File

@@ -2,8 +2,10 @@
//! page server.
use camino::{Utf8DirEntry, Utf8Path, Utf8PathBuf};
use itertools::Itertools;
use pageserver_api::key::Key;
use pageserver_api::shard::{ShardIdentity, ShardNumber, TenantShardId};
use pageserver_api::models::ShardParameters;
use pageserver_api::shard::{ShardCount, ShardIdentity, ShardNumber, TenantShardId};
use rand::{distributions::Alphanumeric, Rng};
use std::borrow::Cow;
use std::collections::{BTreeMap, HashMap};
@@ -20,7 +22,7 @@ use tokio_util::sync::CancellationToken;
use tracing::*;
use remote_storage::GenericRemoteStorage;
use utils::crashsafe;
use utils::{completion, crashsafe};
use crate::config::PageServerConf;
use crate::context::{DownloadBehavior, RequestContext};
@@ -130,7 +132,7 @@ impl TenantsMap {
/// A page service client sends a TenantId, and to look up the correct Tenant we must
/// resolve this to a fully qualified TenantShardId.
fn resolve_shard(
fn resolve_attached_shard(
&self,
tenant_id: &TenantId,
selector: ShardSelector,
@@ -140,25 +142,27 @@ impl TenantsMap {
TenantsMap::Initializing => None,
TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => {
for slot in m.range(TenantShardId::tenant_range(*tenant_id)) {
// Ignore all slots that don't contain an attached tenant
let tenant = match &slot.1 {
TenantSlot::Attached(t) => t,
_ => continue,
};
match selector {
ShardSelector::First => return Some(*slot.0),
ShardSelector::Zero if slot.0.shard_number == ShardNumber(0) => {
return Some(*slot.0)
}
ShardSelector::Page(key) => {
if let Some(tenant) = slot.1.get_attached() {
// First slot we see for this tenant, calculate the expected shard number
// for the key: we will use this for checking if this and subsequent
// slots contain the key, rather than recalculating the hash each time.
if want_shard.is_none() {
want_shard = Some(tenant.shard_identity.get_shard_number(&key));
}
// First slot we see for this tenant, calculate the expected shard number
// for the key: we will use this for checking if this and subsequent
// slots contain the key, rather than recalculating the hash each time.
if want_shard.is_none() {
want_shard = Some(tenant.shard_identity.get_shard_number(&key));
}
if Some(tenant.shard_identity.number) == want_shard {
return Some(*slot.0);
}
} else {
continue;
if Some(tenant.shard_identity.number) == want_shard {
return Some(*slot.0);
}
}
_ => continue,
@@ -616,8 +620,6 @@ pub(crate) async fn shutdown_all_tenants() {
}
async fn shutdown_all_tenants0(tenants: &std::sync::RwLock<TenantsMap>) {
use utils::completion;
let mut join_set = JoinSet::new();
// Atomically, 1. create the shutdown tasks and 2. prevent creation of new tenants.
@@ -743,13 +745,21 @@ pub(crate) async fn create_tenant(
conf: &'static PageServerConf,
tenant_conf: TenantConfOpt,
tenant_shard_id: TenantShardId,
shard_params: ShardParameters,
generation: Generation,
resources: TenantSharedResources,
ctx: &RequestContext,
) -> Result<Arc<Tenant>, TenantMapInsertError> {
let location_conf = LocationConf::attached_single(tenant_conf, generation);
let location_conf = LocationConf::attached_single(tenant_conf, generation, &shard_params);
info!("Creating tenant at location {location_conf:?}");
if shard_params.count != ShardCount(1) {
return Err(TenantMapInsertError::Other(anyhow::anyhow!(
"Only single-shard tenant creations may be serviced directly by a pageserver"
)));
}
let slot_guard =
tenant_map_acquire_slot(&tenant_shard_id, TenantSlotAcquireMode::MustNotExist)?;
let tenant_path = super::create_tenant_files(conf, &location_conf, &tenant_shard_id).await?;
@@ -784,6 +794,8 @@ pub(crate) enum SetNewTenantConfigError {
GetTenant(#[from] GetTenantError),
#[error(transparent)]
Persist(anyhow::Error),
#[error(transparent)]
Other(anyhow::Error),
}
pub(crate) async fn set_new_tenant_config(
@@ -797,10 +809,21 @@ pub(crate) async fn set_new_tenant_config(
info!("configuring tenant {tenant_id}");
let tenant = get_tenant(tenant_shard_id, true)?;
if tenant.tenant_shard_id().shard_count > ShardCount(0) {
// Note that we use ShardParameters::default below.
return Err(SetNewTenantConfigError::Other(anyhow::anyhow!(
"This API may only be used on single-sharded tenants, use the /location_config API for sharded tenants"
)));
}
// This is a legacy API that only operates on attached tenants: the preferred
// API to use is the location_config/ endpoint, which lets the caller provide
// the full LocationConf.
let location_conf = LocationConf::attached_single(new_tenant_conf, tenant.generation);
let location_conf = LocationConf::attached_single(
new_tenant_conf,
tenant.generation,
&ShardParameters::default(),
);
Tenant::persist_tenant_config(conf, &tenant_shard_id, &location_conf)
.await
@@ -1091,6 +1114,112 @@ impl TenantManager {
.collect(),
}
}
#[instrument(skip_all, fields(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), new_shard_count=%new_shard_count.0))]
pub(crate) async fn shard_split(
&self,
tenant_shard_id: TenantShardId,
new_shard_count: ShardCount,
ctx: &RequestContext,
) -> anyhow::Result<Vec<TenantShardId>> {
let tenant = get_tenant(tenant_shard_id, true)?;
// Plan: identify what the new child shards will be
let effective_old_shard_count = std::cmp::max(tenant_shard_id.shard_count.0, 1);
if new_shard_count <= ShardCount(effective_old_shard_count) {
anyhow::bail!("Requested shard count is not an increase");
}
let expansion_factor = new_shard_count.0 / effective_old_shard_count;
if expansion_factor & (expansion_factor - 1) != 0 {
anyhow::bail!("Requested split is not a power of two");
}
// Key mapping is based on a round robin mapping of key hash modulo shard count,
// so our child shards are the ones which the same keys would map to.
let mut child_shards = Vec::new();
for shard_number in 0..ShardNumber(new_shard_count.0).0 {
if shard_number % effective_old_shard_count == tenant_shard_id.shard_number.0 {
child_shards.push(TenantShardId {
tenant_id: tenant_shard_id.tenant_id,
shard_number: ShardNumber(shard_number),
shard_count: new_shard_count,
})
}
}
let parent_shard_identity = tenant.shard_identity;
let parent_tenant_conf = tenant.get_tenant_conf();
let parent_generation = tenant.generation;
// TODO: write a unit test for this
tracing::info!(
"Shard {} splits into: {}",
tenant_shard_id.to_index(),
child_shards
.iter()
.map(|id| format!("{}", id.to_index()))
.join(",")
);
// Phase 1: Write out child shards' remote index files, in the parent tenant's current generation
tenant.split_prepare(&child_shards).await?;
self.resources.deletion_queue_client.flush_advisory();
// Phase 2: Put the parent shard to InProgress and shut it down
drop(tenant);
let mut parent_slot_guard =
tenant_map_acquire_slot(&tenant_shard_id, TenantSlotAcquireMode::Any)?;
match parent_slot_guard.get_old_value() {
Some(TenantSlot::Attached(t)) => {
let (_guard, progress) = completion::channel();
match t.shutdown(progress, false).await {
Ok(()) => {}
Err(other) => {
other.wait().await;
}
}
}
Some(TenantSlot::Secondary) => {}
Some(TenantSlot::InProgress(_)) => {
unreachable!()
}
None => {
// We don't actually need the parent shard to still be attached to do our work, but it's
// a weird enough situation that the caller probably didn't want us to continue working
// if they had detached the tenant they requested the split on.
anyhow::bail!("Detached parent shard in the middle of split!")
}
};
parent_slot_guard.drop_old_value()?;
// TODO: hardlink layers from the parent into the child shard directories so that they don't immediately re-download
// TODO: erase the dentries from the parent
// Phase 3: Spawn the child shards
for child_shard in &child_shards {
let mut child_shard_identity = parent_shard_identity;
child_shard_identity.count = child_shard.shard_count;
child_shard_identity.number = child_shard.shard_number;
let child_location_conf = LocationConf {
mode: LocationMode::Attached(AttachedLocationConfig {
generation: parent_generation,
attach_mode: AttachmentMode::Single,
}),
shard: child_shard_identity,
tenant_conf: parent_tenant_conf,
};
self.upsert_location(*child_shard, child_location_conf, None, ctx)
.await?;
}
// Phase 4: Release the InProgress on the parent shard
drop(parent_slot_guard);
Ok(child_shards)
}
}
#[derive(Debug, thiserror::Error)]
@@ -1192,9 +1321,11 @@ pub(crate) async fn get_active_tenant_with_timeout(
let locked = TENANTS.read().unwrap();
// Resolve TenantId to TenantShardId
let tenant_shard_id = locked.resolve_shard(&tenant_id, shard_selector).ok_or(
GetActiveTenantError::NotFound(GetTenantError::NotFound(tenant_id)),
)?;
let tenant_shard_id = locked
.resolve_attached_shard(&tenant_id, shard_selector)
.ok_or(GetActiveTenantError::NotFound(GetTenantError::NotFound(
tenant_id,
)))?;
let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read)
.map_err(GetTenantError::MapState)?;
@@ -1530,10 +1661,11 @@ pub(crate) async fn attach_tenant(
) -> Result<(), TenantMapInsertError> {
// This is a legacy API (replaced by `/location_conf`). It does not support sharding
let tenant_shard_id = TenantShardId::unsharded(tenant_id);
let shard_params = ShardParameters::default();
let slot_guard =
tenant_map_acquire_slot(&tenant_shard_id, TenantSlotAcquireMode::MustNotExist)?;
let location_conf = LocationConf::attached_single(tenant_conf, generation);
let location_conf = LocationConf::attached_single(tenant_conf, generation, &shard_params);
let tenant_dir = create_tenant_files(conf, &location_conf, &tenant_shard_id).await?;
// TODO: tenant directory remains on disk if we bail out from here on.
// See https://github.com/neondatabase/neon/issues/4233
@@ -1976,8 +2108,6 @@ async fn remove_tenant_from_memory<V, F>(
where
F: std::future::Future<Output = anyhow::Result<V>>,
{
use utils::completion;
let mut slot_guard =
tenant_map_acquire_slot_impl(&tenant_shard_id, tenants, TenantSlotAcquireMode::MustExist)?;

View File

@@ -182,7 +182,7 @@
pub(crate) mod download;
pub mod index;
mod upload;
pub(crate) mod upload;
use anyhow::Context;
use camino::Utf8Path;
@@ -690,7 +690,10 @@ impl RemoteTimelineClient {
.insert(layer.layer_desc().filename(), metadata.clone());
upload_queue.latest_files_changes_since_metadata_upload_scheduled += 1;
info!("scheduled layer file upload {layer}");
info!(
"scheduled layer file upload {layer} gen={:?} shard={:?}",
metadata.generation, metadata.shard
);
let op = UploadOp::UploadLayer(layer, metadata);
self.calls_unfinished_metric_begin(&op);
upload_queue.queued_operations.push_back(op);
@@ -818,8 +821,25 @@ impl RemoteTimelineClient {
fn schedule_deletion_of_unlinked0(
self: &Arc<Self>,
upload_queue: &mut UploadQueueInitialized,
with_metadata: Vec<(LayerFileName, LayerFileMetadata)>,
mut with_metadata: Vec<(LayerFileName, LayerFileMetadata)>,
) {
// Filter out any layers which were not created by this tenant shard. These are
// layers that originate from some ancestor shard after a split, and may still
// be referenced by other shards. We are free to delete them locally and remove
// them from our index (and would have already done so when we reach this point
// in the code), but we may not delete them remotely.
with_metadata.retain(|(name, meta)| {
let retain = meta.shard.shard_number == self.tenant_shard_id.shard_number
&& meta.shard.shard_count == self.tenant_shard_id.shard_count;
if !retain {
tracing::debug!(
"Skipping deletion of ancestor-shard layer {name}, from shard {}",
meta.shard
);
}
retain
});
for (name, meta) in &with_metadata {
info!(
"scheduling deletion of layer {}{} (shard {})",

View File

@@ -25,7 +25,7 @@ use super::index::LayerFileMetadata;
use tracing::info;
/// Serializes and uploads the given index part data to the remote storage.
pub(super) async fn upload_index_part<'a>(
pub(crate) async fn upload_index_part<'a>(
storage: &'a GenericRemoteStorage,
tenant_shard_id: &TenantShardId,
timeline_id: &TimelineId,

View File

@@ -903,10 +903,15 @@ impl Timeline {
background_jobs_can_start: Option<&completion::Barrier>,
ctx: &RequestContext,
) {
tracing::info!("activate 1");
self.spawn_initial_logical_size_computation_task(ctx);
tracing::info!("activate 2");
self.launch_wal_receiver(ctx, broker_client);
tracing::info!("activate 3");
self.set_state(TimelineState::Active);
tracing::info!("activate 4");
self.launch_eviction_task(background_jobs_can_start);
tracing::info!("activate 5");
}
/// Graceful shutdown, may do a lot of I/O as we flush any open layers to disk and then

View File

@@ -308,13 +308,13 @@ lfc_change_limit_hook(int newval, void *extra)
Assert(victim->access_count == 0);
#ifdef FALLOC_FL_PUNCH_HOLE
if (fallocate(lfc_desc, FALLOC_FL_PUNCH_HOLE | FALLOC_FL_KEEP_SIZE, (off_t) victim->offset * BLOCKS_PER_CHUNK * BLCKSZ, BLOCKS_PER_CHUNK * BLCKSZ) < 0)
elog(LOG, "Failed to punch hole in file: %m");
neon_log(LOG, "Failed to punch hole in file: %m");
#endif
hash_search_with_hash_value(lfc_hash, &victim->key, victim->hash, HASH_REMOVE, NULL);
lfc_ctl->used -= 1;
}
lfc_ctl->limit = new_size;
elog(DEBUG1, "set local file cache limit to %d", new_size);
neon_log(DEBUG1, "set local file cache limit to %d", new_size);
LWLockRelease(lfc_lock);
}
@@ -327,7 +327,7 @@ lfc_init(void)
* shared_preload_libraries.
*/
if (!process_shared_preload_libraries_in_progress)
elog(ERROR, "Neon module should be loaded via shared_preload_libraries");
neon_log(ERROR, "Neon module should be loaded via shared_preload_libraries");
DefineCustomIntVariable("neon.max_file_cache_size",
@@ -643,7 +643,7 @@ lfc_write(NRelFileInfo rinfo, ForkNumber forkNum, BlockNumber blkno, const void
Assert(victim->access_count == 0);
entry->offset = victim->offset; /* grab victim's chunk */
hash_search_with_hash_value(lfc_hash, &victim->key, victim->hash, HASH_REMOVE, NULL);
elog(DEBUG2, "Swap file cache page");
neon_log(DEBUG2, "Swap file cache page");
}
else
{
@@ -846,10 +846,10 @@ local_cache_pages(PG_FUNCTION_ARGS)
* wrong) function definition though.
*/
if (get_call_result_type(fcinfo, NULL, &expected_tupledesc) != TYPEFUNC_COMPOSITE)
elog(ERROR, "return type must be a row type");
neon_log(ERROR, "return type must be a row type");
if (expected_tupledesc->natts != NUM_LOCALCACHE_PAGES_ELEM)
elog(ERROR, "incorrect number of output arguments");
neon_log(ERROR, "incorrect number of output arguments");
/* Construct a tuple descriptor for the result rows. */
tupledesc = CreateTemplateTupleDesc(expected_tupledesc->natts);

View File

@@ -15,6 +15,7 @@
#include "postgres.h"
#include "access/xlog.h"
#include "common/hashfn.h"
#include "fmgr.h"
#include "libpq-fe.h"
#include "libpq/libpq.h"
@@ -37,17 +38,6 @@
#define RECONNECT_INTERVAL_USEC 1000000
bool connected = false;
PGconn *pageserver_conn = NULL;
/*
* WaitEventSet containing:
* - WL_SOCKET_READABLE on pageserver_conn,
* - WL_LATCH_SET on MyLatch, and
* - WL_EXIT_ON_PM_DEATH.
*/
WaitEventSet *pageserver_conn_wes = NULL;
/* GUCs */
char *neon_timeline;
char *neon_tenant;
@@ -58,87 +48,206 @@ char *neon_auth_token;
int readahead_buffer_size = 128;
int flush_every_n_requests = 8;
static int n_reconnect_attempts = 0;
static int max_reconnect_attempts = 60;
static int n_reconnect_attempts = 0;
static int max_reconnect_attempts = 60;
static int stripe_size;
#define MAX_PAGESERVER_CONNSTRING_SIZE 256
bool (*old_redo_read_buffer_filter) (XLogReaderState *record, uint8 block_id) = NULL;
static bool pageserver_flush(shardno_t shard_no);
static void pageserver_disconnect(shardno_t shard_no);
static void AssignPageserverConnstring(const char *newval, void *extra);
static bool CheckPageserverConnstring(char **newval, void **extra, GucSource source);
static shmem_startup_hook_type prev_shmem_startup_hook;
#if PG_VERSION_NUM>=150000
static shmem_request_hook_type prev_shmem_request_hook;
#endif
/*
* ShardMap is kept in shared memory. It contains the connection strings for
* each shard.
*
* There is "neon.pageserver_connstring" GUC with PGC_SIGHUP option, allowing to change it using
* pg_reload_conf(). It is used by control plane to update shards information if page server is crashed,
* relocated or new shards are added. This GUC variable contains comma separated list of connection strings.
* It is copied to shared memory because config can not be loaded during query execution and we need to
* reestablish connection to page server.
*
* So usually copying connection string to shared memory is done by postmaster. And other backends
* should check update counter to determine of connection URL is changed and connection needs to be reestablished.
*
* But at startup shared memory is not yet initialized and so we need to copy in some other process.
* Moreover, we can not use standard Postgres LW-locks, because postmaster has proc entry and so can not wait
* on this primitive. This is why lockless access algorithm is implemented using two atomic counters to enforce
* consistent reading of connection string value from shared memory.
*/
typedef struct
{
size_t n_shards;
pg_atomic_uint64 begin_update_counter;
pg_atomic_uint64 end_update_counter;
char shard_connstr[MAX_SHARDS][MAX_PS_CONNSTR_LEN];
} ShardMap;
static ShardMap* shard_map;
static uint64 shard_map_update_counter;
typedef struct
{
LWLockId lock;
pg_atomic_uint64 update_counter;
char pageserver_connstring[MAX_PAGESERVER_CONNSTRING_SIZE];
} PagestoreShmemState;
/*
* Connection for each shard
*/
PGconn *conn;
/*
* WaitEventSet containing:
* - WL_SOCKET_READABLE on 'conn'
* - WL_LATCH_SET on MyLatch, and
* - WL_EXIT_ON_PM_DEATH.
*/
WaitEventSet *wes;
} PageServer;
#if PG_VERSION_NUM >= 150000
static shmem_request_hook_type prev_shmem_request_hook = NULL;
static void walproposer_shmem_request(void);
static PageServer page_servers[MAX_SHARDS];
static shardno_t max_attached_shard_no;
static void
psm_shmem_startup(void)
{
bool found;
if (prev_shmem_startup_hook)
{
prev_shmem_startup_hook();
}
LWLockAcquire(AddinShmemInitLock, LW_EXCLUSIVE);
shard_map = (ShardMap*)ShmemInitStruct("shard_map", sizeof(ShardMap), &found);
if (!found)
{
shard_map->n_shards = 0;
pg_atomic_init_u64(&shard_map->begin_update_counter, 0);
pg_atomic_init_u64(&shard_map->end_update_counter, 0);
AssignPageserverConnstring(page_server_connstring, NULL);
}
LWLockRelease(AddinShmemInitLock);
}
static void
psm_shmem_request(void)
{
#if PG_VERSION_NUM>=150000
if (prev_shmem_request_hook)
prev_shmem_request_hook();
#endif
static shmem_startup_hook_type prev_shmem_startup_hook;
static PagestoreShmemState *pagestore_shared;
static uint64 pagestore_local_counter = 0;
static char local_pageserver_connstring[MAX_PAGESERVER_CONNSTRING_SIZE];
static bool pageserver_flush(void);
static void pageserver_disconnect(void);
static bool
PagestoreShmemIsValid()
{
return pagestore_shared && UsedShmemSegAddr;
}
static bool
CheckPageserverConnstring(char **newval, void **extra, GucSource source)
{
return strlen(*newval) < MAX_PAGESERVER_CONNSTRING_SIZE;
RequestAddinShmemSpace(sizeof(ShardMap));
}
static void
AssignPageserverConnstring(const char *newval, void *extra)
psm_init(void)
{
if (!PagestoreShmemIsValid())
return;
LWLockAcquire(pagestore_shared->lock, LW_EXCLUSIVE);
strlcpy(pagestore_shared->pageserver_connstring, newval, MAX_PAGESERVER_CONNSTRING_SIZE);
pg_atomic_fetch_add_u64(&pagestore_shared->update_counter, 1);
LWLockRelease(pagestore_shared->lock);
prev_shmem_startup_hook = shmem_startup_hook;
shmem_startup_hook = psm_shmem_startup;
#if PG_VERSION_NUM>=150000
prev_shmem_request_hook = shmem_request_hook;
shmem_request_hook = psm_shmem_request;
#else
psm_shmem_request();
#endif
}
/*
* Reload page map if needed and return number of shards and connection string for the specified shard
* 'connstr' is an output buffer. If not NULL, it must point to a buffer at least MAX_PS_CONNSTR_LEN bytes
* long. The connection string for the gven shard is copied to it.
*/
static shardno_t
load_shard_map(shardno_t shard_no, char* connstr)
{
shardno_t n_shards;
uint64 begin_update_counter;
uint64 end_update_counter;
/*
* There is race condition here between backend and postmaster which can update shard map.
* We recheck update counter after copying shard map to check that configuration was not changed.
*/
do
{
begin_update_counter = pg_atomic_read_u64(&shard_map->begin_update_counter);
end_update_counter = pg_atomic_read_u64(&shard_map->end_update_counter);
n_shards = shard_map->n_shards;
if (shard_no >= n_shards)
neon_log(ERROR, "Shard %d is greater or equal than number of shards %d", shard_no, n_shards);
if (connstr)
{
/*
* We need to use strlcpy here because due to race condition string oin shared memory
* may be not zero terminated.
*/
strlcpy(connstr, shard_map->shard_connstr[shard_no], MAX_PS_CONNSTR_LEN);
pg_memory_barrier();
}
}
while (begin_update_counter != end_update_counter
|| begin_update_counter != pg_atomic_read_u64(&shard_map->begin_update_counter)
|| end_update_counter != pg_atomic_read_u64(&shard_map->end_update_counter));
if (shard_map_update_counter != end_update_counter)
{
/* Reset all connections if connection strings are changed */
for (shardno_t i = 0; i < max_attached_shard_no; i++)
{
if (page_servers[i].conn)
pageserver_disconnect(i);
}
max_attached_shard_no = 0;
shard_map_update_counter = end_update_counter;
}
return n_shards;
}
#define MB (1024*1024)
shardno_t
get_shard_number(BufferTag* tag)
{
shardno_t n_shards = load_shard_map(0, NULL);
uint32 hash;
#if PG_MAJORVERSION_NUM < 16
hash = murmurhash32(tag->rnode.relNode);
hash = hash_combine(hash, murmurhash32(tag->blockNum/stripe_size));
#else
hash = murmurhash32(tag->relNumber);
hash = hash_combine(hash, murmurhash32(tag->blockNum/stripe_size));
#endif
return hash % n_shards;
}
static bool
CheckConnstringUpdated()
{
if (!PagestoreShmemIsValid())
return false;
return pagestore_local_counter < pg_atomic_read_u64(&pagestore_shared->update_counter);
}
static void
ReloadConnstring()
{
if (!PagestoreShmemIsValid())
return;
LWLockAcquire(pagestore_shared->lock, LW_SHARED);
strlcpy(local_pageserver_connstring, pagestore_shared->pageserver_connstring, sizeof(local_pageserver_connstring));
pagestore_local_counter = pg_atomic_read_u64(&pagestore_shared->update_counter);
LWLockRelease(pagestore_shared->lock);
}
static bool
pageserver_connect(int elevel)
pageserver_connect(shardno_t shard_no, int elevel)
{
char *query;
int ret;
const char *keywords[3];
const char *values[3];
int n;
PGconn* conn;
WaitEventSet *wes;
char connstr[MAX_PS_CONNSTR_LEN];
Assert(!connected);
Assert(page_servers[shard_no].conn == NULL);
if (CheckConnstringUpdated())
{
ReloadConnstring();
}
(void)load_shard_map(shard_no, connstr); /* refresh page map if needed */
/*
* Connect using the connection string we got from the
@@ -158,50 +267,47 @@ pageserver_connect(int elevel)
n++;
}
keywords[n] = "dbname";
values[n] = local_pageserver_connstring;
values[n] = connstr;
n++;
keywords[n] = NULL;
values[n] = NULL;
n++;
pageserver_conn = PQconnectdbParams(keywords, values, 1);
conn = PQconnectdbParams(keywords, values, 1);
if (PQstatus(pageserver_conn) == CONNECTION_BAD)
if (PQstatus(conn) == CONNECTION_BAD)
{
char *msg = pchomp(PQerrorMessage(pageserver_conn));
char *msg = pchomp(PQerrorMessage(conn));
PQfinish(pageserver_conn);
pageserver_conn = NULL;
PQfinish(conn);
ereport(elevel,
(errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION),
errmsg(NEON_TAG "could not establish connection to pageserver"),
errmsg(NEON_TAG "[shard %d] could not establish connection to pageserver", shard_no),
errdetail_internal("%s", msg)));
return false;
}
query = psprintf("pagestream %s %s", neon_tenant, neon_timeline);
ret = PQsendQuery(pageserver_conn, query);
ret = PQsendQuery(conn, query);
if (ret != 1)
{
PQfinish(pageserver_conn);
pageserver_conn = NULL;
neon_log(elevel, "could not send pagestream command to pageserver");
PQfinish(conn);
neon_shard_log(shard_no, elevel, "could not send pagestream command to pageserver");
return false;
}
pageserver_conn_wes = CreateWaitEventSet(TopMemoryContext, 3);
AddWaitEventToSet(pageserver_conn_wes, WL_LATCH_SET, PGINVALID_SOCKET,
wes = CreateWaitEventSet(TopMemoryContext, 3);
AddWaitEventToSet(wes, WL_LATCH_SET, PGINVALID_SOCKET,
MyLatch, NULL);
AddWaitEventToSet(pageserver_conn_wes, WL_EXIT_ON_PM_DEATH, PGINVALID_SOCKET,
AddWaitEventToSet(wes, WL_EXIT_ON_PM_DEATH, PGINVALID_SOCKET,
NULL, NULL);
AddWaitEventToSet(pageserver_conn_wes, WL_SOCKET_READABLE, PQsocket(pageserver_conn), NULL, NULL);
AddWaitEventToSet(wes, WL_SOCKET_READABLE, PQsocket(conn), NULL, NULL);
while (PQisBusy(pageserver_conn))
while (PQisBusy(conn))
{
WaitEvent event;
/* Sleep until there's something to do */
(void) WaitEventSetWait(pageserver_conn_wes, -1L, &event, 1, PG_WAIT_EXTENSION);
(void) WaitEventSetWait(wes, -1L, &event, 1, PG_WAIT_EXTENSION);
ResetLatch(MyLatch);
CHECK_FOR_INTERRUPTS();
@@ -209,25 +315,25 @@ pageserver_connect(int elevel)
/* Data available in socket? */
if (event.events & WL_SOCKET_READABLE)
{
if (!PQconsumeInput(pageserver_conn))
if (!PQconsumeInput(conn))
{
char *msg = pchomp(PQerrorMessage(pageserver_conn));
char *msg = pchomp(PQerrorMessage(conn));
PQfinish(pageserver_conn);
pageserver_conn = NULL;
FreeWaitEventSet(pageserver_conn_wes);
pageserver_conn_wes = NULL;
PQfinish(conn);
FreeWaitEventSet(wes);
neon_log(elevel, "could not complete handshake with pageserver: %s",
msg);
neon_shard_log(shard_no, elevel, "could not complete handshake with pageserver: %s",
msg);
return false;
}
}
}
neon_log(LOG, "libpagestore: connected to '%s'", page_server_connstring);
neon_shard_log(shard_no, LOG, "libpagestore: connected to '%s'", connstr);
page_servers[shard_no].conn = conn;
page_servers[shard_no].wes = wes;
max_attached_shard_no = Max(shard_no+1, max_attached_shard_no);
connected = true;
return true;
}
@@ -235,10 +341,10 @@ pageserver_connect(int elevel)
* A wrapper around PQgetCopyData that checks for interrupts while sleeping.
*/
static int
call_PQgetCopyData(char **buffer)
call_PQgetCopyData(shardno_t shard_no, char **buffer)
{
int ret;
PGconn* pageserver_conn = page_servers[shard_no].conn;
retry:
ret = PQgetCopyData(pageserver_conn, buffer, 1 /* async */ );
@@ -247,7 +353,7 @@ retry:
WaitEvent event;
/* Sleep until there's something to do */
(void) WaitEventSetWait(pageserver_conn_wes, -1L, &event, 1, PG_WAIT_EXTENSION);
(void) WaitEventSetWait(page_servers[shard_no].wes, -1L, &event, 1, PG_WAIT_EXTENSION);
ResetLatch(MyLatch);
CHECK_FOR_INTERRUPTS();
@@ -259,7 +365,7 @@ retry:
{
char *msg = pchomp(PQerrorMessage(pageserver_conn));
neon_log(LOG, "could not get response from pageserver: %s", msg);
neon_shard_log(shard_no, LOG, "could not get response from pageserver: %s", msg);
pfree(msg);
return -1;
}
@@ -273,7 +379,7 @@ retry:
static void
pageserver_disconnect(void)
pageserver_disconnect(shardno_t shard_no)
{
/*
* If anything goes wrong while we were sending a request, it's not clear
@@ -282,38 +388,32 @@ pageserver_disconnect(void)
* time later after we have already sent a new unrelated request. Close
* the connection to avoid getting confused.
*/
if (connected)
if (page_servers[shard_no].conn)
{
neon_log(LOG, "dropping connection to page server due to error");
PQfinish(pageserver_conn);
pageserver_conn = NULL;
connected = false;
neon_shard_log(shard_no, LOG, "dropping connection to page server due to error");
PQfinish(page_servers[shard_no].conn);
page_servers[shard_no].conn = NULL;
prefetch_on_ps_disconnect();
}
if (pageserver_conn_wes != NULL)
if (page_servers[shard_no].wes != NULL)
{
FreeWaitEventSet(pageserver_conn_wes);
pageserver_conn_wes = NULL;
FreeWaitEventSet(page_servers[shard_no].wes);
page_servers[shard_no].wes = NULL;
}
}
static bool
pageserver_send(NeonRequest *request)
pageserver_send(shardno_t shard_no, NeonRequest *request)
{
StringInfoData req_buff;
if (CheckConnstringUpdated())
{
pageserver_disconnect();
ReloadConnstring();
}
PGconn* pageserver_conn = page_servers[shard_no].conn;
/* If the connection was lost for some reason, reconnect */
if (connected && PQstatus(pageserver_conn) == CONNECTION_BAD)
if (pageserver_conn && PQstatus(pageserver_conn) == CONNECTION_BAD)
{
neon_log(LOG, "pageserver_send disconnect bad connection");
pageserver_disconnect();
neon_shard_log(shard_no, LOG, "pageserver_send disconnect bad connection");
pageserver_disconnect(shard_no);
}
req_buff = nm_pack_request(request);
@@ -327,9 +427,9 @@ pageserver_send(NeonRequest *request)
* https://github.com/neondatabase/neon/issues/1138 So try to reestablish
* connection in case of failure.
*/
if (!connected)
if (!page_servers[shard_no].conn)
{
while (!pageserver_connect(n_reconnect_attempts < max_reconnect_attempts ? LOG : ERROR))
while (!pageserver_connect(shard_no, n_reconnect_attempts < max_reconnect_attempts ? LOG : ERROR))
{
HandleMainLoopInterrupts();
n_reconnect_attempts += 1;
@@ -338,7 +438,9 @@ pageserver_send(NeonRequest *request)
n_reconnect_attempts = 0;
}
/*
pageserver_conn = page_servers[shard_no].conn;
/*
* Send request.
*
* In principle, this could block if the output buffer is full, and we
@@ -349,9 +451,8 @@ pageserver_send(NeonRequest *request)
if (PQputCopyData(pageserver_conn, req_buff.data, req_buff.len) <= 0)
{
char *msg = pchomp(PQerrorMessage(pageserver_conn));
pageserver_disconnect();
neon_log(LOG, "pageserver_send disconnect because failed to send page request (try to reconnect): %s", msg);
pageserver_disconnect(shard_no);
neon_shard_log(shard_no, LOG, "pageserver_send disconnect because failed to send page request (try to reconnect): %s", msg);
pfree(msg);
pfree(req_buff.data);
return false;
@@ -363,19 +464,19 @@ pageserver_send(NeonRequest *request)
{
char *msg = nm_to_string((NeonMessage *) request);
neon_log(PageStoreTrace, "sent request: %s", msg);
neon_shard_log(shard_no, PageStoreTrace, "sent request: %s", msg);
pfree(msg);
}
return true;
}
static NeonResponse *
pageserver_receive(void)
pageserver_receive(shardno_t shard_no)
{
StringInfoData resp_buff;
NeonResponse *resp;
if (!connected)
PGconn* pageserver_conn = page_servers[shard_no].conn;
if (!pageserver_conn)
return NULL;
PG_TRY();
@@ -383,7 +484,7 @@ pageserver_receive(void)
/* read response */
int rc;
rc = call_PQgetCopyData(&resp_buff.data);
rc = call_PQgetCopyData(shard_no, &resp_buff.data);
if (rc >= 0)
{
resp_buff.len = rc;
@@ -395,33 +496,33 @@ pageserver_receive(void)
{
char *msg = nm_to_string((NeonMessage *) resp);
neon_log(PageStoreTrace, "got response: %s", msg);
neon_shard_log(shard_no, PageStoreTrace, "got response: %s", msg);
pfree(msg);
}
}
else if (rc == -1)
{
neon_log(LOG, "pageserver_receive disconnect because call_PQgetCopyData returns -1: %s", pchomp(PQerrorMessage(pageserver_conn)));
pageserver_disconnect();
neon_shard_log(shard_no, LOG, "pageserver_receive disconnect because call_PQgetCopyData returns -1: %s", pchomp(PQerrorMessage(pageserver_conn)));
pageserver_disconnect(shard_no);
resp = NULL;
}
else if (rc == -2)
{
char *msg = pchomp(PQerrorMessage(pageserver_conn));
pageserver_disconnect();
neon_log(ERROR, "pageserver_receive disconnect because could not read COPY data: %s", msg);
pageserver_disconnect(shard_no);
neon_shard_log(shard_no, ERROR, "pageserver_receive disconnect because could not read COPY data: %s", msg);
}
else
{
pageserver_disconnect();
neon_log(ERROR, "pageserver_receive disconnect because unexpected PQgetCopyData return value: %d", rc);
pageserver_disconnect(shard_no);
neon_shard_log(shard_no, ERROR, "pageserver_receive disconnect because unexpected PQgetCopyData return value: %d", rc);
}
}
PG_CATCH();
{
neon_log(LOG, "pageserver_receive disconnect due to caught exception");
pageserver_disconnect();
neon_shard_log(shard_no, LOG, "pageserver_receive disconnect due to caught exception");
pageserver_disconnect(shard_no);
PG_RE_THROW();
}
PG_END_TRY();
@@ -431,11 +532,12 @@ pageserver_receive(void)
static bool
pageserver_flush(void)
pageserver_flush(shardno_t shard_no)
{
if (!connected)
PGconn* pageserver_conn = page_servers[shard_no].conn;
if (!pageserver_conn)
{
neon_log(WARNING, "Tried to flush while disconnected");
neon_shard_log(shard_no, WARNING, "Tried to flush while disconnected");
}
else
{
@@ -443,8 +545,8 @@ pageserver_flush(void)
{
char *msg = pchomp(PQerrorMessage(pageserver_conn));
pageserver_disconnect();
neon_log(LOG, "pageserver_flush disconnect because failed to flush page requests: %s", msg);
pageserver_disconnect(shard_no);
neon_shard_log(shard_no, LOG, "pageserver_flush disconnect because failed to flush page requests: %s", msg);
pfree(msg);
return false;
}
@@ -467,63 +569,83 @@ check_neon_id(char **newval, void **extra, GucSource source)
return **newval == '\0' || HexDecodeString(id, *newval, 16);
}
static Size
PagestoreShmemSize(void)
{
return sizeof(PagestoreShmemState);
}
static bool
PagestoreShmemInit(void)
CheckPageserverConnstring(char **newval, void **extra, GucSource source)
{
bool found;
LWLockAcquire(AddinShmemInitLock, LW_EXCLUSIVE);
pagestore_shared = ShmemInitStruct("libpagestore shared state",
PagestoreShmemSize(),
&found);
if (!found)
const char* shard_connstr = *newval;
const char* sep;
size_t connstr_len;
int i = 0;
do
{
pagestore_shared->lock = &(GetNamedLWLockTranche("neon_libpagestore")->lock);
pg_atomic_init_u64(&pagestore_shared->update_counter, 0);
AssignPageserverConnstring(page_server_connstring, NULL);
sep = strchr(shard_connstr, ',');
connstr_len = sep != NULL ? sep - shard_connstr : strlen(shard_connstr);
if (connstr_len == 0)
break; /* trailing comma */
if (i >= MAX_SHARDS)
{
neon_log(LOG, "Too many shards");
return false;
}
if (connstr_len >= MAX_PS_CONNSTR_LEN)
{
neon_log(LOG, "Connection string too long");
return false;
}
shard_connstr = sep + 1;
i += 1;
} while (sep != NULL);
return true;
}
static void
AssignPageserverConnstring(const char *newval, void *extra)
{
/*
* Load shard map only at Postmaster.
* If old page server is not available, then backends can be blocked in attempts to reconnect to it and do not reload config in this loop
*
* Copying GUC value to shared memory is usually performed by postmaster. But in case of startup,
* shared memory is not yet initialized. So it has to be performed by any other process.
* It is not a problem if more than one process do this initialization.
*/
if (shard_map != NULL && UsedShmemSegAddr != NULL && (MyProcPid == PostmasterPid || shard_map->n_shards == 0))
{
const char* shard_connstr = newval;
const char* sep;
size_t connstr_len;
int i = 0;
bool shard_map_changed = false;
do
{
sep = strchr(shard_connstr, ',');
connstr_len = sep != NULL ? sep - shard_connstr : strlen(shard_connstr);
if (connstr_len == 0)
break; /* trailing comma */
Assert(i < MAX_SHARDS);
Assert(connstr_len < MAX_PS_CONNSTR_LEN);
if (i >= shard_map->n_shards ||
strcmp(shard_map->shard_connstr[i], shard_connstr) != 0)
{
if (!shard_map_changed)
{
pg_atomic_add_fetch_u64(&shard_map->begin_update_counter, 1);
shard_map_changed = true;
}
memcpy(shard_map->shard_connstr[i], shard_connstr, connstr_len+1);
}
shard_connstr = sep + 1;
i += 1;
} while (sep != NULL);
if (shard_map_changed)
{
shard_map->n_shards = i;
pg_memory_barrier();
pg_atomic_add_fetch_u64(&shard_map->end_update_counter, 1);
}
}
LWLockRelease(AddinShmemInitLock);
return found;
}
static void
pagestore_shmem_startup_hook(void)
{
if (prev_shmem_startup_hook)
prev_shmem_startup_hook();
PagestoreShmemInit();
}
static void
pagestore_shmem_request(void)
{
#if PG_VERSION_NUM >= 150000
if (prev_shmem_request_hook)
prev_shmem_request_hook();
#endif
RequestAddinShmemSpace(PagestoreShmemSize());
RequestNamedLWLockTranche("neon_libpagestore", 1);
}
static void
pagestore_prepare_shmem(void)
{
#if PG_VERSION_NUM >= 150000
prev_shmem_request_hook = shmem_request_hook;
shmem_request_hook = pagestore_shmem_request;
#else
pagestore_shmem_request();
#endif
prev_shmem_startup_hook = shmem_startup_hook;
shmem_startup_hook = pagestore_shmem_startup_hook;
}
/*
@@ -532,8 +654,6 @@ pagestore_prepare_shmem(void)
void
pg_init_libpagestore(void)
{
pagestore_prepare_shmem();
DefineCustomStringVariable("neon.pageserver_connstring",
"connection string to the page server",
NULL,
@@ -561,6 +681,15 @@ pg_init_libpagestore(void)
0, /* no flags required */
check_neon_id, NULL, NULL);
DefineCustomIntVariable("neon.stripe_size",
"sharding stripe size",
NULL,
&stripe_size,
32768, 1, INT_MAX,
PGC_SIGHUP,
GUC_UNIT_BLOCKS,
NULL, NULL, NULL);
DefineCustomIntVariable("neon.max_cluster_size",
"cluster size limit",
NULL,
@@ -624,4 +753,5 @@ pg_init_libpagestore(void)
}
lfc_init();
psm_init();
}

View File

@@ -17,12 +17,20 @@
#include "access/xlogdefs.h"
#include RELFILEINFO_HDR
#include "storage/block.h"
#include "storage/smgr.h"
#include "storage/buf_internals.h"
#include "lib/stringinfo.h"
#include "libpq/pqformat.h"
#include "storage/block.h"
#include "storage/smgr.h"
#include "utils/memutils.h"
#include "pg_config.h"
#define MAX_SHARDS 128
#define MAX_PS_CONNSTR_LEN 128
typedef enum
{
/* pagestore_client -> pagestore */
@@ -51,6 +59,9 @@ typedef struct
#define neon_log(tag, fmt, ...) ereport(tag, \
(errmsg(NEON_TAG fmt, ##__VA_ARGS__), \
errhidestmt(true), errhidecontext(true), errposition(0), internalerrposition(0)))
#define neon_shard_log(shard_no, tag, fmt, ...) ereport(tag, \
(errmsg(NEON_TAG "[shard %d] " fmt, shard_no, ##__VA_ARGS__), \
errhidestmt(true), errhidecontext(true), errposition(0), internalerrposition(0)))
/*
* supertype of all the Neon*Request structs below
@@ -141,11 +152,13 @@ extern char *nm_to_string(NeonMessage *msg);
* API
*/
typedef unsigned shardno_t;
typedef struct
{
bool (*send) (NeonRequest *request);
NeonResponse *(*receive) (void);
bool (*flush) (void);
bool (*send) (shardno_t shard_no, NeonRequest * request);
NeonResponse *(*receive) (shardno_t shard_no);
bool (*flush) (shardno_t shard_no);
} page_server_api;
extern void prefetch_on_ps_disconnect(void);
@@ -159,6 +172,8 @@ extern char *neon_timeline;
extern char *neon_tenant;
extern int32 max_cluster_size;
extern shardno_t get_shard_number(BufferTag* tag);
extern const f_smgr *smgr_neon(BackendId backend, NRelFileInfo rinfo);
extern void smgr_init_neon(void);
extern void readahead_buffer_resize(int newsize, void *extra);

View File

@@ -172,6 +172,7 @@ typedef struct PrefetchRequest
XLogRecPtr actual_request_lsn;
NeonResponse *response; /* may be null */
PrefetchStatus status;
shardno_t shard_no;
uint64 my_ring_index;
} PrefetchRequest;
@@ -239,7 +240,9 @@ typedef struct PrefetchState
* also unused */
/* the buffers */
prfh_hash *prf_hash;
prfh_hash *prf_hash;
int max_shard_no;
uint8 shard_bitmap[(MAX_SHARDS + 7)/8];
PrefetchRequest prf_buffer[]; /* prefetch buffers */
} PrefetchState;
@@ -327,6 +330,7 @@ compact_prefetch_buffers(void)
Assert(target_slot->status == PRFS_UNUSED);
target_slot->buftag = source_slot->buftag;
target_slot->shard_no = source_slot->shard_no;
target_slot->status = source_slot->status;
target_slot->response = source_slot->response;
target_slot->effective_request_lsn = source_slot->effective_request_lsn;
@@ -494,6 +498,23 @@ prefetch_cleanup_trailing_unused(void)
}
}
static bool
prefetch_flush_requests(void)
{
for (shardno_t shard_no = 0; shard_no < MyPState->max_shard_no; shard_no++)
{
if (MyPState->shard_bitmap[shard_no >> 3] & (1 << (shard_no & 7)))
{
if (!page_server->flush(shard_no))
return false;
MyPState->shard_bitmap[shard_no >> 3] &= ~(1 << (shard_no & 7));
}
}
MyPState->max_shard_no = 0;
return true;
}
/*
* Wait for slot of ring_index to have received its response.
* The caller is responsible for making sure the request buffer is flushed.
@@ -509,7 +530,7 @@ prefetch_wait_for(uint64 ring_index)
if (MyPState->ring_flush <= ring_index &&
MyPState->ring_unused > MyPState->ring_flush)
{
if (!page_server->flush())
if (!prefetch_flush_requests())
return false;
MyPState->ring_flush = MyPState->ring_unused;
}
@@ -547,7 +568,7 @@ prefetch_read(PrefetchRequest *slot)
Assert(slot->my_ring_index == MyPState->ring_receive);
old = MemoryContextSwitchTo(MyPState->errctx);
response = (NeonResponse *) page_server->receive();
response = (NeonResponse *) page_server->receive(slot->shard_no);
MemoryContextSwitchTo(old);
if (response)
{
@@ -704,12 +725,14 @@ prefetch_do_request(PrefetchRequest *slot, bool *force_latest, XLogRecPtr *force
Assert(slot->response == NULL);
Assert(slot->my_ring_index == MyPState->ring_unused);
while (!page_server->send((NeonRequest *) &request));
while (!page_server->send(slot->shard_no, (NeonRequest *) &request));
/* update prefetch state */
MyPState->n_requests_inflight += 1;
MyPState->n_unused -= 1;
MyPState->ring_unused += 1;
MyPState->shard_bitmap[slot->shard_no >> 3] |= 1 << (slot->shard_no & 7);
MyPState->max_shard_no = Max(slot->shard_no+1, MyPState->max_shard_no);
/* update slot state */
slot->status = PRFS_REQUESTED;
@@ -880,6 +903,7 @@ Retry:
* function reads the buffer tag from the slot.
*/
slot->buftag = tag;
slot->shard_no = get_shard_number(&tag);
slot->my_ring_index = ring_index;
prefetch_do_request(slot, force_latest, force_lsn);
@@ -890,7 +914,7 @@ Retry:
if (flush_every_n_requests > 0 &&
MyPState->ring_unused - MyPState->ring_flush >= flush_every_n_requests)
{
if (!page_server->flush())
if (!prefetch_flush_requests())
{
/*
* Prefetch set is reset in case of error, so we should try to
@@ -908,13 +932,44 @@ static NeonResponse *
page_server_request(void const *req)
{
NeonResponse *resp;
BufferTag tag = {0};
shardno_t shard_no;
switch (((NeonRequest *) req)->tag)
{
case T_NeonExistsRequest:
CopyNRelFileInfoToBufTag(tag, ((NeonExistsRequest *) req)->rinfo);
break;
case T_NeonNblocksRequest:
CopyNRelFileInfoToBufTag(tag, ((NeonNblocksRequest *) req)->rinfo);
break;
case T_NeonDbSizeRequest:
NInfoGetDbOid(BufTagGetNRelFileInfo(tag)) = ((NeonDbSizeRequest *) req)->dbNode;
break;
case T_NeonGetPageRequest:
CopyNRelFileInfoToBufTag(tag, ((NeonGetPageRequest *) req)->rinfo);
tag.blockNum = ((NeonGetPageRequest *) req)->blkno;
break;
default:
neon_log(ERROR, "Unexpected request tag: %d", ((NeonRequest *) req)->tag);
}
shard_no = get_shard_number(&tag);
/*
* Current sharding model assumes that all metadata is present only at shard 0.
* We still need to call get_shard_no() to check if shard map is up-to-date.
*/
if (((NeonRequest *) req)->tag != T_NeonGetPageRequest || ((NeonGetPageRequest *) req)->forknum != MAIN_FORKNUM)
{
shard_no = 0;
}
do
{
while (!page_server->send((NeonRequest *) req) || !page_server->flush());
MyPState->ring_flush = MyPState->ring_unused;
while (!page_server->send(shard_no, (NeonRequest *) req) || !page_server->flush(shard_no));
consume_prefetch_responses();
resp = page_server->receive();
resp = page_server->receive(shard_no);
} while (resp == NULL);
return resp;
@@ -990,7 +1045,7 @@ nm_pack_request(NeonRequest *msg)
case T_NeonErrorResponse:
case T_NeonDbSizeResponse:
default:
elog(ERROR, "unexpected neon message tag 0x%02x", msg->tag);
neon_log(ERROR, "unexpected neon message tag 0x%02x", msg->tag);
break;
}
return s;
@@ -1085,7 +1140,7 @@ nm_unpack_response(StringInfo s)
case T_NeonGetPageRequest:
case T_NeonDbSizeRequest:
default:
elog(ERROR, "unexpected neon message tag 0x%02x", tag);
neon_log(ERROR, "unexpected neon message tag 0x%02x", tag);
break;
}
@@ -1277,7 +1332,7 @@ neon_wallog_page(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, co
XLogFlush(recptr);
lsn = recptr;
ereport(SmgrTrace,
(errmsg("Page %u of relation %u/%u/%u.%u was force logged. Evicted at lsn=%X/%X",
(errmsg(NEON_TAG "Page %u of relation %u/%u/%u.%u was force logged. Evicted at lsn=%X/%X",
blocknum,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum, LSN_FORMAT_ARGS(lsn))));
@@ -1305,7 +1360,7 @@ neon_wallog_page(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, co
if (PageIsNew((Page) buffer))
{
ereport(SmgrTrace,
(errmsg("Page %u of relation %u/%u/%u.%u is all-zeros",
(errmsg(NEON_TAG "Page %u of relation %u/%u/%u.%u is all-zeros",
blocknum,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum)));
@@ -1313,7 +1368,7 @@ neon_wallog_page(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, co
else if (PageIsEmptyHeapPage((Page) buffer))
{
ereport(SmgrTrace,
(errmsg("Page %u of relation %u/%u/%u.%u is an empty heap page with no LSN",
(errmsg(NEON_TAG "Page %u of relation %u/%u/%u.%u is an empty heap page with no LSN",
blocknum,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum)));
@@ -1321,7 +1376,7 @@ neon_wallog_page(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, co
else
{
ereport(PANIC,
(errmsg("Page %u of relation %u/%u/%u.%u is evicted with zero LSN",
(errmsg(NEON_TAG "Page %u of relation %u/%u/%u.%u is evicted with zero LSN",
blocknum,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum)));
@@ -1330,7 +1385,7 @@ neon_wallog_page(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, co
else
{
ereport(SmgrTrace,
(errmsg("Page %u of relation %u/%u/%u.%u is already wal logged at lsn=%X/%X",
(errmsg(NEON_TAG "Page %u of relation %u/%u/%u.%u is already wal logged at lsn=%X/%X",
blocknum,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum, LSN_FORMAT_ARGS(lsn))));
@@ -1430,7 +1485,7 @@ neon_get_request_lsn(bool *latest, NRelFileInfo rinfo, ForkNumber forknum, Block
lsn = GetLastWrittenLSN(rinfo, forknum, blkno);
lsn = nm_adjust_lsn(lsn);
elog(DEBUG1, "neon_get_request_lsn GetXLogReplayRecPtr %X/%X request lsn 0 ",
neon_log(DEBUG1, "neon_get_request_lsn GetXLogReplayRecPtr %X/%X request lsn 0 ",
(uint32) ((lsn) >> 32), (uint32) (lsn));
}
else
@@ -1445,7 +1500,7 @@ neon_get_request_lsn(bool *latest, NRelFileInfo rinfo, ForkNumber forknum, Block
*latest = true;
lsn = GetLastWrittenLSN(rinfo, forknum, blkno);
Assert(lsn != InvalidXLogRecPtr);
elog(DEBUG1, "neon_get_request_lsn GetLastWrittenLSN lsn %X/%X ",
neon_log(DEBUG1, "neon_get_request_lsn GetLastWrittenLSN lsn %X/%X ",
(uint32) ((lsn) >> 32), (uint32) (lsn));
lsn = nm_adjust_lsn(lsn);
@@ -1465,7 +1520,7 @@ neon_get_request_lsn(bool *latest, NRelFileInfo rinfo, ForkNumber forknum, Block
#endif
if (lsn > flushlsn)
{
elog(DEBUG5, "last-written LSN %X/%X is ahead of last flushed LSN %X/%X",
neon_log(DEBUG5, "last-written LSN %X/%X is ahead of last flushed LSN %X/%X",
(uint32) (lsn >> 32), (uint32) lsn,
(uint32) (flushlsn >> 32), (uint32) flushlsn);
XLogFlush(lsn);
@@ -1509,7 +1564,7 @@ neon_exists(SMgrRelation reln, ForkNumber forkNum)
return mdexists(reln, forkNum);
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
if (get_cached_relsize(InfoFromSMgrRel(reln), forkNum, &n_blocks))
@@ -1561,7 +1616,7 @@ neon_exists(SMgrRelation reln, ForkNumber forkNum)
case T_NeonErrorResponse:
ereport(ERROR,
(errcode(ERRCODE_IO_ERROR),
errmsg("could not read relation existence of rel %u/%u/%u.%u from page server at lsn %X/%08X",
errmsg(NEON_TAG "could not read relation existence of rel %u/%u/%u.%u from page server at lsn %X/%08X",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum,
(uint32) (request_lsn >> 32), (uint32) request_lsn),
@@ -1570,7 +1625,7 @@ neon_exists(SMgrRelation reln, ForkNumber forkNum)
break;
default:
elog(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
neon_log(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
}
pfree(resp);
return exists;
@@ -1587,7 +1642,7 @@ neon_create(SMgrRelation reln, ForkNumber forkNum, bool isRedo)
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrcreate() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrcreate() on rel with unknown persistence");
case RELPERSISTENCE_PERMANENT:
break;
@@ -1598,10 +1653,10 @@ neon_create(SMgrRelation reln, ForkNumber forkNum, bool isRedo)
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
elog(SmgrTrace, "Create relation %u/%u/%u.%u",
neon_log(SmgrTrace, "Create relation %u/%u/%u.%u",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum);
@@ -1696,7 +1751,7 @@ neon_extend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno,
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrextend() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrextend() on rel with unknown persistence");
case RELPERSISTENCE_PERMANENT:
break;
@@ -1707,7 +1762,7 @@ neon_extend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno,
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
/*
@@ -1726,7 +1781,7 @@ neon_extend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno,
if (current_size >= ((uint64) max_cluster_size) * 1024 * 1024)
ereport(ERROR,
(errcode(ERRCODE_DISK_FULL),
errmsg("could not extend file because project size limit (%d MB) has been exceeded",
errmsg(NEON_TAG "could not extend file because project size limit (%d MB) has been exceeded",
max_cluster_size),
errhint("This limit is defined externally by the project size limit, and internally by neon.max_cluster_size GUC")));
}
@@ -1745,7 +1800,7 @@ neon_extend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno,
set_cached_relsize(InfoFromSMgrRel(reln), forkNum, blkno + 1);
lsn = PageGetLSN((Page) buffer);
elog(SmgrTrace, "smgrextend called for %u/%u/%u.%u blk %u, page LSN: %X/%08X",
neon_log(SmgrTrace, "smgrextend called for %u/%u/%u.%u blk %u, page LSN: %X/%08X",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum, blkno,
(uint32) (lsn >> 32), (uint32) lsn);
@@ -1785,7 +1840,7 @@ neon_zeroextend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blocknum,
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrextend() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrextend() on rel with unknown persistence");
case RELPERSISTENCE_PERMANENT:
break;
@@ -1796,7 +1851,7 @@ neon_zeroextend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blocknum,
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
if (max_cluster_size > 0 &&
@@ -1808,7 +1863,7 @@ neon_zeroextend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blocknum,
if (current_size >= ((uint64) max_cluster_size) * 1024 * 1024)
ereport(ERROR,
(errcode(ERRCODE_DISK_FULL),
errmsg("could not extend file because cluster size limit (%d MB) has been exceeded",
errmsg(NEON_TAG "could not extend file because cluster size limit (%d MB) has been exceeded",
max_cluster_size),
errhint("This limit is defined by neon.max_cluster_size GUC")));
}
@@ -1821,7 +1876,7 @@ neon_zeroextend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blocknum,
if ((uint64) blocknum + nblocks >= (uint64) InvalidBlockNumber)
ereport(ERROR,
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("cannot extend file \"%s\" beyond %u blocks",
errmsg(NEON_TAG "cannot extend file \"%s\" beyond %u blocks",
relpath(reln->smgr_rlocator, forkNum),
InvalidBlockNumber)));
@@ -1882,7 +1937,7 @@ neon_open(SMgrRelation reln)
mdopen(reln);
/* no work */
elog(SmgrTrace, "[NEON_SMGR] open noop");
neon_log(SmgrTrace, "open noop");
}
/*
@@ -1919,7 +1974,7 @@ neon_prefetch(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum)
return mdprefetch(reln, forknum, blocknum);
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
if (lfc_cache_contains(InfoFromSMgrRel(reln), forknum, blocknum))
@@ -1964,11 +2019,11 @@ neon_writeback(SMgrRelation reln, ForkNumber forknum,
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
/* not implemented */
elog(SmgrTrace, "[NEON_SMGR] writeback noop");
neon_log(SmgrTrace, "writeback noop");
#ifdef DEBUG_COMPARE_LOCAL
if (IS_LOCAL_REL(reln))
@@ -2098,8 +2153,8 @@ neon_read_at_lsn(NRelFileInfo rinfo, ForkNumber forkNum, BlockNumber blkno,
case T_NeonErrorResponse:
ereport(ERROR,
(errcode(ERRCODE_IO_ERROR),
errmsg("could not read block %u in rel %u/%u/%u.%u from page server at lsn %X/%08X",
blkno,
errmsg(NEON_TAG "[shard %d] could not read block %u in rel %u/%u/%u.%u from page server at lsn %X/%08X",
slot->shard_no, blkno,
RelFileInfoFmt(rinfo),
forkNum,
(uint32) (request_lsn >> 32), (uint32) request_lsn),
@@ -2107,7 +2162,7 @@ neon_read_at_lsn(NRelFileInfo rinfo, ForkNumber forkNum, BlockNumber blkno,
((NeonErrorResponse *) resp)->message)));
break;
default:
elog(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
neon_log(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
}
/* buffer was used, clean up for later reuse */
@@ -2131,7 +2186,7 @@ neon_read(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, void *buffer
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrread() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrread() on rel with unknown persistence");
case RELPERSISTENCE_PERMANENT:
break;
@@ -2142,7 +2197,7 @@ neon_read(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, void *buffer
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
/* Try to read from local file cache */
@@ -2170,7 +2225,7 @@ neon_read(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, void *buffer
{
if (!PageIsNew((Page) pageserver_masked))
{
elog(PANIC, "page is new in MD but not in Page Server at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n%s\n",
neon_log(PANIC, "page is new in MD but not in Page Server at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n%s\n",
blkno,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum,
@@ -2180,7 +2235,7 @@ neon_read(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, void *buffer
}
else if (PageIsNew((Page) buffer))
{
elog(PANIC, "page is new in Page Server but not in MD at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n%s\n",
neon_log(PANIC, "page is new in Page Server but not in MD at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n%s\n",
blkno,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum,
@@ -2195,7 +2250,7 @@ neon_read(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, void *buffer
if (memcmp(mdbuf_masked, pageserver_masked, BLCKSZ) != 0)
{
elog(PANIC, "heap buffers differ at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n------ MD ------\n%s\n------ Page Server ------\n%s\n",
neon_log(PANIC, "heap buffers differ at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n------ MD ------\n%s\n------ Page Server ------\n%s\n",
blkno,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum,
@@ -2214,7 +2269,7 @@ neon_read(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, void *buffer
if (memcmp(mdbuf_masked, pageserver_masked, BLCKSZ) != 0)
{
elog(PANIC, "btree buffers differ at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n------ MD ------\n%s\n------ Page Server ------\n%s\n",
neon_log(PANIC, "btree buffers differ at blk %u in rel %u/%u/%u fork %u (request LSN %X/%08X):\n------ MD ------\n%s\n------ Page Server ------\n%s\n",
blkno,
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forkNum,
@@ -2294,13 +2349,13 @@ neon_write(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, const vo
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
neon_wallog_page(reln, forknum, blocknum, buffer, false);
lsn = PageGetLSN((Page) buffer);
elog(SmgrTrace, "smgrwrite called for %u/%u/%u.%u blk %u, page LSN: %X/%08X",
neon_log(SmgrTrace, "smgrwrite called for %u/%u/%u.%u blk %u, page LSN: %X/%08X",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum, blocknum,
(uint32) (lsn >> 32), (uint32) lsn);
@@ -2327,7 +2382,7 @@ neon_nblocks(SMgrRelation reln, ForkNumber forknum)
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrnblocks() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrnblocks() on rel with unknown persistence");
break;
case RELPERSISTENCE_PERMANENT:
@@ -2338,12 +2393,12 @@ neon_nblocks(SMgrRelation reln, ForkNumber forknum)
return mdnblocks(reln, forknum);
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
if (get_cached_relsize(InfoFromSMgrRel(reln), forknum, &n_blocks))
{
elog(SmgrTrace, "cached nblocks for %u/%u/%u.%u: %u blocks",
neon_log(SmgrTrace, "cached nblocks for %u/%u/%u.%u: %u blocks",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum, n_blocks);
return n_blocks;
@@ -2371,7 +2426,7 @@ neon_nblocks(SMgrRelation reln, ForkNumber forknum)
case T_NeonErrorResponse:
ereport(ERROR,
(errcode(ERRCODE_IO_ERROR),
errmsg("could not read relation size of rel %u/%u/%u.%u from page server at lsn %X/%08X",
errmsg(NEON_TAG "could not read relation size of rel %u/%u/%u.%u from page server at lsn %X/%08X",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum,
(uint32) (request_lsn >> 32), (uint32) request_lsn),
@@ -2380,11 +2435,11 @@ neon_nblocks(SMgrRelation reln, ForkNumber forknum)
break;
default:
elog(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
neon_log(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
}
update_cached_relsize(InfoFromSMgrRel(reln), forknum, n_blocks);
elog(SmgrTrace, "neon_nblocks: rel %u/%u/%u fork %u (request LSN %X/%08X): %u blocks",
neon_log(SmgrTrace, "neon_nblocks: rel %u/%u/%u fork %u (request LSN %X/%08X): %u blocks",
RelFileInfoFmt(InfoFromSMgrRel(reln)),
forknum,
(uint32) (request_lsn >> 32), (uint32) request_lsn,
@@ -2427,7 +2482,7 @@ neon_dbsize(Oid dbNode)
case T_NeonErrorResponse:
ereport(ERROR,
(errcode(ERRCODE_IO_ERROR),
errmsg("could not read db size of db %u from page server at lsn %X/%08X",
errmsg(NEON_TAG "could not read db size of db %u from page server at lsn %X/%08X",
dbNode,
(uint32) (request_lsn >> 32), (uint32) request_lsn),
errdetail("page server returned error: %s",
@@ -2435,10 +2490,10 @@ neon_dbsize(Oid dbNode)
break;
default:
elog(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
neon_log(ERROR, "unexpected response from page server with tag 0x%02x", resp->tag);
}
elog(SmgrTrace, "neon_dbsize: db %u (request LSN %X/%08X): %ld bytes",
neon_log(SmgrTrace, "neon_dbsize: db %u (request LSN %X/%08X): %ld bytes",
dbNode,
(uint32) (request_lsn >> 32), (uint32) request_lsn,
db_size);
@@ -2458,7 +2513,7 @@ neon_truncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks)
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrtruncate() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrtruncate() on rel with unknown persistence");
break;
case RELPERSISTENCE_PERMANENT:
@@ -2470,7 +2525,7 @@ neon_truncate(SMgrRelation reln, ForkNumber forknum, BlockNumber nblocks)
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
set_cached_relsize(InfoFromSMgrRel(reln), forknum, nblocks);
@@ -2526,7 +2581,7 @@ neon_immedsync(SMgrRelation reln, ForkNumber forknum)
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgrimmedsync() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgrimmedsync() on rel with unknown persistence");
break;
case RELPERSISTENCE_PERMANENT:
@@ -2538,10 +2593,10 @@ neon_immedsync(SMgrRelation reln, ForkNumber forknum)
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
elog(SmgrTrace, "[NEON_SMGR] immedsync noop");
neon_log(SmgrTrace, "[NEON_SMGR] immedsync noop");
#ifdef DEBUG_COMPARE_LOCAL
if (IS_LOCAL_REL(reln))
@@ -2566,17 +2621,17 @@ neon_start_unlogged_build(SMgrRelation reln)
* progress at a time. That's enough for the current usage.
*/
if (unlogged_build_phase != UNLOGGED_BUILD_NOT_IN_PROGRESS)
elog(ERROR, "unlogged relation build is already in progress");
neon_log(ERROR, "unlogged relation build is already in progress");
Assert(unlogged_build_rel == NULL);
ereport(SmgrTrace,
(errmsg("starting unlogged build of relation %u/%u/%u",
(errmsg(NEON_TAG "starting unlogged build of relation %u/%u/%u",
RelFileInfoFmt(InfoFromSMgrRel(reln)))));
switch (reln->smgr_relpersistence)
{
case 0:
elog(ERROR, "cannot call smgr_start_unlogged_build() on rel with unknown persistence");
neon_log(ERROR, "cannot call smgr_start_unlogged_build() on rel with unknown persistence");
break;
case RELPERSISTENCE_PERMANENT:
@@ -2589,11 +2644,11 @@ neon_start_unlogged_build(SMgrRelation reln)
return;
default:
elog(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
neon_log(ERROR, "unknown relpersistence '%c'", reln->smgr_relpersistence);
}
if (smgrnblocks(reln, MAIN_FORKNUM) != 0)
elog(ERROR, "cannot perform unlogged index build, index is not empty ");
neon_log(ERROR, "cannot perform unlogged index build, index is not empty ");
unlogged_build_rel = reln;
unlogged_build_phase = UNLOGGED_BUILD_PHASE_1;
@@ -2620,7 +2675,7 @@ neon_finish_unlogged_build_phase_1(SMgrRelation reln)
Assert(unlogged_build_rel == reln);
ereport(SmgrTrace,
(errmsg("finishing phase 1 of unlogged build of relation %u/%u/%u",
(errmsg(NEON_TAG "finishing phase 1 of unlogged build of relation %u/%u/%u",
RelFileInfoFmt(InfoFromSMgrRel(reln)))));
if (unlogged_build_phase == UNLOGGED_BUILD_NOT_PERMANENT)
@@ -2649,7 +2704,7 @@ neon_end_unlogged_build(SMgrRelation reln)
Assert(unlogged_build_rel == reln);
ereport(SmgrTrace,
(errmsg("ending unlogged build of relation %u/%u/%u",
(errmsg(NEON_TAG "ending unlogged build of relation %u/%u/%u",
RelFileInfoFmt(InfoFromNInfoB(rinfob)))));
if (unlogged_build_phase != UNLOGGED_BUILD_NOT_PERMANENT)
@@ -2664,7 +2719,7 @@ neon_end_unlogged_build(SMgrRelation reln)
rinfob = InfoBFromSMgrRel(reln);
for (int forknum = 0; forknum <= MAX_FORKNUM; forknum++)
{
elog(SmgrTrace, "forgetting cached relsize for %u/%u/%u.%u",
neon_log(SmgrTrace, "forgetting cached relsize for %u/%u/%u.%u",
RelFileInfoFmt(InfoFromNInfoB(rinfob)),
forknum);
@@ -2707,7 +2762,7 @@ AtEOXact_neon(XactEvent event, void *arg)
unlogged_build_phase = UNLOGGED_BUILD_NOT_IN_PROGRESS;
ereport(ERROR,
(errcode(ERRCODE_INTERNAL_ERROR),
(errmsg("unlogged index build was not properly finished"))));
(errmsg(NEON_TAG "unlogged index build was not properly finished"))));
}
break;
}
@@ -2806,14 +2861,14 @@ neon_extend_rel_size(NRelFileInfo rinfo, ForkNumber forknum, BlockNumber blkno,
set_cached_relsize(rinfo, forknum, relsize);
SetLastWrittenLSNForRelation(end_recptr, rinfo, forknum);
elog(SmgrTrace, "Set length to %d", relsize);
neon_log(SmgrTrace, "Set length to %d", relsize);
}
}
#define FSM_TREE_DEPTH ((SlotsPerFSMPage >= 1626) ? 3 : 4)
/*
* TODO: May be it is better to make correspondent fgunctio from freespace.c public?
* TODO: May be it is better to make correspondent function from freespace.c public?
*/
static BlockNumber
get_fsm_physical_block(BlockNumber heapblk)
@@ -2894,7 +2949,7 @@ neon_redo_read_buffer_filter(XLogReaderState *record, uint8 block_id)
#if PG_VERSION_NUM < 150000
if (!XLogRecGetBlockTag(record, block_id, &rinfo, &forknum, &blkno))
elog(PANIC, "failed to locate backup block with ID %d", block_id);
neon_log(PANIC, "failed to locate backup block with ID %d", block_id);
#else
XLogRecGetBlockTag(record, block_id, &rinfo, &forknum, &blkno);
#endif

View File

@@ -16,6 +16,7 @@ class Metrics:
def query_all(self, name: str, filter: Optional[Dict[str, str]] = None) -> List[Sample]:
filter = filter or {}
res = []
for sample in self.metrics[name]:
try:
if all(sample.labels[k] == v for k, v in filter.items()):

View File

@@ -60,7 +60,7 @@ from fixtures.remote_storage import (
default_remote_storage,
remote_storage_to_toml_inline_table,
)
from fixtures.types import Lsn, TenantId, TimelineId
from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId
from fixtures.utils import (
ATTACHMENT_NAME_REGEX,
allure_add_grafana_links,
@@ -481,6 +481,8 @@ class NeonEnvBuilder:
self,
initial_tenant_conf: Optional[Dict[str, str]] = None,
default_remote_storage_if_missing: bool = True,
initial_tenant_shard_count: Optional[int] = None,
initial_tenant_shard_stripe_size: Optional[int] = None,
) -> NeonEnv:
"""
Default way to create and start NeonEnv. Also creates the initial_tenant with root initial_timeline.
@@ -498,7 +500,11 @@ class NeonEnvBuilder:
f"Services started, creating initial tenant {env.initial_tenant} and its initial timeline"
)
initial_tenant, initial_timeline = env.neon_cli.create_tenant(
tenant_id=env.initial_tenant, conf=initial_tenant_conf, timeline_id=env.initial_timeline
tenant_id=env.initial_tenant,
conf=initial_tenant_conf,
timeline_id=env.initial_timeline,
shard_count=initial_tenant_shard_count,
shard_stripe_size=initial_tenant_shard_stripe_size,
)
assert env.initial_tenant == initial_tenant
assert env.initial_timeline == initial_timeline
@@ -1121,15 +1127,29 @@ class AbstractNeonCli(abc.ABC):
env_vars[var] = val
# Intercept CalledProcessError and print more info
res = subprocess.run(
args,
env=env_vars,
check=False,
universal_newlines=True,
stdout=subprocess.PIPE,
stderr=subprocess.PIPE,
timeout=timeout,
)
try:
res = subprocess.run(
args,
env=env_vars,
check=False,
universal_newlines=True,
stdout=subprocess.PIPE,
stderr=subprocess.PIPE,
timeout=timeout,
)
except subprocess.TimeoutExpired as e:
if e.stderr:
stderr = e.stderr.decode(errors="replace")
else:
stderr = ""
if e.stdout:
stdout = e.stdout.decode(errors="replace")
else:
stdout = ""
log.warn(f"CLI timeout: stderr={stderr}, stdout={stdout}")
raise
indent = " "
if not res.returncode:
@@ -1180,6 +1200,8 @@ class NeonCli(AbstractNeonCli):
tenant_id: Optional[TenantId] = None,
timeline_id: Optional[TimelineId] = None,
conf: Optional[Dict[str, str]] = None,
shard_count: Optional[int] = None,
shard_stripe_size: Optional[int] = None,
set_default: bool = False,
) -> Tuple[TenantId, TimelineId]:
"""
@@ -1207,6 +1229,12 @@ class NeonCli(AbstractNeonCli):
if set_default:
args.append("--set-default")
if shard_count is not None:
args.extend(["--shard-count", str(shard_count)])
if shard_stripe_size is not None:
args.extend(["--shard-stripe-size", str(shard_stripe_size)])
res = self.raw_cli(args)
res.check_returncode()
return tenant_id, timeline_id
@@ -1527,6 +1555,19 @@ class NeonCli(AbstractNeonCli):
return self.raw_cli(args, check_return_code=True)
def tenant_migrate(
self, tenant_shard_id: TenantShardId, new_pageserver: int, timeout_secs: Optional[int]
):
args = [
"tenant",
"migrate",
"--tenant-id",
str(tenant_shard_id),
"--id",
str(new_pageserver),
]
return self.raw_cli(args, check_return_code=True, timeout=timeout_secs)
def start(self, check_return_code=True) -> "subprocess.CompletedProcess[str]":
return self.raw_cli(["start"], check_return_code=check_return_code)
@@ -1622,6 +1663,66 @@ class NeonAttachmentService:
else:
return None
def node_register(self, node: NeonPageserver):
body = {
"node_id": int(node.id),
"listen_http_addr": "localhost",
"listen_http_port": node.service_port.http,
}
log.info(f"node_register({body})")
requests.post(f"{self.env.control_plane_api}/node", json=body).raise_for_status()
def tenant_create(
self,
tenant_id: TenantId,
shard_count: Optional[int] = None,
shard_stripe_size: Optional[int] = None,
tenant_config: Optional[Dict[Any, Any]] = None,
):
body: Dict[str, Any] = {"new_tenant_id": str(tenant_id)}
if shard_count is not None:
shard_params = {"count": shard_count}
if shard_stripe_size is not None:
shard_params["stripe_size"] = shard_stripe_size
body["shard_parameters"] = shard_params
if tenant_config is not None:
for k, v in tenant_config.items():
body[k] = v
response = requests.post(f"{self.env.control_plane_api}/tenant", json=body)
response.raise_for_status()
log.info(f"tenant_create success: {response.json()}")
def tenant_timeline_create(self, tenant_id: TenantId, timeline_id: TimelineId):
body: Dict[str, Any] = {"new_timeline_id": str(timeline_id)}
response = requests.post(
f"{self.env.control_plane_api}/tenant/{tenant_id}/timeline", json=body
)
response.raise_for_status()
log.info(f"tenant_timeline_create success: {response.json()}")
def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]:
response = requests.get(f"{self.env.control_plane_api}/tenant/{tenant_id}/locate")
response.raise_for_status()
body = response.json()
shards: list[dict[str, Any]] = body["shards"]
return shards
def tenant_shard_split(self, tenant_id: TenantId, shard_count: int) -> list[TenantShardId]:
response = requests.put(
f"{self.env.control_plane_api}/tenant/{tenant_id}/shard_split",
json={"new_shard_count": shard_count},
)
response.raise_for_status()
body = response.json()
log.info(f"tenant_shard_split success: {body}")
shards: list[TenantShardId] = body["new_shards"]
return shards
def __enter__(self) -> "NeonAttachmentService":
return self
@@ -3147,7 +3248,7 @@ def pytest_addoption(parser: Parser):
SMALL_DB_FILE_NAME_REGEX: re.Pattern = re.compile( # type: ignore[type-arg]
r"config|config-v1|heatmap-v1|metadata|.+\.(?:toml|pid|json|sql)"
r"config|config-v1|heatmap-v1|metadata|.+\.(?:toml|pid|json|sql|conf)"
)
@@ -3243,9 +3344,7 @@ def list_files_to_compare(pgdata_dir: Path) -> List[str]:
# pg is the existing and running compute node, that we want to compare with a basebackup
def check_restored_datadir_content(
test_output_dir: Path, env: NeonEnv, endpoint: Endpoint, pageserver_id: Optional[int] = None
):
def check_restored_datadir_content(test_output_dir: Path, env: NeonEnv, endpoint: Endpoint):
# Get the timeline ID. We need it for the 'basebackup' command
timeline_id = TimelineId(endpoint.safe_psql("SHOW neon.timeline_id")[0][0])
@@ -3266,6 +3365,7 @@ def check_restored_datadir_content(
pg_bin = PgBin(test_output_dir, env.pg_distrib_dir, env.pg_version)
psql_path = os.path.join(pg_bin.pg_bin_path, "psql")
pageserver_id = env.attachment_service.locate(endpoint.tenant_id)[0]["node_id"]
cmd = rf"""
{psql_path} \
--no-psqlrc \
@@ -3334,6 +3434,27 @@ def logical_replication_sync(subscriber: VanillaPostgres, publisher: Endpoint) -
time.sleep(0.5)
def tenant_get_shards(
env: NeonEnv, tenant_id: TenantId, pageserver_id: Optional[int] = None
) -> list[tuple[TenantShardId, NeonPageserver]]:
"""
Helper for when you want to talk to one or more pageservers, and the
caller _might_ have specified a pageserver, or they might leave it to
us to figure out the shards for a tenant.
Caller should over the response to apply their per-pageserver action to
each shard
"""
if len(env.pageservers) > 1:
return [
(TenantShardId.parse(s["shard_id"]), env.get_pageserver(s["node_id"]))
for s in env.attachment_service.locate(tenant_id)
]
else:
# Assume an unsharded tenant
return [(TenantShardId(tenant_id, 0, 0), env.pageserver)]
def wait_for_last_flush_lsn(
env: NeonEnv,
endpoint: Endpoint,
@@ -3343,10 +3464,22 @@ def wait_for_last_flush_lsn(
) -> Lsn:
"""Wait for pageserver to catch up the latest flush LSN, returns the last observed lsn."""
shards = tenant_get_shards(env, tenant, pageserver_id)
last_flush_lsn = Lsn(endpoint.safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0])
return wait_for_last_record_lsn(
env.get_pageserver(pageserver_id).http_client(), tenant, timeline, last_flush_lsn
)
results = []
for tenant_shard_id, pageserver in shards:
log.info(f"wait_for_last_flush_lsn: shard {tenant_shard_id}")
waited = wait_for_last_record_lsn(
pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn
)
assert waited >= last_flush_lsn
results.append(waited)
# Return the lowest LSN that has been ingested by all shards
return min(results)
def wait_for_wal_insert_lsn(
@@ -3358,9 +3491,16 @@ def wait_for_wal_insert_lsn(
) -> Lsn:
"""Wait for pageserver to catch up the latest flush LSN, returns the last observed lsn."""
last_flush_lsn = Lsn(endpoint.safe_psql("SELECT pg_current_wal_insert_lsn()")[0][0])
return wait_for_last_record_lsn(
env.get_pageserver(pageserver_id).http_client(), tenant, timeline, last_flush_lsn
)
result = None
for tenant_shard_id, pageserver in tenant_get_shards(env, tenant, pageserver_id):
shard_r = wait_for_last_record_lsn(
pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn
)
if result is None:
result = shard_r
assert result is not None
return result
def fork_at_current_lsn(
@@ -3394,11 +3534,13 @@ def last_flush_lsn_upload(
last_flush_lsn = wait_for_last_flush_lsn(
env, endpoint, tenant_id, timeline_id, pageserver_id=pageserver_id
)
ps_http = env.get_pageserver(pageserver_id).http_client()
wait_for_last_record_lsn(ps_http, tenant_id, timeline_id, last_flush_lsn)
# force a checkpoint to trigger upload
ps_http.timeline_checkpoint(tenant_id, timeline_id)
wait_for_upload(ps_http, tenant_id, timeline_id, last_flush_lsn)
shards = tenant_get_shards(env, tenant_id, pageserver_id)
for tenant_shard_id, pageserver in shards:
ps_http = pageserver.http_client()
wait_for_last_record_lsn(ps_http, tenant_shard_id, timeline_id, last_flush_lsn)
# force a checkpoint to trigger upload
ps_http.timeline_checkpoint(tenant_shard_id, timeline_id)
wait_for_upload(ps_http, tenant_shard_id, timeline_id, last_flush_lsn)
return last_flush_lsn

View File

@@ -13,7 +13,7 @@ from urllib3.util.retry import Retry
from fixtures.log_helper import log
from fixtures.metrics import Metrics, parse_metrics
from fixtures.pg_version import PgVersion
from fixtures.types import Lsn, TenantId, TimelineId
from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId
from fixtures.utils import Fn
@@ -433,7 +433,7 @@ class PageserverHttpClient(requests.Session):
def timeline_detail(
self,
tenant_id: TenantId,
tenant_id: TenantShardId,
timeline_id: TimelineId,
include_non_incremental_logical_size: bool = False,
include_timeline_dir_layer_file_size_sum: bool = False,
@@ -455,7 +455,7 @@ class PageserverHttpClient(requests.Session):
assert isinstance(res_json, dict)
return res_json
def timeline_delete(self, tenant_id: TenantId, timeline_id: TimelineId, **kwargs):
def timeline_delete(self, tenant_id: TenantShardId, timeline_id: TimelineId, **kwargs):
"""
Note that deletion is not instant, it is scheduled and performed mostly in the background.
So if you need to wait for it to complete use `timeline_delete_wait_completed`.
@@ -469,7 +469,7 @@ class PageserverHttpClient(requests.Session):
assert res_json is None
def timeline_gc(
self, tenant_id: TenantId, timeline_id: TimelineId, gc_horizon: Optional[int]
self, tenant_id: TenantShardId, timeline_id: TimelineId, gc_horizon: Optional[int]
) -> dict[str, Any]:
"""
Unlike most handlers, this will wait for the layers to be actually
@@ -540,7 +540,7 @@ class PageserverHttpClient(requests.Session):
return res_json
def timeline_checkpoint(
self, tenant_id: TenantId, timeline_id: TimelineId, force_repartition=False
self, tenant_id: TenantShardId, timeline_id: TimelineId, force_repartition=False
):
self.is_testing_enabled_or_skip()
query = {}
@@ -682,6 +682,34 @@ class PageserverHttpClient(requests.Session):
assert len(results) == 1, f"metric {name} with given filters is not unique, got: {results}"
return results[0].value
def get_metrics_values(
self, names: list[str], filter: Optional[Dict[str, str]] = None
) -> Dict[str, float]:
"""
When fetching multiple named metrics, it is more efficient to use this
than to call `get_metric_value` repeatedly.
Throws RuntimeError if no metrics matching `names` are found, or if
not all of `names` are found: this method is intended for loading sets
of metrics whose existence is coupled.
"""
metrics = self.get_metrics()
samples = []
for name in names:
samples.extend(metrics.query_all(name, filter=filter))
result = {}
for sample in samples:
if sample.name in result:
raise RuntimeError(f"Multiple values found for {sample.name}")
result[sample.name] = sample.value
if len(result) != len(names):
log.info(f"Metrics found: {metrics.metrics}")
raise RuntimeError(f"could not find all metrics {' '.join(names)}")
return result
def layer_map_info(
self,
tenant_id: TenantId,

View File

@@ -6,7 +6,7 @@ from mypy_boto3_s3.type_defs import ListObjectsV2OutputTypeDef, ObjectTypeDef
from fixtures.log_helper import log
from fixtures.pageserver.http import PageserverApiException, PageserverHttpClient
from fixtures.remote_storage import RemoteStorageKind, S3Storage
from fixtures.types import Lsn, TenantId, TimelineId
from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId
from fixtures.utils import wait_until
@@ -22,7 +22,7 @@ def assert_tenant_state(
def remote_consistent_lsn(
pageserver_http: PageserverHttpClient, tenant: TenantId, timeline: TimelineId
pageserver_http: PageserverHttpClient, tenant: TenantShardId, timeline: TimelineId
) -> Lsn:
detail = pageserver_http.timeline_detail(tenant, timeline)
@@ -39,7 +39,7 @@ def remote_consistent_lsn(
def wait_for_upload(
pageserver_http: PageserverHttpClient,
tenant: TenantId,
tenant: TenantShardId,
timeline: TimelineId,
lsn: Lsn,
):
@@ -92,7 +92,7 @@ def wait_until_tenant_state(
def wait_until_timeline_state(
pageserver_http: PageserverHttpClient,
tenant_id: TenantId,
tenant_id: TenantShardId,
timeline_id: TimelineId,
expected_state: str,
iterations: int,
@@ -141,7 +141,7 @@ def wait_until_tenant_active(
def last_record_lsn(
pageserver_http_client: PageserverHttpClient, tenant: TenantId, timeline: TimelineId
pageserver_http_client: PageserverHttpClient, tenant: TenantShardId, timeline: TimelineId
) -> Lsn:
detail = pageserver_http_client.timeline_detail(tenant, timeline)
@@ -152,7 +152,7 @@ def last_record_lsn(
def wait_for_last_record_lsn(
pageserver_http: PageserverHttpClient,
tenant: TenantId,
tenant: TenantShardId,
timeline: TimelineId,
lsn: Lsn,
) -> Lsn:
@@ -194,7 +194,7 @@ def wait_for_upload_queue_empty(
def wait_timeline_detail_404(
pageserver_http: PageserverHttpClient,
tenant_id: TenantId,
tenant_id: TenantShardId,
timeline_id: TimelineId,
iterations: int,
interval: Optional[float] = None,
@@ -219,7 +219,7 @@ def wait_timeline_detail_404(
def timeline_delete_wait_completed(
pageserver_http: PageserverHttpClient,
tenant_id: TenantId,
tenant_id: TenantShardId,
timeline_id: TimelineId,
iterations: int = 20,
interval: Optional[float] = None,

View File

@@ -5,6 +5,7 @@ from fixtures.neon_fixtures import (
Endpoint,
NeonEnv,
last_flush_lsn_upload,
tenant_get_shards,
wait_for_last_flush_lsn,
)
from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload
@@ -31,7 +32,7 @@ class Workload:
self._endpoint: Optional[Endpoint] = None
def endpoint(self, pageserver_id: int) -> Endpoint:
def endpoint(self, pageserver_id: Optional[int] = None) -> Endpoint:
if self._endpoint is None:
self._endpoint = self.env.endpoints.create(
"main",
@@ -54,7 +55,7 @@ class Workload:
if self._endpoint is not None:
self._endpoint.stop()
def init(self, pageserver_id: int):
def init(self, pageserver_id: Optional[int] = None):
endpoint = self.endpoint(pageserver_id)
endpoint.safe_psql(f"CREATE TABLE {self.table} (id INTEGER PRIMARY KEY, val text);")
@@ -63,7 +64,7 @@ class Workload:
self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id
)
def write_rows(self, n, pageserver_id):
def write_rows(self, n, pageserver_id: Optional[int] = None):
endpoint = self.endpoint(pageserver_id)
start = self.expect_rows
end = start + n - 1
@@ -81,7 +82,7 @@ class Workload:
self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id
)
def churn_rows(self, n, pageserver_id, upload=True):
def churn_rows(self, n, pageserver_id: Optional[int] = None, upload=True):
assert self.expect_rows >= n
max_iters = 10
@@ -119,21 +120,24 @@ class Workload:
]
)
last_flush_lsn = wait_for_last_flush_lsn(
self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id
)
ps_http = self.env.get_pageserver(pageserver_id).http_client()
wait_for_last_record_lsn(ps_http, self.tenant_id, self.timeline_id, last_flush_lsn)
for tenant_shard_id, pageserver in tenant_get_shards(
self.env, self.tenant_id, pageserver_id
):
last_flush_lsn = wait_for_last_flush_lsn(
self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id
)
ps_http = pageserver.http_client()
wait_for_last_record_lsn(ps_http, tenant_shard_id, self.timeline_id, last_flush_lsn)
if upload:
# force a checkpoint to trigger upload
ps_http.timeline_checkpoint(self.tenant_id, self.timeline_id)
wait_for_upload(ps_http, self.tenant_id, self.timeline_id, last_flush_lsn)
log.info(f"Churn: waiting for remote LSN {last_flush_lsn}")
else:
log.info(f"Churn: not waiting for upload, disk LSN {last_flush_lsn}")
if upload:
# force a checkpoint to trigger upload
ps_http.timeline_checkpoint(tenant_shard_id, self.timeline_id)
wait_for_upload(ps_http, tenant_shard_id, self.timeline_id, last_flush_lsn)
log.info(f"Churn: waiting for remote LSN {last_flush_lsn}")
else:
log.info(f"Churn: not waiting for upload, disk LSN {last_flush_lsn}")
def validate(self, pageserver_id):
def validate(self, pageserver_id: Optional[int] = None):
endpoint = self.endpoint(pageserver_id)
result = endpoint.safe_psql_many(
[

View File

@@ -1,4 +1,6 @@
import random
from contextlib import closing
from typing import Optional
import pytest
from fixtures.log_helper import log
@@ -141,18 +143,24 @@ def test_pageserver_restart(neon_env_builder: NeonEnvBuilder):
# Test that repeatedly kills and restarts the page server, while the
# safekeeper and compute node keep running.
@pytest.mark.timeout(540)
def test_pageserver_chaos(neon_env_builder: NeonEnvBuilder, build_type: str):
@pytest.mark.parametrize("shard_count", [None, 4])
def test_pageserver_chaos(
neon_env_builder: NeonEnvBuilder, build_type: str, shard_count: Optional[int]
):
if build_type == "debug":
pytest.skip("times out in debug builds")
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
neon_env_builder.enable_scrub_on_exit()
if shard_count is not None:
neon_env_builder.num_pageservers = shard_count
env = neon_env_builder.init_start()
env = neon_env_builder.init_start(initial_tenant_shard_count=shard_count)
# these can happen, if we shutdown at a good time. to be fixed as part of #5172.
message = ".*duplicated L1 layer layer=.*"
env.pageserver.allowed_errors.append(message)
for ps in env.pageservers:
ps.allowed_errors.append(message)
# Use a tiny checkpoint distance, to create a lot of layers quickly.
# That allows us to stress the compaction and layer flushing logic more.
@@ -192,13 +200,19 @@ def test_pageserver_chaos(neon_env_builder: NeonEnvBuilder, build_type: str):
log.info(f"shared_buffers is {row[0]}, table size {row[1]}")
assert int(row[0]) < int(row[1])
# We run "random" kills using a fixed seed, to improve reproducibility if a test
# failure is related to a particular order of operations.
seed = 0xDEADBEEF
rng = random.Random(seed)
# Update the whole table, then immediately kill and restart the pageserver
for i in range(1, 15):
endpoint.safe_psql("UPDATE foo set updates = updates + 1")
# This kills the pageserver immediately, to simulate a crash
env.pageserver.stop(immediate=True)
env.pageserver.start()
to_kill = rng.choice(env.pageservers)
to_kill.stop(immediate=True)
to_kill.start()
# Check that all the updates are visible
num_updates = endpoint.safe_psql("SELECT sum(updates) FROM foo")[0][0]

View File

@@ -2,25 +2,40 @@
# This file runs pg_regress-based tests.
#
from pathlib import Path
from typing import Optional
from fixtures.neon_fixtures import NeonEnv, check_restored_datadir_content
import pytest
from fixtures.neon_fixtures import (
NeonEnvBuilder,
check_restored_datadir_content,
)
from fixtures.remote_storage import s3_storage
# Run the main PostgreSQL regression tests, in src/test/regress.
#
@pytest.mark.parametrize("shard_count", [None, 4])
def test_pg_regress(
neon_simple_env: NeonEnv,
neon_env_builder: NeonEnvBuilder,
test_output_dir: Path,
pg_bin,
capsys,
base_dir: Path,
pg_distrib_dir: Path,
shard_count: Optional[int],
):
env = neon_simple_env
"""
:param shard_count: if None, create an unsharded tenant. Otherwise create a tenant with this
many shards.
"""
if shard_count is not None:
neon_env_builder.num_pageservers = shard_count
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
neon_env_builder.enable_scrub_on_exit()
env = neon_env_builder.init_start(initial_tenant_shard_count=shard_count)
env.neon_cli.create_branch("test_pg_regress", "empty")
# Connect to postgres and create a database called "regression".
endpoint = env.endpoints.create_start("test_pg_regress")
endpoint = env.endpoints.create_start("main")
endpoint.safe_psql("CREATE DATABASE regression")
# Create some local directories for pg_regress to run in.
@@ -61,22 +76,25 @@ def test_pg_regress(
# Run the PostgreSQL "isolation" tests, in src/test/isolation.
#
@pytest.mark.parametrize("shard_count", [None, 4])
def test_isolation(
neon_simple_env: NeonEnv,
neon_env_builder: NeonEnvBuilder,
test_output_dir: Path,
pg_bin,
capsys,
base_dir: Path,
pg_distrib_dir: Path,
shard_count: Optional[int],
):
env = neon_simple_env
if shard_count is not None:
neon_env_builder.num_pageservers = shard_count
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
neon_env_builder.enable_scrub_on_exit()
env = neon_env_builder.init_start(initial_tenant_shard_count=shard_count)
env.neon_cli.create_branch("test_isolation", "empty")
# Connect to postgres and create a database called "regression".
# isolation tests use prepared transactions, so enable them
endpoint = env.endpoints.create_start(
"test_isolation", config_lines=["max_prepared_transactions=100"]
)
endpoint = env.endpoints.create_start("main", config_lines=["max_prepared_transactions=100"])
endpoint.safe_psql("CREATE DATABASE isolation_regression")
# Create some local directories for pg_isolation_regress to run in.
@@ -114,19 +132,24 @@ def test_isolation(
# Run extra Neon-specific pg_regress-based tests. The tests and their
# schedule file are in the sql_regress/ directory.
@pytest.mark.parametrize("shard_count", [None, 4])
def test_sql_regress(
neon_simple_env: NeonEnv,
neon_env_builder: NeonEnvBuilder,
test_output_dir: Path,
pg_bin,
capsys,
base_dir: Path,
pg_distrib_dir: Path,
shard_count: Optional[int],
):
env = neon_simple_env
if shard_count is not None:
neon_env_builder.num_pageservers = shard_count
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
neon_env_builder.enable_scrub_on_exit()
env = neon_env_builder.init_start(initial_tenant_shard_count=shard_count)
env.neon_cli.create_branch("test_sql_regress", "empty")
# Connect to postgres and create a database called "regression".
endpoint = env.endpoints.create_start("test_sql_regress")
endpoint = env.endpoints.create_start("main")
endpoint.safe_psql("CREATE DATABASE regression")
# Create some local directories for pg_regress to run in.

View File

@@ -0,0 +1,144 @@
from fixtures.log_helper import log
from fixtures.neon_fixtures import (
NeonEnvBuilder,
tenant_get_shards,
)
from fixtures.remote_storage import s3_storage
from fixtures.workload import Workload
def test_sharding_smoke(
neon_env_builder: NeonEnvBuilder,
):
"""
Test the basic lifecycle of a sharded tenant:
- ingested data gets split up
- page service reads
- timeline creation and deletion
- splits
"""
# We will start with 4 shards and split into 8, then migrate all those
# 8 shards onto separate pageservers
shard_count = 4
split_shard_count = 8
neon_env_builder.num_pageservers = split_shard_count
# 1MiB stripes: enable getting some meaningful data distribution without
# writing large quantities of data in this test.
stripe_size = 128
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
neon_env_builder.enable_scrub_on_exit()
neon_env_builder.preserve_database_files = True
env = neon_env_builder.init_start(
initial_tenant_shard_count=shard_count, initial_tenant_shard_stripe_size=stripe_size
)
tenant_id = env.initial_tenant
timeline_id = env.initial_timeline
for ps in env.pageservers:
ps.allowed_errors.extend(
[
# FIXME: during a split, control plane should respond affirmatively to validation requests
# that refer to a shard that no longer exists, but has a child shard.
".*Dropped remote consistent LSN updates.*",
# FIXME: improve logging in the pageserver so that this isn't considered an erorr, or
# figure out how to make the migration even more seamless.
".*Tenant.*is not active.*",
]
)
# TODO: do some timeline creations & deletions on the sharded tenant
# TODO: validate that timeline APIs show the created timelines on all shards
workload = Workload(env, tenant_id, timeline_id)
workload.init()
workload.write_rows(1000)
# Note which pageservers initially hold a shard after tenant creation
pre_split_pageserver_ids = [loc["node_id"] for loc in env.attachment_service.locate(tenant_id)]
# For pageservers holding a shard, validate their ingest statistics
# reflect a proper splitting of the WAL.
for pageserver in env.pageservers:
if pageserver.id not in pre_split_pageserver_ids:
continue
metrics = pageserver.http_client().get_metrics_values(
[
"pageserver_wal_ingest_records_received_total",
"pageserver_wal_ingest_records_committed_total",
"pageserver_wal_ingest_records_filtered_total",
]
)
log.info(f"Pageserver {pageserver.id} metrics: {metrics}")
# Not everything received was committed
assert (
metrics["pageserver_wal_ingest_records_received_total"]
> metrics["pageserver_wal_ingest_records_committed_total"]
)
# Something was committed
assert metrics["pageserver_wal_ingest_records_committed_total"] > 0
# Counts are self consistent
assert (
metrics["pageserver_wal_ingest_records_received_total"]
== metrics["pageserver_wal_ingest_records_committed_total"]
+ metrics["pageserver_wal_ingest_records_filtered_total"]
)
# TODO: validate that shards have different sizes
workload.validate()
assert len(pre_split_pageserver_ids) == 4
env.attachment_service.tenant_shard_split(tenant_id, shard_count=split_shard_count)
post_split_pageserver_ids = [loc["node_id"] for loc in env.attachment_service.locate(tenant_id)]
# We should have split into 8 shards, on the same 4 pageservers we started on.
assert len(post_split_pageserver_ids) == split_shard_count
assert len(set(post_split_pageserver_ids)) == shard_count
assert set(post_split_pageserver_ids) == set(pre_split_pageserver_ids)
workload.validate()
workload.churn_rows(1000)
workload.validate()
# Run GC on all new shards, to check they don't barf or delete anything that breaks reads
# (compaction was already run as part of churn_rows)
all_shards = tenant_get_shards(env, tenant_id)
for tenant_shard_id, pageserver in all_shards:
pageserver.http_client().timeline_gc(tenant_shard_id, timeline_id, None)
# Restart all nodes, to check that the newly created shards are durable
for ps in env.pageservers:
ps.restart()
workload.validate()
migrate_to_pageserver_ids = list(
set(p.id for p in env.pageservers) - set(pre_split_pageserver_ids)
)
assert len(migrate_to_pageserver_ids) == split_shard_count - shard_count
# Migrate shards away from the node where the split happened
for ps_id in pre_split_pageserver_ids:
shards_here = [
tenant_shard_id
for (tenant_shard_id, pageserver) in all_shards
if pageserver.id == ps_id
]
assert len(shards_here) == 2
migrate_shard = shards_here[0]
destination = migrate_to_pageserver_ids.pop()
log.info(f"Migrating shard {migrate_shard} from {ps_id} to {destination}")
env.neon_cli.tenant_migrate(migrate_shard, destination, timeout_secs=10)