storcon: implement safekeeper_migrate handler (#11849)

This PR implements a safekeeper migration algorithm from RFC-035


https://github.com/neondatabase/neon/blob/main/docs/rfcs/035-safekeeper-dynamic-membership-change.md#change-algorithm

- Closes: https://github.com/neondatabase/neon/issues/11823

It is not production-ready yet, but I think it's good enough to commit
and start testing.

There are some known issues which will be addressed in later PRs:
- https://github.com/neondatabase/neon/issues/12186
- https://github.com/neondatabase/neon/issues/12187
- https://github.com/neondatabase/neon/issues/12188
- https://github.com/neondatabase/neon/issues/12189
- https://github.com/neondatabase/neon/issues/12190
- https://github.com/neondatabase/neon/issues/12191
- https://github.com/neondatabase/neon/issues/12192

## Summary of changes
- Implement `tenant_timeline_safekeeper_migrate` handler to drive the
migration
- Add possibility to specify number of safekeepers per timeline in tests
(`timeline_safekeeper_count`)
- Add `term` and `flush_lsn` to `TimelineMembershipSwitchResponse`
- Implement compare-and-swap (CAS) operation over timeline in DB for
updating membership configuration safely.
- Write simple test to verify that migration code works
This commit is contained in:
Dmitrii Kovalkov
2025-06-30 12:30:05 +04:00
committed by GitHub
parent 9bb4688c54
commit c746678bbc
15 changed files with 802 additions and 79 deletions

View File

@@ -22,7 +22,7 @@ use pageserver_api::controller_api::{
MetadataHealthListUnhealthyResponse, MetadataHealthUpdateRequest, MetadataHealthUpdateResponse,
NodeAvailability, NodeConfigureRequest, NodeRegisterRequest, SafekeeperSchedulingPolicyRequest,
ShardsPreferredAzsRequest, TenantCreateRequest, TenantPolicyRequest, TenantShardMigrateRequest,
TimelineImportRequest,
TimelineImportRequest, TimelineSafekeeperMigrateRequest,
};
use pageserver_api::models::{
DetachBehavior, LsnLeaseRequest, TenantConfigPatchRequest, TenantConfigRequest,
@@ -34,6 +34,7 @@ use pageserver_api::upcall_api::{
PutTimelineImportStatusRequest, ReAttachRequest, TimelineImportStatusRequest, ValidateRequest,
};
use pageserver_client::{BlockUnblock, mgmt_api};
use routerify::Middleware;
use tokio_util::sync::CancellationToken;
use tracing::warn;
@@ -635,6 +636,32 @@ async fn handle_tenant_timeline_download_heatmap_layers(
json_response(StatusCode::OK, ())
}
async fn handle_tenant_timeline_safekeeper_migrate(
service: Arc<Service>,
req: Request<Body>,
) -> Result<Response<Body>, ApiError> {
let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?;
check_permissions(&req, Scope::PageServerApi)?;
maybe_rate_limit(&req, tenant_id).await;
let timeline_id: TimelineId = parse_request_param(&req, "timeline_id")?;
let mut req = match maybe_forward(req).await {
ForwardOutcome::Forwarded(res) => {
return res;
}
ForwardOutcome::NotForwarded(req) => req,
};
let migrate_req = json_request::<TimelineSafekeeperMigrateRequest>(&mut req).await?;
service
.tenant_timeline_safekeeper_migrate(tenant_id, timeline_id, migrate_req)
.await?;
json_response(StatusCode::OK, ())
}
async fn handle_tenant_timeline_lsn_lease(
service: Arc<Service>,
req: Request<Body>,
@@ -2458,6 +2485,16 @@ pub fn make_router(
)
},
)
.post(
"/v1/tenant/:tenant_id/timeline/:timeline_id/safekeeper_migrate",
|r| {
tenant_service_handler(
r,
handle_tenant_timeline_safekeeper_migrate,
RequestName("v1_tenant_timeline_safekeeper_migrate"),
)
},
)
// LSN lease passthrough to all shards
.post(
"/v1/tenant/:tenant_id/timeline/:timeline_id/lsn_lease",

View File

@@ -333,6 +333,7 @@ pub(crate) enum DatabaseErrorLabel {
ConnectionPool,
Logical,
Migration,
Cas,
}
impl DatabaseError {
@@ -343,6 +344,7 @@ impl DatabaseError {
Self::ConnectionPool(_) => DatabaseErrorLabel::ConnectionPool,
Self::Logical(_) => DatabaseErrorLabel::Logical,
Self::Migration(_) => DatabaseErrorLabel::Migration,
Self::Cas(_) => DatabaseErrorLabel::Cas,
}
}
}

View File

@@ -29,6 +29,7 @@ use pageserver_api::shard::{
use rustls::client::WebPkiServerVerifier;
use rustls::client::danger::{ServerCertVerified, ServerCertVerifier};
use rustls::crypto::ring;
use safekeeper_api::membership::SafekeeperGeneration;
use scoped_futures::ScopedBoxFuture;
use serde::{Deserialize, Serialize};
use utils::generation::Generation;
@@ -94,6 +95,8 @@ pub(crate) enum DatabaseError {
Logical(String),
#[error("Migration error: {0}")]
Migration(String),
#[error("CAS error: {0}")]
Cas(String),
}
#[derive(measured::FixedCardinalityLabel, Copy, Clone)]
@@ -126,6 +129,7 @@ pub(crate) enum DatabaseOperation {
UpdateLeader,
SetPreferredAzs,
InsertTimeline,
UpdateTimelineMembership,
GetTimeline,
InsertTimelineReconcile,
RemoveTimelineReconcile,
@@ -1410,6 +1414,56 @@ impl Persistence {
.await
}
/// Update timeline membership configuration in the database.
/// Perform a compare-and-swap (CAS) operation on the timeline's generation.
/// The `new_generation` must be the next (+1) generation after the one in the database.
pub(crate) async fn update_timeline_membership(
&self,
tenant_id: TenantId,
timeline_id: TimelineId,
new_generation: SafekeeperGeneration,
sk_set: &[NodeId],
new_sk_set: Option<&[NodeId]>,
) -> DatabaseResult<()> {
use crate::schema::timelines::dsl;
let prev_generation = new_generation.previous().unwrap();
let tenant_id = &tenant_id;
let timeline_id = &timeline_id;
self.with_measured_conn(DatabaseOperation::UpdateTimelineMembership, move |conn| {
Box::pin(async move {
let updated = diesel::update(dsl::timelines)
.filter(dsl::tenant_id.eq(&tenant_id.to_string()))
.filter(dsl::timeline_id.eq(&timeline_id.to_string()))
.filter(dsl::generation.eq(prev_generation.into_inner() as i32))
.set((
dsl::generation.eq(new_generation.into_inner() as i32),
dsl::sk_set.eq(sk_set.iter().map(|id| id.0 as i64).collect::<Vec<_>>()),
dsl::new_sk_set.eq(new_sk_set
.map(|set| set.iter().map(|id| id.0 as i64).collect::<Vec<_>>())),
))
.execute(conn)
.await?;
match updated {
0 => {
// TODO(diko): It makes sense to select the current generation
// and include it in the error message for better debuggability.
Err(DatabaseError::Cas(
"Failed to update membership configuration".to_string(),
))
}
1 => Ok(()),
_ => Err(DatabaseError::Logical(format!(
"unexpected number of rows ({updated})"
))),
}
})
})
.await
}
/// Load timeline from db. Returns `None` if not present.
pub(crate) async fn get_timeline(
&self,

View File

@@ -2,6 +2,7 @@ use std::time::Duration;
use pageserver_api::controller_api::{SafekeeperDescribeResponse, SkSchedulingPolicy};
use reqwest::StatusCode;
use safekeeper_api::membership::SafekeeperId;
use safekeeper_client::mgmt_api;
use tokio_util::sync::CancellationToken;
use utils::backoff;
@@ -92,6 +93,13 @@ impl Safekeeper {
pub(crate) fn has_https_port(&self) -> bool {
self.listen_https_port.is_some()
}
pub(crate) fn get_safekeeper_id(&self) -> SafekeeperId {
SafekeeperId {
id: self.id,
host: self.skp.host.clone(),
pg_port: self.skp.port as u16,
}
}
/// Perform an operation (which is given a [`SafekeeperClient`]) with retries
#[allow(clippy::too_many_arguments)]
pub(crate) async fn with_client_retries<T, O, F>(

View File

@@ -56,6 +56,10 @@ impl SafekeeperClient {
}
}
pub(crate) fn node_id_label(&self) -> &str {
&self.node_id_label
}
pub(crate) async fn create_timeline(
&self,
req: &TimelineCreateRequest,

View File

@@ -161,6 +161,7 @@ enum TenantOperations {
DropDetached,
DownloadHeatmapLayers,
TimelineLsnLease,
TimelineSafekeeperMigrate,
}
#[derive(Clone, strum_macros::Display)]
@@ -491,6 +492,7 @@ impl From<DatabaseError> for ApiError {
DatabaseError::Logical(reason) | DatabaseError::Migration(reason) => {
ApiError::InternalServerError(anyhow::anyhow!(reason))
}
DatabaseError::Cas(reason) => ApiError::Conflict(reason),
}
}
}

View File

@@ -145,7 +145,7 @@ pub(crate) async fn load_schedule_requests(
}
let Some(sk) = safekeepers.get(&other_node_id) else {
tracing::warn!(
"couldnt find safekeeper with pending op id {other_node_id}, not pulling from it"
"couldn't find safekeeper with pending op id {other_node_id}, not pulling from it"
);
return None;
};

View File

@@ -5,20 +5,29 @@ use std::time::Duration;
use super::safekeeper_reconciler::ScheduleRequest;
use crate::heartbeater::SafekeeperState;
use crate::id_lock_map::trace_shared_lock;
use crate::metrics;
use crate::persistence::{
DatabaseError, SafekeeperTimelineOpKind, TimelinePendingOpPersistence, TimelinePersistence,
};
use crate::safekeeper::Safekeeper;
use crate::safekeeper_client::SafekeeperClient;
use crate::service::TenantOperations;
use crate::timeline_import::TimelineImportFinalizeError;
use anyhow::Context;
use http_utils::error::ApiError;
use pageserver_api::controller_api::{
SafekeeperDescribeResponse, SkSchedulingPolicy, TimelineImportRequest,
TimelineSafekeeperMigrateRequest,
};
use pageserver_api::models::{SafekeeperInfo, SafekeepersInfo, TimelineInfo};
use safekeeper_api::PgVersionId;
use safekeeper_api::membership::{MemberSet, SafekeeperGeneration, SafekeeperId};
use safekeeper_api::membership::{self, MemberSet, SafekeeperGeneration};
use safekeeper_api::models::{
PullTimelineRequest, TimelineMembershipSwitchRequest, TimelineMembershipSwitchResponse,
};
use safekeeper_api::{INITIAL_TERM, Term};
use safekeeper_client::mgmt_api;
use tokio::task::JoinSet;
use tokio_util::sync::CancellationToken;
use utils::id::{NodeId, TenantId, TimelineId};
@@ -35,6 +44,33 @@ pub struct TimelineLocateResponse {
}
impl Service {
fn make_member_set(safekeepers: &[Safekeeper]) -> Result<MemberSet, ApiError> {
let members = safekeepers
.iter()
.map(|sk| sk.get_safekeeper_id())
.collect::<Vec<_>>();
MemberSet::new(members).map_err(ApiError::InternalServerError)
}
fn get_safekeepers(&self, ids: &[i64]) -> Result<Vec<Safekeeper>, ApiError> {
let safekeepers = {
let locked = self.inner.read().unwrap();
locked.safekeepers.clone()
};
ids.iter()
.map(|&id| {
let node_id = NodeId(id as u64);
safekeepers.get(&node_id).cloned().ok_or_else(|| {
ApiError::InternalServerError(anyhow::anyhow!(
"safekeeper {node_id} is not registered"
))
})
})
.collect::<Result<Vec<_>, _>>()
}
/// Timeline creation on safekeepers
///
/// Returns `Ok(left)` if the timeline has been created on a quorum of safekeepers,
@@ -47,35 +83,9 @@ impl Service {
pg_version: PgVersionId,
timeline_persistence: &TimelinePersistence,
) -> Result<Vec<NodeId>, ApiError> {
// If quorum is reached, return if we are outside of a specified timeout
let jwt = self
.config
.safekeeper_jwt_token
.clone()
.map(SecretString::from);
let mut joinset = JoinSet::new();
let safekeepers = self.get_safekeepers(&timeline_persistence.sk_set)?;
// Prepare membership::Configuration from choosen safekeepers.
let safekeepers = {
let locked = self.inner.read().unwrap();
locked.safekeepers.clone()
};
let mut members = Vec::new();
for sk_id in timeline_persistence.sk_set.iter() {
let sk_id = NodeId(*sk_id as u64);
let Some(safekeeper) = safekeepers.get(&sk_id) else {
return Err(ApiError::InternalServerError(anyhow::anyhow!(
"couldn't find entry for safekeeper with id {sk_id}"
)))?;
};
members.push(SafekeeperId {
id: sk_id,
host: safekeeper.skp.host.clone(),
pg_port: safekeeper.skp.port as u16,
});
}
let mset = MemberSet::new(members).map_err(ApiError::InternalServerError)?;
let mset = Self::make_member_set(&safekeepers)?;
let mconf = safekeeper_api::membership::Configuration::new(mset);
let req = safekeeper_api::models::TimelineCreateRequest {
@@ -88,79 +98,150 @@ impl Service {
timeline_id,
wal_seg_size: None,
};
const SK_CREATE_TIMELINE_RECONCILE_TIMEOUT: Duration = Duration::from_secs(30);
for sk in timeline_persistence.sk_set.iter() {
let sk_id = NodeId(*sk as u64);
let safekeepers = safekeepers.clone();
let results = self
.tenant_timeline_safekeeper_op_quorum(
&safekeepers,
move |client| {
let req = req.clone();
async move { client.create_timeline(&req).await }
},
SK_CREATE_TIMELINE_RECONCILE_TIMEOUT,
)
.await?;
Ok(results
.into_iter()
.enumerate()
.filter_map(|(idx, res)| {
if res.is_ok() {
None // Success, don't return this safekeeper
} else {
Some(safekeepers[idx].get_id()) // Failure, return this safekeeper
}
})
.collect::<Vec<_>>())
}
/// Perform an operation on a list of safekeepers in parallel with retries.
///
/// Return the results of the operation on each safekeeper in the input order.
async fn tenant_timeline_safekeeper_op<T, O, F>(
&self,
safekeepers: &[Safekeeper],
op: O,
timeout: Duration,
) -> Result<Vec<mgmt_api::Result<T>>, ApiError>
where
O: FnMut(SafekeeperClient) -> F + Send + 'static,
O: Clone,
F: std::future::Future<Output = mgmt_api::Result<T>> + Send + 'static,
T: Sync + Send + 'static,
{
let jwt = self
.config
.safekeeper_jwt_token
.clone()
.map(SecretString::from);
let mut joinset = JoinSet::new();
for (idx, sk) in safekeepers.iter().enumerate() {
let sk = sk.clone();
let http_client = self.http_client.clone();
let jwt = jwt.clone();
let req = req.clone();
let op = op.clone();
joinset.spawn(async move {
// Unwrap is fine as we already would have returned error above
let sk_p = safekeepers.get(&sk_id).unwrap();
let res = sk_p
let res = sk
.with_client_retries(
|client| {
let req = req.clone();
async move { client.create_timeline(&req).await }
},
op,
&http_client,
&jwt,
3,
3,
SK_CREATE_TIMELINE_RECONCILE_TIMEOUT,
// TODO(diko): This is a wrong timeout.
// It should be scaled to the retry count.
timeout,
&CancellationToken::new(),
)
.await;
(sk_id, sk_p.skp.host.clone(), res)
(idx, res)
});
}
// Initialize results with timeout errors in case we never get a response.
let mut results: Vec<mgmt_api::Result<T>> = safekeepers
.iter()
.map(|_| {
Err(mgmt_api::Error::Timeout(
"safekeeper operation timed out".to_string(),
))
})
.collect();
// After we have built the joinset, we now wait for the tasks to complete,
// but with a specified timeout to make sure we return swiftly, either with
// a failure or success.
let reconcile_deadline = tokio::time::Instant::now() + SK_CREATE_TIMELINE_RECONCILE_TIMEOUT;
let reconcile_deadline = tokio::time::Instant::now() + timeout;
// Wait until all tasks finish or timeout is hit, whichever occurs
// first.
let mut reconcile_results = Vec::new();
let mut result_count = 0;
loop {
if let Ok(res) = tokio::time::timeout_at(reconcile_deadline, joinset.join_next()).await
{
let Some(res) = res else { break };
match res {
Ok(res) => {
Ok((idx, res)) => {
let sk = &safekeepers[idx];
tracing::info!(
"response from safekeeper id:{} at {}: {:?}",
res.0,
res.1,
res.2
sk.get_id(),
sk.skp.host,
// Only print errors, as there is no Debug trait for T.
res.as_ref().map(|_| ()),
);
reconcile_results.push(res);
results[idx] = res;
result_count += 1;
}
Err(join_err) => {
tracing::info!("join_err for task in joinset: {join_err}");
}
}
} else {
tracing::info!(
"timeout for creation call after {} responses",
reconcile_results.len()
);
tracing::info!("timeout for operation call after {result_count} responses",);
break;
}
}
// Now check now if quorum was reached in reconcile_results.
let total_result_count = reconcile_results.len();
let remaining = reconcile_results
.into_iter()
.filter_map(|res| res.2.is_err().then_some(res.0))
.collect::<Vec<_>>();
tracing::info!(
"Got {} non-successful responses from initial creation request of total {total_result_count} responses",
remaining.len()
);
let target_sk_count = timeline_persistence.sk_set.len();
Ok(results)
}
/// Perform an operation on a list of safekeepers in parallel with retries,
/// and validates that we reach a quorum of successful responses.
///
/// Return the results of the operation on each safekeeper in the input order.
/// It's guaranteed that at least a quorum of the responses are successful.
async fn tenant_timeline_safekeeper_op_quorum<T, O, F>(
&self,
safekeepers: &[Safekeeper],
op: O,
timeout: Duration,
) -> Result<Vec<mgmt_api::Result<T>>, ApiError>
where
O: FnMut(SafekeeperClient) -> F,
O: Clone + Send + 'static,
F: std::future::Future<Output = mgmt_api::Result<T>> + Send + 'static,
T: Sync + Send + 'static,
{
let results = self
.tenant_timeline_safekeeper_op(safekeepers, op, timeout)
.await?;
// Now check if quorum was reached in results.
let target_sk_count = safekeepers.len();
let quorum_size = match target_sk_count {
0 => {
return Err(ApiError::InternalServerError(anyhow::anyhow!(
@@ -179,7 +260,7 @@ impl Service {
// in order to schedule work to them
tracing::warn!(
"couldn't find at least 3 safekeepers for timeline, found: {:?}",
timeline_persistence.sk_set
target_sk_count
);
return Err(ApiError::InternalServerError(anyhow::anyhow!(
"couldn't find at least 3 safekeepers to put timeline to"
@@ -188,7 +269,7 @@ impl Service {
}
_ => target_sk_count / 2 + 1,
};
let success_count = target_sk_count - remaining.len();
let success_count = results.iter().filter(|res| res.is_ok()).count();
if success_count < quorum_size {
// Failure
return Err(ApiError::InternalServerError(anyhow::anyhow!(
@@ -196,7 +277,7 @@ impl Service {
)));
}
Ok(remaining)
Ok(results)
}
/// Create timeline in controller database and on safekeepers.
@@ -797,4 +878,435 @@ impl Service {
}
Ok(())
}
/// Call `switch_timeline_membership` on all safekeepers with retries
/// till the quorum of successful responses is reached.
///
/// If min_position is not None, validates that majority of safekeepers
/// reached at least min_position.
///
/// Return responses from safekeepers in the input order.
async fn tenant_timeline_set_membership_quorum(
self: &Arc<Self>,
tenant_id: TenantId,
timeline_id: TimelineId,
safekeepers: &[Safekeeper],
config: &membership::Configuration,
min_position: Option<(Term, Lsn)>,
) -> Result<Vec<mgmt_api::Result<TimelineMembershipSwitchResponse>>, ApiError> {
let req = TimelineMembershipSwitchRequest {
mconf: config.clone(),
};
const SK_SET_MEM_TIMELINE_RECONCILE_TIMEOUT: Duration = Duration::from_secs(30);
let results = self
.tenant_timeline_safekeeper_op_quorum(
safekeepers,
move |client| {
let req = req.clone();
async move {
let mut res = client
.switch_timeline_membership(tenant_id, timeline_id, &req)
.await;
// If min_position is not reached, map the response to an error,
// so it isn't counted toward the quorum.
if let Some(min_position) = min_position {
if let Ok(ok_res) = &res {
if (ok_res.term, ok_res.flush_lsn) < min_position {
// Use Error::Timeout to make this error retriable.
res = Err(mgmt_api::Error::Timeout(
format!(
"safekeeper {} returned position {:?} which is less than minimum required position {:?}",
client.node_id_label(),
(ok_res.term, ok_res.flush_lsn),
min_position
)
));
}
}
}
res
}
},
SK_SET_MEM_TIMELINE_RECONCILE_TIMEOUT,
)
.await?;
for res in results.iter().flatten() {
if res.current_conf.generation > config.generation {
// Antoher switch_membership raced us.
return Err(ApiError::Conflict(format!(
"received configuration with generation {} from safekeeper, but expected {}",
res.current_conf.generation, config.generation
)));
} else if res.current_conf.generation < config.generation {
// Note: should never happen.
// If we get a response, it should be at least the sent generation.
tracing::error!(
"received configuration with generation {} from safekeeper, but expected {}",
res.current_conf.generation,
config.generation
);
return Err(ApiError::InternalServerError(anyhow::anyhow!(
"received configuration with generation {} from safekeeper, but expected {}",
res.current_conf.generation,
config.generation
)));
}
}
Ok(results)
}
/// Pull timeline to to_safekeepers from from_safekeepers with retries.
///
/// Returns Ok(()) only if all the pull_timeline requests were successful.
async fn tenant_timeline_pull_from_peers(
self: &Arc<Self>,
tenant_id: TenantId,
timeline_id: TimelineId,
to_safekeepers: &[Safekeeper],
from_safekeepers: &[Safekeeper],
) -> Result<(), ApiError> {
let http_hosts = from_safekeepers
.iter()
.map(|sk| sk.base_url())
.collect::<Vec<_>>();
tracing::info!(
"pulling timeline to {:?} from {:?}",
to_safekeepers
.iter()
.map(|sk| sk.get_id())
.collect::<Vec<_>>(),
from_safekeepers
.iter()
.map(|sk| sk.get_id())
.collect::<Vec<_>>()
);
// TODO(diko): need to pass mconf/generation with the request
// to properly handle tombstones. Ignore tombstones for now.
// Worst case: we leave a timeline on a safekeeper which is not in the current set.
let req = PullTimelineRequest {
tenant_id,
timeline_id,
http_hosts,
ignore_tombstone: Some(true),
};
const SK_PULL_TIMELINE_RECONCILE_TIMEOUT: Duration = Duration::from_secs(30);
let responses = self
.tenant_timeline_safekeeper_op(
to_safekeepers,
move |client| {
let req = req.clone();
async move { client.pull_timeline(&req).await }
},
SK_PULL_TIMELINE_RECONCILE_TIMEOUT,
)
.await?;
if let Some((idx, err)) = responses
.iter()
.enumerate()
.find_map(|(idx, res)| Some((idx, res.as_ref().err()?)))
{
let sk_id = to_safekeepers[idx].get_id();
return Err(ApiError::InternalServerError(anyhow::anyhow!(
"pull_timeline to {sk_id} failed: {err}",
)));
}
Ok(())
}
/// Exclude a timeline from safekeepers in parallel with retries.
/// If an exclude request is unsuccessful, it will be added to
/// the reconciler, and after that the function will succeed.
async fn tenant_timeline_safekeeper_exclude(
self: &Arc<Self>,
tenant_id: TenantId,
timeline_id: TimelineId,
safekeepers: &[Safekeeper],
config: &membership::Configuration,
) -> Result<(), ApiError> {
let req = TimelineMembershipSwitchRequest {
mconf: config.clone(),
};
const SK_EXCLUDE_TIMELINE_TIMEOUT: Duration = Duration::from_secs(30);
let results = self
.tenant_timeline_safekeeper_op(
safekeepers,
move |client| {
let req = req.clone();
async move { client.exclude_timeline(tenant_id, timeline_id, &req).await }
},
SK_EXCLUDE_TIMELINE_TIMEOUT,
)
.await?;
let mut reconcile_requests = Vec::new();
for (idx, res) in results.iter().enumerate() {
if res.is_err() {
let sk_id = safekeepers[idx].skp.id;
let pending_op = TimelinePendingOpPersistence {
tenant_id: tenant_id.to_string(),
timeline_id: timeline_id.to_string(),
generation: config.generation.into_inner() as i32,
op_kind: SafekeeperTimelineOpKind::Exclude,
sk_id,
};
tracing::info!("writing pending exclude op for sk id {sk_id}");
self.persistence.insert_pending_op(pending_op).await?;
let req = ScheduleRequest {
safekeeper: Box::new(safekeepers[idx].clone()),
host_list: Vec::new(),
tenant_id,
timeline_id: Some(timeline_id),
generation: config.generation.into_inner(),
kind: SafekeeperTimelineOpKind::Exclude,
};
reconcile_requests.push(req);
}
}
if !reconcile_requests.is_empty() {
let locked = self.inner.read().unwrap();
for req in reconcile_requests {
locked.safekeeper_reconcilers.schedule_request(req);
}
}
Ok(())
}
/// Migrate timeline safekeeper set to a new set.
///
/// This function implements an algorithm from RFC-035.
/// <https://github.com/neondatabase/neon/blob/main/docs/rfcs/035-safekeeper-dynamic-membership-change.md>
pub(crate) async fn tenant_timeline_safekeeper_migrate(
self: &Arc<Self>,
tenant_id: TenantId,
timeline_id: TimelineId,
req: TimelineSafekeeperMigrateRequest,
) -> Result<(), ApiError> {
let all_safekeepers = self.inner.read().unwrap().safekeepers.clone();
let new_sk_set = req.new_sk_set;
for sk_id in new_sk_set.iter() {
if !all_safekeepers.contains_key(sk_id) {
return Err(ApiError::BadRequest(anyhow::anyhow!(
"safekeeper {sk_id} does not exist"
)));
}
}
// TODO(diko): per-tenant lock is too wide. Consider introducing per-timeline locks.
let _tenant_lock = trace_shared_lock(
&self.tenant_op_locks,
tenant_id,
TenantOperations::TimelineSafekeeperMigrate,
)
.await;
// 1. Fetch current timeline configuration from the configuration storage.
let timeline = self
.persistence
.get_timeline(tenant_id, timeline_id)
.await?;
let Some(timeline) = timeline else {
return Err(ApiError::NotFound(
anyhow::anyhow!(
"timeline {tenant_id}/{timeline_id} doesn't exist in timelines table"
)
.into(),
));
};
let cur_sk_set = timeline
.sk_set
.iter()
.map(|&id| NodeId(id as u64))
.collect::<Vec<_>>();
tracing::info!(
?cur_sk_set,
?new_sk_set,
"Migrating timeline to new safekeeper set",
);
let mut generation = SafekeeperGeneration::new(timeline.generation as u32);
if let Some(ref presistent_new_sk_set) = timeline.new_sk_set {
// 2. If it is already joint one and new_set is different from desired_set refuse to change.
if presistent_new_sk_set
.iter()
.map(|&id| NodeId(id as u64))
.ne(new_sk_set.iter().cloned())
{
tracing::info!(
?presistent_new_sk_set,
?new_sk_set,
"different new safekeeper set is already set in the database",
);
return Err(ApiError::Conflict(format!(
"the timeline is already migrating to a different safekeeper set: {presistent_new_sk_set:?}"
)));
}
// It it is the same new_sk_set, we can continue the migration (retry).
} else {
// 3. No active migration yet.
// Increment current generation and put desired_set to new_sk_set.
generation = generation.next();
self.persistence
.update_timeline_membership(
tenant_id,
timeline_id,
generation,
&cur_sk_set,
Some(&new_sk_set),
)
.await?;
}
let cur_safekeepers = self.get_safekeepers(&timeline.sk_set)?;
let cur_sk_member_set = Self::make_member_set(&cur_safekeepers)?;
let new_sk_set_i64 = new_sk_set.iter().map(|id| id.0 as i64).collect::<Vec<_>>();
let new_safekeepers = self.get_safekeepers(&new_sk_set_i64)?;
let new_sk_member_set = Self::make_member_set(&new_safekeepers)?;
let joint_config = membership::Configuration {
generation,
members: cur_sk_member_set,
new_members: Some(new_sk_member_set.clone()),
};
// 4. Call PUT configuration on safekeepers from the current set,
// delivering them joint_conf.
// TODO(diko): need to notify cplane with an updated set of safekeepers.
let results = self
.tenant_timeline_set_membership_quorum(
tenant_id,
timeline_id,
&cur_safekeepers,
&joint_config,
None, // no min position
)
.await?;
let mut sync_position = (INITIAL_TERM, Lsn::INVALID);
for res in results.into_iter().flatten() {
let sk_position = (res.term, res.flush_lsn);
if sync_position < sk_position {
sync_position = sk_position;
}
}
tracing::info!(
%generation,
?sync_position,
"safekeepers set membership updated",
);
// 5. Initialize timeline on safekeeper(s) from new_sk_set where it doesn't exist yet
// by doing pull_timeline from the majority of the current set.
// Filter out safekeepers which are already in the current set.
let from_ids: HashSet<NodeId> = cur_safekeepers.iter().map(|sk| sk.get_id()).collect();
let pull_to_safekeepers = new_safekeepers
.iter()
.filter(|sk| !from_ids.contains(&sk.get_id()))
.cloned()
.collect::<Vec<_>>();
self.tenant_timeline_pull_from_peers(
tenant_id,
timeline_id,
&pull_to_safekeepers,
&cur_safekeepers,
)
.await?;
// 6. Call POST bump_term(sync_term) on safekeepers from the new set. Success on majority is enough.
// TODO(diko): do we need to bump timeline term?
// 7. Repeatedly call PUT configuration on safekeepers from the new set,
// delivering them joint_conf and collecting their positions.
tracing::info!(?sync_position, "waiting for safekeepers to sync position");
self.tenant_timeline_set_membership_quorum(
tenant_id,
timeline_id,
&new_safekeepers,
&joint_config,
Some(sync_position),
)
.await?;
// 8. Create new_conf: Configuration incrementing joint_conf generation and
// having new safekeeper set as sk_set and None new_sk_set.
let generation = generation.next();
let new_conf = membership::Configuration {
generation,
members: new_sk_member_set,
new_members: None,
};
self.persistence
.update_timeline_membership(tenant_id, timeline_id, generation, &new_sk_set, None)
.await?;
// TODO(diko): at this point we have already updated the timeline in the database,
// but we still need to notify safekeepers and cplane about the new configuration,
// and put delition of the timeline from the old safekeepers into the reconciler.
// Ideally it should be done atomically, but now it's not.
// Worst case: the timeline is not deleted from old safekeepers,
// the compute may require both quorums till the migration is retried and completed.
self.tenant_timeline_set_membership_quorum(
tenant_id,
timeline_id,
&new_safekeepers,
&new_conf,
None, // no min position
)
.await?;
let new_ids: HashSet<NodeId> = new_safekeepers.iter().map(|sk| sk.get_id()).collect();
let exclude_safekeepers = cur_safekeepers
.into_iter()
.filter(|sk| !new_ids.contains(&sk.get_id()))
.collect::<Vec<_>>();
self.tenant_timeline_safekeeper_exclude(
tenant_id,
timeline_id,
&exclude_safekeepers,
&new_conf,
)
.await?;
// TODO(diko): need to notify cplane with an updated set of safekeepers.
Ok(())
}
}