mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 13:32:57 +00:00
storage_controller: coordinate imports across shards in the storage controller (#11345)
## Problem Pageservers notify control plane directly when a shard import has completed. Control plane has to download the status of each shard from S3 and figure out if everything is truly done, before proceeding with branch activation. Issues with this approach are: * We can't control shard split behaviour on the storage controller side. It's unsafe to split during import. * Control plane needs to know about shards and implement logic to check all timelines are indeed ready. ## Summary of changes In short, storage controller coordinates imports, and, only when everything is done, notifies control plane. Big rocks: 1. Store timeline imports in the storage controller database. Each import stores the status of its shards in the database. We hook into the timeline creation call as our entry point for this. 2. Pageservers get a new upcall endpoint to notify the storage controller of shard import updates. 3. Storage controller handles these updates by updating persisted state. If an update finalizes the import, then poll pageservers until timeline activation, and, then, notify the control plane that the import is complete. Cplane side change with new endpoint is in https://github.com/neondatabase/cloud/pull/26166 Closes https://github.com/neondatabase/neon/issues/11566
This commit is contained in:
@@ -30,7 +30,9 @@ use pageserver_api::models::{
|
||||
TimelineArchivalConfigRequest, TimelineCreateRequest,
|
||||
};
|
||||
use pageserver_api::shard::TenantShardId;
|
||||
use pageserver_api::upcall_api::{ReAttachRequest, ValidateRequest};
|
||||
use pageserver_api::upcall_api::{
|
||||
PutTimelineImportStatusRequest, ReAttachRequest, ValidateRequest,
|
||||
};
|
||||
use pageserver_client::{BlockUnblock, mgmt_api};
|
||||
use routerify::Middleware;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
@@ -154,6 +156,28 @@ async fn handle_validate(req: Request<Body>) -> Result<Response<Body>, ApiError>
|
||||
json_response(StatusCode::OK, state.service.validate(validate_req).await?)
|
||||
}
|
||||
|
||||
async fn handle_put_timeline_import_status(req: Request<Body>) -> Result<Response<Body>, ApiError> {
|
||||
check_permissions(&req, Scope::GenerationsApi)?;
|
||||
|
||||
let mut req = match maybe_forward(req).await {
|
||||
ForwardOutcome::Forwarded(res) => {
|
||||
return res;
|
||||
}
|
||||
ForwardOutcome::NotForwarded(req) => req,
|
||||
};
|
||||
|
||||
let put_req = json_request::<PutTimelineImportStatusRequest>(&mut req).await?;
|
||||
|
||||
let state = get_state(&req);
|
||||
json_response(
|
||||
StatusCode::OK,
|
||||
state
|
||||
.service
|
||||
.handle_timeline_shard_import_progress_upcall(put_req)
|
||||
.await?,
|
||||
)
|
||||
}
|
||||
|
||||
/// Call into this before attaching a tenant to a pageserver, to acquire a generation number
|
||||
/// (in the real control plane this is unnecessary, because the same program is managing
|
||||
/// generation numbers and doing attachments).
|
||||
@@ -1961,6 +1985,13 @@ pub fn make_router(
|
||||
.post("/upcall/v1/validate", |r| {
|
||||
named_request_span(r, handle_validate, RequestName("upcall_v1_validate"))
|
||||
})
|
||||
.post("/upcall/v1/timeline_import_status", |r| {
|
||||
named_request_span(
|
||||
r,
|
||||
handle_put_timeline_import_status,
|
||||
RequestName("upcall_v1_timeline_import_status"),
|
||||
)
|
||||
})
|
||||
// Test/dev/debug endpoints
|
||||
.post("/debug/v1/attach-hook", |r| {
|
||||
named_request_span(r, handle_attach_hook, RequestName("debug_v1_attach_hook"))
|
||||
|
||||
@@ -23,6 +23,7 @@ mod scheduler;
|
||||
mod schema;
|
||||
pub mod service;
|
||||
mod tenant_shard;
|
||||
mod timeline_import;
|
||||
|
||||
#[derive(Ord, PartialOrd, Eq, PartialEq, Copy, Clone, Serialize)]
|
||||
struct Sequence(u64);
|
||||
|
||||
@@ -212,6 +212,21 @@ impl PageserverClient {
|
||||
)
|
||||
}
|
||||
|
||||
pub(crate) async fn timeline_detail(
|
||||
&self,
|
||||
tenant_shard_id: TenantShardId,
|
||||
timeline_id: TimelineId,
|
||||
) -> Result<TimelineInfo> {
|
||||
measured_request!(
|
||||
"timeline_detail",
|
||||
crate::metrics::Method::Get,
|
||||
&self.node_id_label,
|
||||
self.inner
|
||||
.timeline_detail(tenant_shard_id, timeline_id)
|
||||
.await
|
||||
)
|
||||
}
|
||||
|
||||
pub(crate) async fn tenant_shard_split(
|
||||
&self,
|
||||
tenant_shard_id: TenantShardId,
|
||||
|
||||
@@ -22,7 +22,7 @@ use pageserver_api::controller_api::{
|
||||
AvailabilityZone, MetadataHealthRecord, NodeSchedulingPolicy, PlacementPolicy,
|
||||
SafekeeperDescribeResponse, ShardSchedulingPolicy, SkSchedulingPolicy,
|
||||
};
|
||||
use pageserver_api::models::TenantConfig;
|
||||
use pageserver_api::models::{ShardImportStatus, TenantConfig};
|
||||
use pageserver_api::shard::{
|
||||
ShardConfigError, ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId,
|
||||
};
|
||||
@@ -40,6 +40,9 @@ use crate::metrics::{
|
||||
DatabaseQueryErrorLabelGroup, DatabaseQueryLatencyLabelGroup, METRICS_REGISTRY,
|
||||
};
|
||||
use crate::node::Node;
|
||||
use crate::timeline_import::{
|
||||
TimelineImport, TimelineImportUpdateError, TimelineImportUpdateFollowUp,
|
||||
};
|
||||
const MIGRATIONS: EmbeddedMigrations = embed_migrations!("./migrations");
|
||||
|
||||
/// ## What do we store?
|
||||
@@ -127,6 +130,9 @@ pub(crate) enum DatabaseOperation {
|
||||
RemoveTimelineReconcile,
|
||||
ListTimelineReconcile,
|
||||
ListTimelineReconcileStartup,
|
||||
InsertTimelineImport,
|
||||
UpdateTimelineImport,
|
||||
DeleteTimelineImport,
|
||||
}
|
||||
|
||||
#[must_use]
|
||||
@@ -1614,6 +1620,129 @@ impl Persistence {
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) async fn insert_timeline_import(
|
||||
&self,
|
||||
import: TimelineImportPersistence,
|
||||
) -> DatabaseResult<bool> {
|
||||
self.with_measured_conn(DatabaseOperation::InsertTimelineImport, move |conn| {
|
||||
Box::pin({
|
||||
let import = import.clone();
|
||||
async move {
|
||||
let inserted = diesel::insert_into(crate::schema::timeline_imports::table)
|
||||
.values(import)
|
||||
.execute(conn)
|
||||
.await?;
|
||||
Ok(inserted == 1)
|
||||
}
|
||||
})
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
pub(crate) async fn delete_timeline_import(
|
||||
&self,
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
) -> DatabaseResult<()> {
|
||||
use crate::schema::timeline_imports::dsl;
|
||||
|
||||
self.with_measured_conn(DatabaseOperation::DeleteTimelineImport, move |conn| {
|
||||
Box::pin(async move {
|
||||
diesel::delete(crate::schema::timeline_imports::table)
|
||||
.filter(
|
||||
dsl::tenant_id
|
||||
.eq(tenant_id.to_string())
|
||||
.and(dsl::timeline_id.eq(timeline_id.to_string())),
|
||||
)
|
||||
.execute(conn)
|
||||
.await?;
|
||||
|
||||
Ok(())
|
||||
})
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
/// Idempotently update the status of one shard for an ongoing timeline import
|
||||
///
|
||||
/// If the update was persisted to the database, then the current state of the
|
||||
/// import is returned to the caller. In case of logical errors a bespoke
|
||||
/// [`TimelineImportUpdateError`] instance is returned. Other database errors
|
||||
/// are covered by the outer [`DatabaseError`].
|
||||
pub(crate) async fn update_timeline_import(
|
||||
&self,
|
||||
tenant_shard_id: TenantShardId,
|
||||
timeline_id: TimelineId,
|
||||
shard_status: ShardImportStatus,
|
||||
) -> DatabaseResult<Result<Option<TimelineImport>, TimelineImportUpdateError>> {
|
||||
use crate::schema::timeline_imports::dsl;
|
||||
|
||||
self.with_measured_conn(DatabaseOperation::UpdateTimelineImport, move |conn| {
|
||||
Box::pin({
|
||||
let shard_status = shard_status.clone();
|
||||
async move {
|
||||
// Load the current state from the database
|
||||
let mut from_db: Vec<TimelineImportPersistence> = dsl::timeline_imports
|
||||
.filter(
|
||||
dsl::tenant_id
|
||||
.eq(tenant_shard_id.tenant_id.to_string())
|
||||
.and(dsl::timeline_id.eq(timeline_id.to_string())),
|
||||
)
|
||||
.load(conn)
|
||||
.await?;
|
||||
|
||||
assert!(from_db.len() <= 1);
|
||||
|
||||
let mut status = match from_db.pop() {
|
||||
Some(some) => TimelineImport::from_persistent(some).unwrap(),
|
||||
None => {
|
||||
return Ok(Err(TimelineImportUpdateError::ImportNotFound {
|
||||
tenant_id: tenant_shard_id.tenant_id,
|
||||
timeline_id,
|
||||
}));
|
||||
}
|
||||
};
|
||||
|
||||
// Perform the update in-memory
|
||||
let follow_up = match status.update(tenant_shard_id.to_index(), shard_status) {
|
||||
Ok(ok) => ok,
|
||||
Err(err) => {
|
||||
return Ok(Err(err));
|
||||
}
|
||||
};
|
||||
|
||||
let new_persistent = status.to_persistent();
|
||||
|
||||
// Write back if required (in the same transaction)
|
||||
match follow_up {
|
||||
TimelineImportUpdateFollowUp::Persist => {
|
||||
let updated = diesel::update(dsl::timeline_imports)
|
||||
.filter(
|
||||
dsl::tenant_id
|
||||
.eq(tenant_shard_id.tenant_id.to_string())
|
||||
.and(dsl::timeline_id.eq(timeline_id.to_string())),
|
||||
)
|
||||
.set(dsl::shard_statuses.eq(new_persistent.shard_statuses))
|
||||
.execute(conn)
|
||||
.await?;
|
||||
|
||||
if updated != 1 {
|
||||
return Ok(Err(TimelineImportUpdateError::ImportNotFound {
|
||||
tenant_id: tenant_shard_id.tenant_id,
|
||||
timeline_id,
|
||||
}));
|
||||
}
|
||||
|
||||
Ok(Ok(Some(status)))
|
||||
}
|
||||
TimelineImportUpdateFollowUp::None => Ok(Ok(None)),
|
||||
}
|
||||
}
|
||||
})
|
||||
})
|
||||
.await
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn load_certs() -> anyhow::Result<Arc<rustls::RootCertStore>> {
|
||||
@@ -2171,3 +2300,11 @@ impl ToSql<diesel::sql_types::VarChar, Pg> for SafekeeperTimelineOpKind {
|
||||
.map_err(Into::into)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, Queryable, Selectable, Insertable, Eq, PartialEq, Clone)]
|
||||
#[diesel(table_name = crate::schema::timeline_imports)]
|
||||
pub(crate) struct TimelineImportPersistence {
|
||||
pub(crate) tenant_id: String,
|
||||
pub(crate) timeline_id: String,
|
||||
pub(crate) shard_statuses: serde_json::Value,
|
||||
}
|
||||
|
||||
@@ -76,6 +76,14 @@ diesel::table! {
|
||||
}
|
||||
}
|
||||
|
||||
diesel::table! {
|
||||
timeline_imports (tenant_id, timeline_id) {
|
||||
tenant_id -> Varchar,
|
||||
timeline_id -> Varchar,
|
||||
shard_statuses -> Jsonb,
|
||||
}
|
||||
}
|
||||
|
||||
diesel::table! {
|
||||
use diesel::sql_types::*;
|
||||
use super::sql_types::PgLsn;
|
||||
@@ -99,5 +107,6 @@ diesel::allow_tables_to_appear_in_same_query!(
|
||||
safekeeper_timeline_pending_ops,
|
||||
safekeepers,
|
||||
tenant_shards,
|
||||
timeline_imports,
|
||||
timelines,
|
||||
);
|
||||
|
||||
@@ -40,14 +40,14 @@ use pageserver_api::models::{
|
||||
TenantLocationConfigResponse, TenantShardLocation, TenantShardSplitRequest,
|
||||
TenantShardSplitResponse, TenantSorting, TenantTimeTravelRequest,
|
||||
TimelineArchivalConfigRequest, TimelineCreateRequest, TimelineCreateResponseStorcon,
|
||||
TimelineInfo, TopTenantShardItem, TopTenantShardsRequest,
|
||||
TimelineInfo, TimelineState, TopTenantShardItem, TopTenantShardsRequest,
|
||||
};
|
||||
use pageserver_api::shard::{
|
||||
DEFAULT_STRIPE_SIZE, ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId,
|
||||
};
|
||||
use pageserver_api::upcall_api::{
|
||||
ReAttachRequest, ReAttachResponse, ReAttachResponseTenant, ValidateRequest, ValidateResponse,
|
||||
ValidateResponseTenant,
|
||||
PutTimelineImportStatusRequest, ReAttachRequest, ReAttachResponse, ReAttachResponseTenant,
|
||||
ValidateRequest, ValidateResponse, ValidateResponseTenant,
|
||||
};
|
||||
use pageserver_client::{BlockUnblock, mgmt_api};
|
||||
use reqwest::{Certificate, StatusCode};
|
||||
@@ -97,6 +97,7 @@ use crate::tenant_shard::{
|
||||
ReconcileNeeded, ReconcileResult, ReconcileWaitError, ReconcilerStatus, ReconcilerWaiter,
|
||||
ScheduleOptimization, ScheduleOptimizationAction, TenantShard,
|
||||
};
|
||||
use crate::timeline_import::{ShardImportStatuses, TimelineImport, UpcallClient};
|
||||
|
||||
const WAITER_FILL_DRAIN_POLL_TIMEOUT: Duration = Duration::from_millis(500);
|
||||
|
||||
@@ -3732,11 +3733,14 @@ impl Service {
|
||||
create_req: TimelineCreateRequest,
|
||||
) -> Result<TimelineCreateResponseStorcon, ApiError> {
|
||||
let safekeepers = self.config.timelines_onto_safekeepers;
|
||||
let timeline_id = create_req.new_timeline_id;
|
||||
|
||||
tracing::info!(
|
||||
mode=%create_req.mode_tag(),
|
||||
%safekeepers,
|
||||
"Creating timeline {}/{}",
|
||||
tenant_id,
|
||||
create_req.new_timeline_id,
|
||||
timeline_id,
|
||||
);
|
||||
|
||||
let _tenant_lock = trace_shared_lock(
|
||||
@@ -3746,15 +3750,62 @@ impl Service {
|
||||
)
|
||||
.await;
|
||||
failpoint_support::sleep_millis_async!("tenant-create-timeline-shared-lock");
|
||||
let create_mode = create_req.mode.clone();
|
||||
let is_import = create_req.is_import();
|
||||
|
||||
let timeline_info = self
|
||||
.tenant_timeline_create_pageservers(tenant_id, create_req)
|
||||
.await?;
|
||||
|
||||
let safekeepers = if safekeepers {
|
||||
let selected_safekeepers = if is_import {
|
||||
let shards = {
|
||||
let locked = self.inner.read().unwrap();
|
||||
locked
|
||||
.tenants
|
||||
.range(TenantShardId::tenant_range(tenant_id))
|
||||
.map(|(ts_id, _)| ts_id.to_index())
|
||||
.collect::<Vec<_>>()
|
||||
};
|
||||
|
||||
if !shards
|
||||
.iter()
|
||||
.map(|shard_index| shard_index.shard_count)
|
||||
.all_equal()
|
||||
{
|
||||
return Err(ApiError::InternalServerError(anyhow::anyhow!(
|
||||
"Inconsistent shard count"
|
||||
)));
|
||||
}
|
||||
|
||||
let import = TimelineImport {
|
||||
tenant_id,
|
||||
timeline_id,
|
||||
shard_statuses: ShardImportStatuses::new(shards),
|
||||
};
|
||||
|
||||
let inserted = self
|
||||
.persistence
|
||||
.insert_timeline_import(import.to_persistent())
|
||||
.await
|
||||
.context("timeline import insert")
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
match inserted {
|
||||
true => {
|
||||
tracing::info!(%tenant_id, %timeline_id, "Inserted timeline import");
|
||||
}
|
||||
false => {
|
||||
tracing::info!(%tenant_id, %timeline_id, "Timeline import entry already present");
|
||||
}
|
||||
}
|
||||
|
||||
None
|
||||
} else if safekeepers {
|
||||
// Note that we do not support creating the timeline on the safekeepers
|
||||
// for imported timelines. The `start_lsn` of the timeline is not known
|
||||
// until the import finshes.
|
||||
// https://github.com/neondatabase/neon/issues/11569
|
||||
let res = self
|
||||
.tenant_timeline_create_safekeepers(tenant_id, &timeline_info, create_mode)
|
||||
.tenant_timeline_create_safekeepers(tenant_id, &timeline_info)
|
||||
.instrument(tracing::info_span!("timeline_create_safekeepers", %tenant_id, timeline_id=%timeline_info.timeline_id))
|
||||
.await?;
|
||||
Some(res)
|
||||
@@ -3764,10 +3815,168 @@ impl Service {
|
||||
|
||||
Ok(TimelineCreateResponseStorcon {
|
||||
timeline_info,
|
||||
safekeepers,
|
||||
safekeepers: selected_safekeepers,
|
||||
})
|
||||
}
|
||||
|
||||
pub(crate) async fn handle_timeline_shard_import_progress_upcall(
|
||||
self: &Arc<Self>,
|
||||
req: PutTimelineImportStatusRequest,
|
||||
) -> Result<(), ApiError> {
|
||||
let res = self
|
||||
.persistence
|
||||
.update_timeline_import(req.tenant_shard_id, req.timeline_id, req.status)
|
||||
.await;
|
||||
let timeline_import = match res {
|
||||
Ok(Ok(Some(timeline_import))) => timeline_import,
|
||||
Ok(Ok(None)) => {
|
||||
// Idempotency: we've already seen and handled this update.
|
||||
return Ok(());
|
||||
}
|
||||
Ok(Err(logical_err)) => {
|
||||
return Err(logical_err.into());
|
||||
}
|
||||
Err(db_err) => {
|
||||
return Err(db_err.into());
|
||||
}
|
||||
};
|
||||
|
||||
tracing::info!(
|
||||
tenant_id=%req.tenant_shard_id.tenant_id,
|
||||
timeline_id=%req.timeline_id,
|
||||
shard_id=%req.tenant_shard_id.shard_slug(),
|
||||
"Updated timeline import status to: {timeline_import:?}");
|
||||
|
||||
if timeline_import.is_complete() {
|
||||
tokio::task::spawn({
|
||||
let this = self.clone();
|
||||
async move { this.finalize_timeline_import(timeline_import).await }
|
||||
});
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[instrument(skip_all, fields(
|
||||
tenant_id=%import.tenant_id,
|
||||
shard_id=%import.timeline_id,
|
||||
))]
|
||||
async fn finalize_timeline_import(
|
||||
self: &Arc<Self>,
|
||||
import: TimelineImport,
|
||||
) -> anyhow::Result<()> {
|
||||
// TODO(vlad): On start-up, load up the imports and notify cplane of the
|
||||
// ones that have been completed. This assumes the new cplane API will
|
||||
// be idempotent. If that's not possible, bang a flag in the database.
|
||||
// https://github.com/neondatabase/neon/issues/11570
|
||||
|
||||
tracing::info!("Finalizing timeline import");
|
||||
|
||||
let import_failed = import.completion_error().is_some();
|
||||
|
||||
if !import_failed {
|
||||
loop {
|
||||
if self.cancel.is_cancelled() {
|
||||
anyhow::bail!("Shut down requested while finalizing import");
|
||||
}
|
||||
|
||||
let active = self.timeline_active_on_all_shards(&import).await?;
|
||||
|
||||
match active {
|
||||
true => {
|
||||
tracing::info!("Timeline became active on all shards");
|
||||
break;
|
||||
}
|
||||
false => {
|
||||
tracing::info!("Timeline not active on all shards yet");
|
||||
|
||||
tokio::select! {
|
||||
_ = self.cancel.cancelled() => {
|
||||
anyhow::bail!("Shut down requested while finalizing import");
|
||||
},
|
||||
_ = tokio::time::sleep(Duration::from_secs(5)) => {}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
tracing::info!(%import_failed, "Notifying cplane of import completion");
|
||||
|
||||
let client = UpcallClient::new(self.get_config(), self.cancel.child_token());
|
||||
client.notify_import_complete(&import).await?;
|
||||
|
||||
if let Err(err) = self
|
||||
.persistence
|
||||
.delete_timeline_import(import.tenant_id, import.timeline_id)
|
||||
.await
|
||||
{
|
||||
tracing::warn!("Failed to delete timeline import entry from database: {err}");
|
||||
}
|
||||
|
||||
// TODO(vlad): Timeline creations in import mode do not return a correct initdb lsn,
|
||||
// so we can't create the timeline on the safekeepers. Fix by moving creation here.
|
||||
// https://github.com/neondatabase/neon/issues/11569
|
||||
tracing::info!(%import_failed, "Timeline import complete");
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn timeline_active_on_all_shards(
|
||||
self: &Arc<Self>,
|
||||
import: &TimelineImport,
|
||||
) -> anyhow::Result<bool> {
|
||||
let targets = {
|
||||
let locked = self.inner.read().unwrap();
|
||||
let mut targets = Vec::new();
|
||||
|
||||
for (tenant_shard_id, shard) in locked
|
||||
.tenants
|
||||
.range(TenantShardId::tenant_range(import.tenant_id))
|
||||
{
|
||||
if !import
|
||||
.shard_statuses
|
||||
.0
|
||||
.contains_key(&tenant_shard_id.to_index())
|
||||
{
|
||||
anyhow::bail!("Shard layout change detected on completion");
|
||||
}
|
||||
|
||||
if let Some(node_id) = shard.intent.get_attached() {
|
||||
let node = locked
|
||||
.nodes
|
||||
.get(node_id)
|
||||
.expect("Pageservers may not be deleted while referenced");
|
||||
targets.push((*tenant_shard_id, node.clone()));
|
||||
} else {
|
||||
return Ok(false);
|
||||
}
|
||||
}
|
||||
|
||||
targets
|
||||
};
|
||||
|
||||
let results = self
|
||||
.tenant_for_shards_api(
|
||||
targets,
|
||||
|tenant_shard_id, client| async move {
|
||||
client
|
||||
.timeline_detail(tenant_shard_id, import.timeline_id)
|
||||
.await
|
||||
},
|
||||
1,
|
||||
1,
|
||||
SHORT_RECONCILE_TIMEOUT,
|
||||
&self.cancel,
|
||||
)
|
||||
.await;
|
||||
|
||||
Ok(results.into_iter().all(|res| match res {
|
||||
Ok(info) => info.state == TimelineState::Active,
|
||||
Err(_) => false,
|
||||
}))
|
||||
}
|
||||
|
||||
pub(crate) async fn tenant_timeline_archival_config(
|
||||
&self,
|
||||
tenant_id: TenantId,
|
||||
|
||||
@@ -15,7 +15,7 @@ use http_utils::error::ApiError;
|
||||
use pageserver_api::controller_api::{
|
||||
SafekeeperDescribeResponse, SkSchedulingPolicy, TimelineImportRequest,
|
||||
};
|
||||
use pageserver_api::models::{self, SafekeeperInfo, SafekeepersInfo, TimelineInfo};
|
||||
use pageserver_api::models::{SafekeeperInfo, SafekeepersInfo, TimelineInfo};
|
||||
use safekeeper_api::membership::{MemberSet, SafekeeperId};
|
||||
use tokio::task::JoinSet;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
@@ -207,7 +207,6 @@ impl Service {
|
||||
self: &Arc<Self>,
|
||||
tenant_id: TenantId,
|
||||
timeline_info: &TimelineInfo,
|
||||
create_mode: models::TimelineCreateRequestMode,
|
||||
) -> Result<SafekeepersInfo, ApiError> {
|
||||
let timeline_id = timeline_info.timeline_id;
|
||||
let pg_version = timeline_info.pg_version * 10000;
|
||||
@@ -217,15 +216,8 @@ impl Service {
|
||||
// previously existed as on retries in theory endpoint might have
|
||||
// already written some data and advanced last_record_lsn, while we want
|
||||
// safekeepers to have consistent start_lsn.
|
||||
let start_lsn = match create_mode {
|
||||
models::TimelineCreateRequestMode::Bootstrap { .. } => timeline_info.last_record_lsn,
|
||||
models::TimelineCreateRequestMode::Branch { .. } => timeline_info.last_record_lsn,
|
||||
models::TimelineCreateRequestMode::ImportPgdata { .. } => {
|
||||
return Err(ApiError::InternalServerError(anyhow::anyhow!(
|
||||
"import pgdata doesn't specify the start lsn, aborting creation on safekeepers"
|
||||
)))?;
|
||||
}
|
||||
};
|
||||
let start_lsn = timeline_info.last_record_lsn;
|
||||
|
||||
// Choose initial set of safekeepers respecting affinity
|
||||
let sks = self.safekeepers_for_new_timeline().await?;
|
||||
let sks_persistence = sks.iter().map(|sk| sk.id.0 as i64).collect::<Vec<_>>();
|
||||
|
||||
260
storage_controller/src/timeline_import.rs
Normal file
260
storage_controller/src/timeline_import.rs
Normal file
@@ -0,0 +1,260 @@
|
||||
use std::time::Duration;
|
||||
use std::{collections::HashMap, str::FromStr};
|
||||
|
||||
use http_utils::error::ApiError;
|
||||
use reqwest::Method;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use pageserver_api::models::ShardImportStatus;
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use utils::{
|
||||
id::{TenantId, TimelineId},
|
||||
shard::ShardIndex,
|
||||
};
|
||||
|
||||
use crate::{persistence::TimelineImportPersistence, service::Config};
|
||||
|
||||
#[derive(Serialize, Deserialize, Clone, Debug)]
|
||||
pub(crate) struct ShardImportStatuses(pub(crate) HashMap<ShardIndex, ShardImportStatus>);
|
||||
|
||||
impl ShardImportStatuses {
|
||||
pub(crate) fn new(shards: Vec<ShardIndex>) -> Self {
|
||||
ShardImportStatuses(
|
||||
shards
|
||||
.into_iter()
|
||||
.map(|ts_id| (ts_id, ShardImportStatus::InProgress))
|
||||
.collect(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct TimelineImport {
|
||||
pub(crate) tenant_id: TenantId,
|
||||
pub(crate) timeline_id: TimelineId,
|
||||
pub(crate) shard_statuses: ShardImportStatuses,
|
||||
}
|
||||
|
||||
pub(crate) enum TimelineImportUpdateFollowUp {
|
||||
Persist,
|
||||
None,
|
||||
}
|
||||
|
||||
pub(crate) enum TimelineImportUpdateError {
|
||||
ImportNotFound {
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
},
|
||||
MismatchedShards,
|
||||
UnexpectedUpdate,
|
||||
}
|
||||
|
||||
impl From<TimelineImportUpdateError> for ApiError {
|
||||
fn from(err: TimelineImportUpdateError) -> ApiError {
|
||||
match err {
|
||||
TimelineImportUpdateError::ImportNotFound {
|
||||
tenant_id,
|
||||
timeline_id,
|
||||
} => ApiError::NotFound(
|
||||
anyhow::anyhow!("Import for {tenant_id}/{timeline_id} not found").into(),
|
||||
),
|
||||
TimelineImportUpdateError::MismatchedShards => {
|
||||
ApiError::InternalServerError(anyhow::anyhow!(
|
||||
"Import shards do not match update request, likely a shard split happened during import, this is a bug"
|
||||
))
|
||||
}
|
||||
TimelineImportUpdateError::UnexpectedUpdate => {
|
||||
ApiError::InternalServerError(anyhow::anyhow!("Update request is unexpected"))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl TimelineImport {
|
||||
pub(crate) fn from_persistent(persistent: TimelineImportPersistence) -> anyhow::Result<Self> {
|
||||
let tenant_id = TenantId::from_str(persistent.tenant_id.as_str())?;
|
||||
let timeline_id = TimelineId::from_str(persistent.timeline_id.as_str())?;
|
||||
let shard_statuses = serde_json::from_value(persistent.shard_statuses)?;
|
||||
|
||||
Ok(TimelineImport {
|
||||
tenant_id,
|
||||
timeline_id,
|
||||
shard_statuses,
|
||||
})
|
||||
}
|
||||
|
||||
pub(crate) fn to_persistent(&self) -> TimelineImportPersistence {
|
||||
TimelineImportPersistence {
|
||||
tenant_id: self.tenant_id.to_string(),
|
||||
timeline_id: self.timeline_id.to_string(),
|
||||
shard_statuses: serde_json::to_value(self.shard_statuses.clone()).unwrap(),
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn update(
|
||||
&mut self,
|
||||
shard: ShardIndex,
|
||||
status: ShardImportStatus,
|
||||
) -> Result<TimelineImportUpdateFollowUp, TimelineImportUpdateError> {
|
||||
use std::collections::hash_map::Entry::*;
|
||||
|
||||
match self.shard_statuses.0.entry(shard) {
|
||||
Occupied(mut occ) => {
|
||||
let crnt = occ.get_mut();
|
||||
if *crnt == status {
|
||||
Ok(TimelineImportUpdateFollowUp::None)
|
||||
} else if crnt.is_terminal() && !status.is_terminal() {
|
||||
Err(TimelineImportUpdateError::UnexpectedUpdate)
|
||||
} else {
|
||||
*crnt = status;
|
||||
Ok(TimelineImportUpdateFollowUp::Persist)
|
||||
}
|
||||
}
|
||||
Vacant(_) => Err(TimelineImportUpdateError::MismatchedShards),
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn is_complete(&self) -> bool {
|
||||
self.shard_statuses
|
||||
.0
|
||||
.values()
|
||||
.all(|status| status.is_terminal())
|
||||
}
|
||||
|
||||
pub(crate) fn completion_error(&self) -> Option<String> {
|
||||
assert!(self.is_complete());
|
||||
|
||||
let shard_errors: HashMap<_, _> = self
|
||||
.shard_statuses
|
||||
.0
|
||||
.iter()
|
||||
.filter_map(|(shard, status)| {
|
||||
if let ShardImportStatus::Error(err) = status {
|
||||
Some((*shard, err.clone()))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
})
|
||||
.collect();
|
||||
|
||||
if shard_errors.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(serde_json::to_string(&shard_errors).unwrap())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct UpcallClient {
|
||||
authorization_header: Option<String>,
|
||||
client: reqwest::Client,
|
||||
cancel: CancellationToken,
|
||||
base_url: String,
|
||||
}
|
||||
|
||||
const IMPORT_COMPLETE_REQUEST_TIMEOUT: Duration = Duration::from_secs(10);
|
||||
|
||||
#[derive(Serialize, Deserialize, Debug)]
|
||||
struct ImportCompleteRequest {
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
error: Option<String>,
|
||||
}
|
||||
|
||||
impl UpcallClient {
|
||||
pub(crate) fn new(config: &Config, cancel: CancellationToken) -> Self {
|
||||
let authorization_header = config
|
||||
.control_plane_jwt_token
|
||||
.clone()
|
||||
.map(|jwt| format!("Bearer {}", jwt));
|
||||
|
||||
let client = reqwest::ClientBuilder::new()
|
||||
.timeout(IMPORT_COMPLETE_REQUEST_TIMEOUT)
|
||||
.build()
|
||||
.expect("Failed to construct HTTP client");
|
||||
|
||||
let base_url = config
|
||||
.control_plane_url
|
||||
.clone()
|
||||
.expect("must be configured");
|
||||
|
||||
Self {
|
||||
authorization_header,
|
||||
client,
|
||||
cancel,
|
||||
base_url,
|
||||
}
|
||||
}
|
||||
|
||||
/// Notify control plane of a completed import
|
||||
///
|
||||
/// This method guarantees at least once delivery semantics assuming
|
||||
/// eventual cplane availability. The cplane API is idempotent.
|
||||
pub(crate) async fn notify_import_complete(
|
||||
&self,
|
||||
import: &TimelineImport,
|
||||
) -> anyhow::Result<()> {
|
||||
let endpoint = if self.base_url.ends_with('/') {
|
||||
format!("{}import_complete", self.base_url)
|
||||
} else {
|
||||
format!("{}/import_complete", self.base_url)
|
||||
};
|
||||
|
||||
tracing::info!("Endpoint is {endpoint}");
|
||||
|
||||
let request = self
|
||||
.client
|
||||
.request(Method::PUT, endpoint)
|
||||
.json(&ImportCompleteRequest {
|
||||
tenant_id: import.tenant_id,
|
||||
timeline_id: import.timeline_id,
|
||||
error: import.completion_error(),
|
||||
})
|
||||
.timeout(IMPORT_COMPLETE_REQUEST_TIMEOUT);
|
||||
|
||||
let request = if let Some(auth) = &self.authorization_header {
|
||||
request.header(reqwest::header::AUTHORIZATION, auth)
|
||||
} else {
|
||||
request
|
||||
};
|
||||
|
||||
const RETRY_DELAY: Duration = Duration::from_secs(1);
|
||||
let mut attempt = 1;
|
||||
|
||||
loop {
|
||||
if self.cancel.is_cancelled() {
|
||||
return Err(anyhow::anyhow!(
|
||||
"Shutting down while notifying cplane of import completion"
|
||||
));
|
||||
}
|
||||
|
||||
match request.try_clone().unwrap().send().await {
|
||||
Ok(response) if response.status().is_success() => {
|
||||
return Ok(());
|
||||
}
|
||||
Ok(response) => {
|
||||
tracing::warn!(
|
||||
"Import complete notification failed with status {}, attempt {}",
|
||||
response.status(),
|
||||
attempt
|
||||
);
|
||||
}
|
||||
Err(e) => {
|
||||
tracing::warn!(
|
||||
"Import complete notification failed with error: {}, attempt {}",
|
||||
e,
|
||||
attempt
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
tokio::select! {
|
||||
_ = tokio::time::sleep(RETRY_DELAY) => {}
|
||||
_ = self.cancel.cancelled() => {
|
||||
return Err(anyhow::anyhow!("Shutting down while notifying cplane of import completion"));
|
||||
}
|
||||
}
|
||||
attempt += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user