mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-06 13:02:55 +00:00
Introduce first version of tenant migraiton between pageservers
This patch includes attach/detach http endpoints in pageservers. Some changes in callmemaybe handling inside safekeeper and an integrational test to check migration with and without load. There are still some rough edges that will be addressed in follow up patches
This commit is contained in:
committed by
Dmitry Rodionov
parent
81e94d1897
commit
37c440c5d3
@@ -7,6 +7,7 @@
|
||||
//!
|
||||
use crate::SafeKeeperConf;
|
||||
use anyhow::{Context, Result};
|
||||
use std::collections::hash_map::Entry;
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Mutex;
|
||||
use std::time::{Duration, Instant};
|
||||
@@ -41,9 +42,12 @@ async fn request_callback(
|
||||
let me_conf: postgres::config::Config = me_connstr.parse().unwrap();
|
||||
let (host, port) = connection_host_port(&me_conf);
|
||||
|
||||
// pageserver connstr is needed to be able to distinguish between different pageservers
|
||||
// it is required to correctly manage callmemaybe subscriptions when more than one pageserver is involved
|
||||
// TODO it is better to use some sort of a unique id instead of connection string, see https://github.com/zenithdb/zenith/issues/1105
|
||||
let callme = format!(
|
||||
"callmemaybe {} {} host={} port={} options='-c ztimelineid={} ztenantid={}'",
|
||||
tenantid, timelineid, host, port, timelineid, tenantid
|
||||
"callmemaybe {} {} host={} port={} options='-c ztimelineid={} ztenantid={} pageserver_connstr={}'",
|
||||
tenantid, timelineid, host, port, timelineid, tenantid, pageserver_connstr,
|
||||
);
|
||||
|
||||
let _ = client.simple_query(&callme).await?;
|
||||
@@ -66,12 +70,13 @@ pub enum CallmeEvent {
|
||||
// add new subscription to the list
|
||||
Subscribe(ZTenantId, ZTimelineId, String),
|
||||
// remove the subscription from the list
|
||||
Unsubscribe(ZTenantId, ZTimelineId),
|
||||
Unsubscribe(ZTenantId, ZTimelineId, String),
|
||||
// don't serve this subscription, but keep it in the list
|
||||
Pause(ZTenantId, ZTimelineId),
|
||||
Pause(ZTenantId, ZTimelineId, String),
|
||||
// resume this subscription, if it exists,
|
||||
// but don't create a new one if it is gone
|
||||
Resume(ZTenantId, ZTimelineId),
|
||||
Resume(ZTenantId, ZTimelineId, String),
|
||||
// TODO how do we delete from subscriptions?
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
@@ -117,6 +122,7 @@ impl SubscriptionState {
|
||||
|
||||
let timelineid = self.timelineid;
|
||||
let tenantid = self.tenantid;
|
||||
let pageserver_connstr = self.pageserver_connstr.clone();
|
||||
tokio::spawn(async move {
|
||||
if let Err(err) = handle.await {
|
||||
if err.is_cancelled() {
|
||||
@@ -124,8 +130,8 @@ impl SubscriptionState {
|
||||
timelineid, tenantid);
|
||||
} else {
|
||||
error!(
|
||||
"callback task for timelineid={} tenantid={} failed: {}",
|
||||
timelineid, tenantid, err
|
||||
"callback task for timelineid={} tenantid={} pageserver_connstr={} failed: {}",
|
||||
timelineid, tenantid, pageserver_connstr, err
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -137,7 +143,7 @@ impl SubscriptionState {
|
||||
// Ignore call request if this subscription is paused
|
||||
if self.paused {
|
||||
debug!(
|
||||
"ignore call request for paused subscription
|
||||
"ignore call request for paused subscription \
|
||||
tenantid: {}, timelineid: {}",
|
||||
self.tenantid, self.timelineid
|
||||
);
|
||||
@@ -147,7 +153,7 @@ impl SubscriptionState {
|
||||
// Check if it too early to recall
|
||||
if self.handle.is_some() && self.last_call_time.elapsed() < recall_period {
|
||||
debug!(
|
||||
"too early to recall. self.last_call_time.elapsed: {:?}, recall_period: {:?}
|
||||
"too early to recall. self.last_call_time.elapsed: {:?}, recall_period: {:?} \
|
||||
tenantid: {}, timelineid: {}",
|
||||
self.last_call_time, recall_period, self.tenantid, self.timelineid
|
||||
);
|
||||
@@ -175,8 +181,7 @@ impl SubscriptionState {
|
||||
// Update last_call_time
|
||||
self.last_call_time = Instant::now();
|
||||
info!(
|
||||
"new call spawned. time {:?}
|
||||
tenantid: {}, timelineid: {}",
|
||||
"new call spawned. last call time {:?} tenantid: {}, timelineid: {}",
|
||||
self.last_call_time, self.tenantid, self.timelineid
|
||||
);
|
||||
}
|
||||
@@ -189,7 +194,7 @@ impl Drop for SubscriptionState {
|
||||
}
|
||||
|
||||
pub async fn main_loop(conf: SafeKeeperConf, mut rx: UnboundedReceiver<CallmeEvent>) -> Result<()> {
|
||||
let subscriptions: Mutex<HashMap<(ZTenantId, ZTimelineId), SubscriptionState>> =
|
||||
let subscriptions: Mutex<HashMap<(ZTenantId, ZTimelineId, String), SubscriptionState>> =
|
||||
Mutex::new(HashMap::new());
|
||||
|
||||
let mut ticker = tokio::time::interval(conf.recall_period);
|
||||
@@ -201,50 +206,73 @@ pub async fn main_loop(conf: SafeKeeperConf, mut rx: UnboundedReceiver<CallmeEve
|
||||
{
|
||||
CallmeEvent::Subscribe(tenantid, timelineid, pageserver_connstr) =>
|
||||
{
|
||||
let _enter = info_span!("callmemaybe: subscribe", timelineid = %timelineid, tenantid = %tenantid, pageserver_connstr=%pageserver_connstr.clone()).entered();
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
if let Some(sub) = subscriptions.get(&(tenantid, timelineid))
|
||||
{
|
||||
info!("callmemaybe. subscription already exists {:?}", sub);
|
||||
match subscriptions.entry((tenantid, timelineid, pageserver_connstr.clone())) {
|
||||
Entry::Occupied(_) => {
|
||||
// Do nothing if subscription already exists
|
||||
// If it is paused it means that there is already established replication connection.
|
||||
// If it is not paused it will be polled with other subscriptions when timeout expires.
|
||||
// This can occur when replication channel is established before subscription is added.
|
||||
info!(
|
||||
"subscription already exists",
|
||||
);
|
||||
}
|
||||
Entry::Vacant(entry) => {
|
||||
let subscription = entry.insert(SubscriptionState::new(
|
||||
tenantid,
|
||||
timelineid,
|
||||
pageserver_connstr,
|
||||
));
|
||||
subscription.call(conf.recall_period, conf.listen_pg_addr.clone());
|
||||
}
|
||||
}
|
||||
if let Some(mut sub) = subscriptions.insert((tenantid, timelineid),
|
||||
SubscriptionState::new(tenantid, timelineid, pageserver_connstr))
|
||||
{
|
||||
sub.call(conf.recall_period, conf.listen_pg_addr.clone());
|
||||
},
|
||||
CallmeEvent::Unsubscribe(tenantid, timelineid, pageserver_connstr) => {
|
||||
let _enter = debug_span!("callmemaybe: unsubscribe", timelineid = %timelineid, tenantid = %tenantid, pageserver_connstr=%pageserver_connstr.clone()).entered();
|
||||
debug!("unsubscribe");
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
subscriptions.remove(&(tenantid, timelineid, pageserver_connstr));
|
||||
|
||||
},
|
||||
CallmeEvent::Pause(tenantid, timelineid, pageserver_connstr) => {
|
||||
let _enter = debug_span!("callmemaybe: pause", timelineid = %timelineid, tenantid = %tenantid, pageserver_connstr=%pageserver_connstr.clone()).entered();
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
// If pause received when no corresponding subscription exists it means that someone started replication
|
||||
// without using callmemaybe. So we create subscription and pause it.
|
||||
// In tenant relocation scenario subscribe call will be executed after pause when compute is restarted.
|
||||
// In that case there is no need to create new/unpause existing subscription.
|
||||
match subscriptions.entry((tenantid, timelineid, pageserver_connstr.clone())) {
|
||||
Entry::Occupied(mut sub) => {
|
||||
debug!("pause existing");
|
||||
sub.get_mut().pause();
|
||||
}
|
||||
Entry::Vacant(entry) => {
|
||||
debug!("create paused");
|
||||
let subscription = entry.insert(SubscriptionState::new(
|
||||
tenantid,
|
||||
timelineid,
|
||||
pageserver_connstr,
|
||||
));
|
||||
subscription.pause();
|
||||
}
|
||||
}
|
||||
info!("callmemaybe. thread_main. subscribe callback request for timelineid={} tenantid={}",
|
||||
timelineid, tenantid);
|
||||
},
|
||||
CallmeEvent::Unsubscribe(tenantid, timelineid) => {
|
||||
CallmeEvent::Resume(tenantid, timelineid, pageserver_connstr) => {
|
||||
debug!("callmemaybe. thread_main. resume callback request for timelineid={} tenantid={} pageserver_connstr={}",
|
||||
timelineid, tenantid, pageserver_connstr);
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
subscriptions.remove(&(tenantid, timelineid));
|
||||
info!("callmemaybe. thread_main. unsubscribe callback. request for timelineid={} tenantid={}",
|
||||
timelineid, tenantid);
|
||||
},
|
||||
CallmeEvent::Pause(tenantid, timelineid) => {
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
if let Some(sub) = subscriptions.get_mut(&(tenantid, timelineid))
|
||||
{
|
||||
sub.pause();
|
||||
};
|
||||
info!("callmemaybe. thread_main. pause callback request for timelineid={} tenantid={}",
|
||||
timelineid, tenantid);
|
||||
},
|
||||
CallmeEvent::Resume(tenantid, timelineid) => {
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
if let Some(sub) = subscriptions.get_mut(&(tenantid, timelineid))
|
||||
if let Some(sub) = subscriptions.get_mut(&(tenantid, timelineid, pageserver_connstr))
|
||||
{
|
||||
sub.resume();
|
||||
};
|
||||
|
||||
info!("callmemaybe. thread_main. resume callback request for timelineid={} tenantid={}",
|
||||
timelineid, tenantid);
|
||||
},
|
||||
}
|
||||
},
|
||||
_ = ticker.tick() => {
|
||||
let mut subscriptions = subscriptions.lock().unwrap();
|
||||
|
||||
for (&(_tenantid, _timelineid), state) in subscriptions.iter_mut() {
|
||||
for (&(_tenantid, _timelineid, _), state) in subscriptions.iter_mut() {
|
||||
state.call(conf.recall_period, conf.listen_pg_addr.clone());
|
||||
}
|
||||
},
|
||||
|
||||
@@ -30,6 +30,7 @@ pub struct SafekeeperPostgresHandler {
|
||||
pub ztenantid: Option<ZTenantId>,
|
||||
pub ztimelineid: Option<ZTimelineId>,
|
||||
pub timeline: Option<Arc<Timeline>>,
|
||||
pageserver_connstr: Option<String>,
|
||||
//sender to communicate with callmemaybe thread
|
||||
pub tx: UnboundedSender<CallmeEvent>,
|
||||
}
|
||||
@@ -88,6 +89,8 @@ impl postgres_backend::Handler for SafekeeperPostgresHandler {
|
||||
self.appname = Some(app_name.clone());
|
||||
}
|
||||
|
||||
self.pageserver_connstr = params.get("pageserver_connstr").cloned();
|
||||
|
||||
Ok(())
|
||||
} else {
|
||||
bail!("Walkeeper received unexpected initial message: {:?}", sm);
|
||||
@@ -127,7 +130,7 @@ impl postgres_backend::Handler for SafekeeperPostgresHandler {
|
||||
}
|
||||
SafekeeperPostgresCommand::StartReplication { start_lsn } => {
|
||||
ReplicationConn::new(pgb)
|
||||
.run(self, pgb, start_lsn)
|
||||
.run(self, pgb, start_lsn, self.pageserver_connstr.clone())
|
||||
.context("failed to run ReplicationConn")?;
|
||||
}
|
||||
SafekeeperPostgresCommand::IdentifySystem => {
|
||||
@@ -149,6 +152,7 @@ impl SafekeeperPostgresHandler {
|
||||
ztenantid: None,
|
||||
ztimelineid: None,
|
||||
timeline: None,
|
||||
pageserver_connstr: None,
|
||||
tx,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,10 +18,9 @@ use crate::handler::SafekeeperPostgresHandler;
|
||||
use crate::timeline::TimelineTools;
|
||||
use zenith_utils::postgres_backend::PostgresBackend;
|
||||
use zenith_utils::pq_proto::{BeMessage, FeMessage};
|
||||
use zenith_utils::zid::{ZTenantId, ZTimelineId};
|
||||
use zenith_utils::zid::ZTenantId;
|
||||
|
||||
use crate::callmemaybe::CallmeEvent;
|
||||
use tokio::sync::mpsc::UnboundedSender;
|
||||
|
||||
pub struct ReceiveWalConn<'pg> {
|
||||
/// Postgres connection
|
||||
@@ -106,13 +105,11 @@ impl<'pg> ReceiveWalConn<'pg> {
|
||||
// Add far as replication in postgres is initiated by receiver
|
||||
// we should use callmemaybe mechanism.
|
||||
let timelineid = spg.timeline.get().timelineid;
|
||||
let tx_clone = spg.tx.clone();
|
||||
let pageserver_connstr = pageserver_connstr.to_owned();
|
||||
spg.tx
|
||||
.send(CallmeEvent::Subscribe(
|
||||
tenant_id,
|
||||
timelineid,
|
||||
pageserver_connstr,
|
||||
pageserver_connstr.to_owned(),
|
||||
))
|
||||
.unwrap_or_else(|e| {
|
||||
error!(
|
||||
@@ -123,9 +120,6 @@ impl<'pg> ReceiveWalConn<'pg> {
|
||||
|
||||
// create a guard to unsubscribe callback, when this wal_stream will exit
|
||||
Some(SendWalHandlerGuard {
|
||||
_tx: tx_clone,
|
||||
_tenant_id: tenant_id,
|
||||
_timelineid: timelineid,
|
||||
timeline: Arc::clone(spg.timeline.get()),
|
||||
})
|
||||
}
|
||||
@@ -147,22 +141,11 @@ impl<'pg> ReceiveWalConn<'pg> {
|
||||
}
|
||||
|
||||
struct SendWalHandlerGuard {
|
||||
_tx: UnboundedSender<CallmeEvent>,
|
||||
_tenant_id: ZTenantId,
|
||||
_timelineid: ZTimelineId,
|
||||
timeline: Arc<Timeline>,
|
||||
}
|
||||
|
||||
impl Drop for SendWalHandlerGuard {
|
||||
fn drop(&mut self) {
|
||||
self.timeline.stop_streaming();
|
||||
// self.tx
|
||||
// .send(CallmeEvent::Unsubscribe(self.tenant_id, self.timelineid))
|
||||
// .unwrap_or_else(|e| {
|
||||
// error!(
|
||||
// "failed to send Unsubscribe request to callmemaybe thread {}",
|
||||
// e
|
||||
// );
|
||||
// });
|
||||
}
|
||||
}
|
||||
|
||||
@@ -91,17 +91,30 @@ struct ReplicationStreamGuard {
|
||||
tx: UnboundedSender<CallmeEvent>,
|
||||
tenant_id: ZTenantId,
|
||||
timelineid: ZTimelineId,
|
||||
pageserver_connstr: String,
|
||||
}
|
||||
|
||||
impl Drop for ReplicationStreamGuard {
|
||||
fn drop(&mut self) {
|
||||
// the connection with pageserver is lost,
|
||||
// resume callback subscription
|
||||
debug!("Connection to pageserver is gone. Resume callmemeybe subsciption if necessary. tenantid {} timelineid {}",
|
||||
self.tenant_id, self.timelineid);
|
||||
debug!(
|
||||
"Connection to pageserver is gone. Resume callmemaybe subsciption if necessary. tenantid {} timelineid {}",
|
||||
self.tenant_id, self.timeline_id,
|
||||
);
|
||||
|
||||
let subscription_key = SubscriptionStateKey::new(
|
||||
self.tenant_id,
|
||||
self.timeline_id,
|
||||
self.pageserver_connstr.to_owned(),
|
||||
);
|
||||
|
||||
self.tx
|
||||
.send(CallmeEvent::Resume(self.tenant_id, self.timelineid))
|
||||
.send(CallmeEvent::Resume(
|
||||
self.tenant_id,
|
||||
self.timelineid,
|
||||
self.pageserver_connstr.to_owned(),
|
||||
))
|
||||
.unwrap_or_else(|e| {
|
||||
error!("failed to send Resume request to callmemaybe thread {}", e);
|
||||
});
|
||||
@@ -194,8 +207,9 @@ impl ReplicationConn {
|
||||
spg: &mut SafekeeperPostgresHandler,
|
||||
pgb: &mut PostgresBackend,
|
||||
mut start_pos: Lsn,
|
||||
pageserver_connstr: Option<String>,
|
||||
) -> Result<()> {
|
||||
let _enter = info_span!("WAL sender", timeline = %spg.ztimelineid.unwrap()).entered();
|
||||
let _enter = info_span!("WAL sender", timeline = %spg.ztimelineid.unwrap(), pageserver_connstr = %pageserver_connstr.as_deref().unwrap_or_default()).entered();
|
||||
|
||||
// spawn the background thread which receives HotStandbyFeedback messages.
|
||||
let bg_timeline = Arc::clone(spg.timeline.get());
|
||||
@@ -256,11 +270,16 @@ impl ReplicationConn {
|
||||
if spg.appname == Some("wal_proposer_recovery".to_string()) {
|
||||
None
|
||||
} else {
|
||||
let pageserver_connstr = pageserver_connstr.clone().expect("there should be a pageserver connection string since this is not a wal_proposer_recovery");
|
||||
let timelineid = spg.timeline.get().timelineid;
|
||||
let tenant_id = spg.ztenantid.unwrap();
|
||||
let tx_clone = spg.tx.clone();
|
||||
spg.tx
|
||||
.send(CallmeEvent::Pause(tenant_id, timelineid))
|
||||
.send(CallmeEvent::Pause(
|
||||
tenant_id,
|
||||
timelineid,
|
||||
pageserver_connstr.clone(),
|
||||
))
|
||||
.unwrap_or_else(|e| {
|
||||
error!("failed to send Pause request to callmemaybe thread {}", e);
|
||||
});
|
||||
@@ -270,6 +289,7 @@ impl ReplicationConn {
|
||||
tx: tx_clone,
|
||||
tenant_id,
|
||||
timelineid,
|
||||
pageserver_connstr,
|
||||
})
|
||||
}
|
||||
};
|
||||
@@ -293,12 +313,20 @@ impl ReplicationConn {
|
||||
if let Some(lsn) = lsn {
|
||||
end_pos = lsn;
|
||||
} else {
|
||||
// Is is time to end streaming to this replica?
|
||||
// Is it time to end streaming to this replica?
|
||||
if spg.timeline.get().check_stop_streaming(replica_id) {
|
||||
let timelineid = spg.timeline.get().timelineid;
|
||||
// this expect should never fail because in wal_proposer_recovery mode stop_pos is set
|
||||
// and this code is not reachable
|
||||
let pageserver_connstr = pageserver_connstr
|
||||
.expect("there should be a pageserver connection string since this is not a wal_proposer_recovery");
|
||||
let timelineid = spg.timeline.get().timeline_id;
|
||||
let tenant_id = spg.ztenantid.unwrap();
|
||||
spg.tx
|
||||
.send(CallmeEvent::Unsubscribe(tenant_id, timelineid))
|
||||
.send(CallmeEvent::Unsubscribe(
|
||||
tenant_id,
|
||||
timelineid,
|
||||
pageserver_connstr,
|
||||
))
|
||||
.unwrap_or_else(|e| {
|
||||
error!("failed to send Pause request to callmemaybe thread {}", e);
|
||||
});
|
||||
|
||||
Reference in New Issue
Block a user