mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-17 10:22:56 +00:00
Compare commits
45 Commits
release-pr
...
jcsp/attac
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
3f1e5b38d8 | ||
|
|
56fddf1ea6 | ||
|
|
2a13306c71 | ||
|
|
54386a537d | ||
|
|
b4b5d20ddc | ||
|
|
fa058efa58 | ||
|
|
cd1e438169 | ||
|
|
f67080410f | ||
|
|
82ccd449d7 | ||
|
|
d4c4ee6a14 | ||
|
|
758bb24445 | ||
|
|
ed11fc0ec8 | ||
|
|
61044aa7f0 | ||
|
|
35de775ed5 | ||
|
|
95505e5ac1 | ||
|
|
dda046bbcd | ||
|
|
7b2019fdfe | ||
|
|
b57848236f | ||
|
|
218a8a7461 | ||
|
|
e745391c72 | ||
|
|
6e56f88b79 | ||
|
|
3ccf5abc9d | ||
|
|
bc90272e47 | ||
|
|
aa1252d687 | ||
|
|
727eef05b3 | ||
|
|
a5813e2516 | ||
|
|
3c1d8e7239 | ||
|
|
3835a51429 | ||
|
|
495c3d70f3 | ||
|
|
d1af9d480e | ||
|
|
14b0acbda7 | ||
|
|
583375e6f6 | ||
|
|
1e542b3187 | ||
|
|
37db221a60 | ||
|
|
9055985d72 | ||
|
|
fc2f9fa3fe | ||
|
|
9cd72caabf | ||
|
|
d1a0a0941a | ||
|
|
347bd012b3 | ||
|
|
58f64339f3 | ||
|
|
78e673fbb3 | ||
|
|
3d573be816 | ||
|
|
bdc4a7512b | ||
|
|
24d0395f66 | ||
|
|
7bbfc160aa |
@@ -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) => {
|
||||
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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]
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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!(
|
||||
|
||||
@@ -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?)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
61
demo_sharding.sh
Normal 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
16
demo_split_4.sh
Normal 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
20
demo_split_8.sh
Normal 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
|
||||
@@ -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.
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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(),
|
||||
)
|
||||
}
|
||||
});
|
||||
|
||||
@@ -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)
|
||||
})
|
||||
|
||||
@@ -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)?;
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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)?;
|
||||
|
||||
|
||||
@@ -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 {})",
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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()):
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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(
|
||||
[
|
||||
|
||||
@@ -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]
|
||||
|
||||
@@ -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.
|
||||
|
||||
144
test_runner/regress/test_sharding.py
Normal file
144
test_runner/regress/test_sharding.py
Normal 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)
|
||||
Reference in New Issue
Block a user