mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 05:22:56 +00:00
Evict WAL files from disk (#8022)
Fixes https://github.com/neondatabase/neon/issues/6337 Add safekeeper support to switch between `Present` and `Offloaded(flush_lsn)` states. The offloading is disabled by default, but can be controlled using new cmdline arguments: ``` --enable-offload Enable automatic switching to offloaded state --delete-offloaded-wal Delete local WAL files after offloading. When disabled, they will be left on disk --control-file-save-interval <CONTROL_FILE_SAVE_INTERVAL> Pending updates to control file will be automatically saved after this interval [default: 300s] ``` Manager watches state updates and detects when there are no actvity on the timeline and actual partial backup upload in remote storage. When all conditions are met, the state can be switched to offloaded. In `timeline.rs` there is `StateSK` enum to support switching between states. When offloaded, code can access only control file structure and cannot use `SafeKeeper` to accept new WAL. `FullAccessTimeline` is now renamed to `WalResidentTimeline`. This struct contains guard to notify manager about active tasks requiring on-disk WAL access. All guards are issued by the manager, all requests are sent via channel using `ManagerCtl`. When manager receives request to issue a guard, it unevicts timeline if it's currently evicted. Fixed a bug in partial WAL backup, it used `term` instead of `last_log_term` previously. After this commit is merged, next step is to roll this change out, as in issue #6338.
This commit is contained in:
committed by
GitHub
parent
dd3adc3693
commit
76fc3d4aa1
@@ -28,8 +28,8 @@ use utils::pid_file;
|
||||
|
||||
use metrics::set_build_info_metric;
|
||||
use safekeeper::defaults::{
|
||||
DEFAULT_HEARTBEAT_TIMEOUT, DEFAULT_HTTP_LISTEN_ADDR, DEFAULT_MAX_OFFLOADER_LAG_BYTES,
|
||||
DEFAULT_PARTIAL_BACKUP_TIMEOUT, DEFAULT_PG_LISTEN_ADDR,
|
||||
DEFAULT_CONTROL_FILE_SAVE_INTERVAL, DEFAULT_HEARTBEAT_TIMEOUT, DEFAULT_HTTP_LISTEN_ADDR,
|
||||
DEFAULT_MAX_OFFLOADER_LAG_BYTES, DEFAULT_PARTIAL_BACKUP_TIMEOUT, DEFAULT_PG_LISTEN_ADDR,
|
||||
};
|
||||
use safekeeper::http;
|
||||
use safekeeper::wal_service;
|
||||
@@ -172,6 +172,7 @@ struct Args {
|
||||
walsenders_keep_horizon: bool,
|
||||
/// Enable partial backup. If disabled, safekeeper will not upload partial
|
||||
/// segments to remote storage.
|
||||
/// TODO: now partial backup is always enabled, remove this flag.
|
||||
#[arg(long)]
|
||||
partial_backup_enabled: bool,
|
||||
/// Controls how long backup will wait until uploading the partial segment.
|
||||
@@ -181,6 +182,15 @@ struct Args {
|
||||
/// be used in tests.
|
||||
#[arg(long)]
|
||||
disable_periodic_broker_push: bool,
|
||||
/// Enable automatic switching to offloaded state.
|
||||
#[arg(long)]
|
||||
enable_offload: bool,
|
||||
/// Delete local WAL files after offloading. When disabled, they will be left on disk.
|
||||
#[arg(long)]
|
||||
delete_offloaded_wal: bool,
|
||||
/// Pending updates to control file will be automatically saved after this interval.
|
||||
#[arg(long, value_parser = humantime::parse_duration, default_value = DEFAULT_CONTROL_FILE_SAVE_INTERVAL)]
|
||||
control_file_save_interval: Duration,
|
||||
}
|
||||
|
||||
// Like PathBufValueParser, but allows empty string.
|
||||
@@ -328,9 +338,12 @@ async fn main() -> anyhow::Result<()> {
|
||||
sk_auth_token,
|
||||
current_thread_runtime: args.current_thread_runtime,
|
||||
walsenders_keep_horizon: args.walsenders_keep_horizon,
|
||||
partial_backup_enabled: args.partial_backup_enabled,
|
||||
partial_backup_enabled: true,
|
||||
partial_backup_timeout: args.partial_backup_timeout,
|
||||
disable_periodic_broker_push: args.disable_periodic_broker_push,
|
||||
enable_offload: args.enable_offload,
|
||||
delete_offloaded_wal: args.delete_offloaded_wal,
|
||||
control_file_save_interval: args.control_file_save_interval,
|
||||
};
|
||||
|
||||
// initialize sentry if SENTRY_DSN is provided
|
||||
|
||||
@@ -72,6 +72,9 @@ impl FileStorage {
|
||||
conf: &SafeKeeperConf,
|
||||
state: TimelinePersistentState,
|
||||
) -> Result<FileStorage> {
|
||||
// we don't support creating new timelines in offloaded state
|
||||
assert!(matches!(state.eviction_state, EvictionState::Present));
|
||||
|
||||
let store = FileStorage {
|
||||
timeline_dir,
|
||||
no_sync: conf.no_sync,
|
||||
@@ -103,7 +106,7 @@ impl FileStorage {
|
||||
}
|
||||
|
||||
/// Load control file from given directory.
|
||||
pub fn load_control_file_from_dir(timeline_dir: &Utf8Path) -> Result<TimelinePersistentState> {
|
||||
fn load_control_file_from_dir(timeline_dir: &Utf8Path) -> Result<TimelinePersistentState> {
|
||||
let path = timeline_dir.join(CONTROL_FILE_NAME);
|
||||
Self::load_control_file(path)
|
||||
}
|
||||
|
||||
@@ -15,7 +15,7 @@ use crate::{
|
||||
control_file::{FileStorage, Storage},
|
||||
pull_timeline::{create_temp_timeline_dir, load_temp_timeline, validate_temp_timeline},
|
||||
state::TimelinePersistentState,
|
||||
timeline::{FullAccessTimeline, Timeline, TimelineError},
|
||||
timeline::{Timeline, TimelineError, WalResidentTimeline},
|
||||
wal_backup::copy_s3_segments,
|
||||
wal_storage::{wal_file_paths, WalReader},
|
||||
GlobalTimelines,
|
||||
@@ -46,7 +46,7 @@ pub async fn handle_request(request: Request) -> Result<()> {
|
||||
}
|
||||
}
|
||||
|
||||
let source_tli = request.source.full_access_guard().await?;
|
||||
let source_tli = request.source.wal_residence_guard().await?;
|
||||
|
||||
let conf = &GlobalTimelines::get_global_config();
|
||||
let ttid = request.destination_ttid;
|
||||
@@ -159,7 +159,7 @@ pub async fn handle_request(request: Request) -> Result<()> {
|
||||
}
|
||||
|
||||
async fn copy_disk_segments(
|
||||
tli: &FullAccessTimeline,
|
||||
tli: &WalResidentTimeline,
|
||||
wal_seg_size: usize,
|
||||
start_lsn: Lsn,
|
||||
end_lsn: Lsn,
|
||||
@@ -183,7 +183,7 @@ async fn copy_disk_segments(
|
||||
let copy_end = copy_end - segment_start;
|
||||
|
||||
let wal_file_path = {
|
||||
let (normal, partial) = wal_file_paths(tli_dir_path, segment, wal_seg_size)?;
|
||||
let (normal, partial) = wal_file_paths(tli_dir_path, segment, wal_seg_size);
|
||||
|
||||
if segment == last_segment {
|
||||
partial
|
||||
|
||||
@@ -28,7 +28,8 @@ use crate::send_wal::WalSenderState;
|
||||
use crate::state::TimelineMemState;
|
||||
use crate::state::TimelinePersistentState;
|
||||
use crate::timeline::get_timeline_dir;
|
||||
use crate::timeline::FullAccessTimeline;
|
||||
use crate::timeline::WalResidentTimeline;
|
||||
use crate::timeline_manager;
|
||||
use crate::GlobalTimelines;
|
||||
use crate::SafeKeeperConf;
|
||||
|
||||
@@ -168,6 +169,7 @@ pub struct Memory {
|
||||
pub last_removed_segno: XLogSegNo,
|
||||
pub epoch_start_lsn: Lsn,
|
||||
pub mem_state: TimelineMemState,
|
||||
pub mgr_status: timeline_manager::Status,
|
||||
|
||||
// PhysicalStorage state.
|
||||
pub write_lsn: Lsn,
|
||||
@@ -326,7 +328,7 @@ pub struct TimelineDigest {
|
||||
}
|
||||
|
||||
pub async fn calculate_digest(
|
||||
tli: &FullAccessTimeline,
|
||||
tli: &WalResidentTimeline,
|
||||
request: TimelineDigestRequest,
|
||||
) -> Result<TimelineDigest> {
|
||||
if request.from_lsn > request.until_lsn {
|
||||
|
||||
@@ -214,10 +214,10 @@ async fn timeline_snapshot_handler(request: Request<Body>) -> Result<Response<Bo
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
// Note: with evicted timelines it should work better then de-evict them and
|
||||
// stream; probably start_snapshot would copy partial s3 file to dest path
|
||||
// and stream control file, or return FullAccessTimeline if timeline is not
|
||||
// and stream control file, or return WalResidentTimeline if timeline is not
|
||||
// evicted.
|
||||
let tli = tli
|
||||
.full_access_guard()
|
||||
.wal_residence_guard()
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -283,7 +283,7 @@ async fn timeline_digest_handler(request: Request<Body>) -> Result<Response<Body
|
||||
|
||||
let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?;
|
||||
let tli = tli
|
||||
.full_access_guard()
|
||||
.wal_residence_guard()
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -306,7 +306,7 @@ async fn timeline_checkpoint_handler(request: Request<Body>) -> Result<Response<
|
||||
tli.write_shared_state()
|
||||
.await
|
||||
.sk
|
||||
.state
|
||||
.state_mut()
|
||||
.flush()
|
||||
.await
|
||||
.map_err(ApiError::InternalServerError)?;
|
||||
|
||||
@@ -21,7 +21,7 @@ use crate::safekeeper::{
|
||||
};
|
||||
use crate::safekeeper::{Term, TermHistory, TermLsn};
|
||||
use crate::state::TimelinePersistentState;
|
||||
use crate::timeline::FullAccessTimeline;
|
||||
use crate::timeline::WalResidentTimeline;
|
||||
use crate::GlobalTimelines;
|
||||
use postgres_backend::PostgresBackend;
|
||||
use postgres_ffi::encode_logical_message;
|
||||
@@ -102,7 +102,7 @@ pub async fn handle_json_ctrl<IO: AsyncRead + AsyncWrite + Unpin>(
|
||||
async fn prepare_safekeeper(
|
||||
ttid: TenantTimelineId,
|
||||
pg_version: u32,
|
||||
) -> anyhow::Result<FullAccessTimeline> {
|
||||
) -> anyhow::Result<WalResidentTimeline> {
|
||||
let tli = GlobalTimelines::create(
|
||||
ttid,
|
||||
ServerInfo {
|
||||
@@ -115,11 +115,11 @@ async fn prepare_safekeeper(
|
||||
)
|
||||
.await?;
|
||||
|
||||
tli.full_access_guard().await
|
||||
tli.wal_residence_guard().await
|
||||
}
|
||||
|
||||
async fn send_proposer_elected(
|
||||
tli: &FullAccessTimeline,
|
||||
tli: &WalResidentTimeline,
|
||||
term: Term,
|
||||
lsn: Lsn,
|
||||
) -> anyhow::Result<()> {
|
||||
@@ -151,7 +151,7 @@ pub struct InsertedWAL {
|
||||
/// Extend local WAL with new LogicalMessage record. To do that,
|
||||
/// create AppendRequest with new WAL and pass it to safekeeper.
|
||||
pub async fn append_logical_message(
|
||||
tli: &FullAccessTimeline,
|
||||
tli: &WalResidentTimeline,
|
||||
msg: &AppendLogicalMessage,
|
||||
) -> anyhow::Result<InsertedWAL> {
|
||||
let wal_data = encode_logical_message(&msg.lm_prefix, &msg.lm_message);
|
||||
|
||||
@@ -28,6 +28,8 @@ pub mod safekeeper;
|
||||
pub mod send_wal;
|
||||
pub mod state;
|
||||
pub mod timeline;
|
||||
pub mod timeline_eviction;
|
||||
pub mod timeline_guard;
|
||||
pub mod timeline_manager;
|
||||
pub mod timelines_set;
|
||||
pub mod wal_backup;
|
||||
@@ -49,6 +51,7 @@ pub mod defaults {
|
||||
pub const DEFAULT_HEARTBEAT_TIMEOUT: &str = "5000ms";
|
||||
pub const DEFAULT_MAX_OFFLOADER_LAG_BYTES: u64 = 128 * (1 << 20);
|
||||
pub const DEFAULT_PARTIAL_BACKUP_TIMEOUT: &str = "15m";
|
||||
pub const DEFAULT_CONTROL_FILE_SAVE_INTERVAL: &str = "300s";
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
@@ -85,6 +88,9 @@ pub struct SafeKeeperConf {
|
||||
pub partial_backup_enabled: bool,
|
||||
pub partial_backup_timeout: Duration,
|
||||
pub disable_periodic_broker_push: bool,
|
||||
pub enable_offload: bool,
|
||||
pub delete_offloaded_wal: bool,
|
||||
pub control_file_save_interval: Duration,
|
||||
}
|
||||
|
||||
impl SafeKeeperConf {
|
||||
@@ -124,6 +130,9 @@ impl SafeKeeperConf {
|
||||
partial_backup_enabled: false,
|
||||
partial_backup_timeout: Duration::from_secs(0),
|
||||
disable_periodic_broker_push: false,
|
||||
enable_offload: false,
|
||||
delete_offloaded_wal: false,
|
||||
control_file_save_interval: Duration::from_secs(1),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -32,7 +32,7 @@ use crate::{
|
||||
routes::TimelineStatus,
|
||||
},
|
||||
safekeeper::Term,
|
||||
timeline::{get_tenant_dir, get_timeline_dir, FullAccessTimeline, Timeline, TimelineError},
|
||||
timeline::{get_tenant_dir, get_timeline_dir, Timeline, TimelineError, WalResidentTimeline},
|
||||
wal_storage::{self, open_wal_file, Storage},
|
||||
GlobalTimelines, SafeKeeperConf,
|
||||
};
|
||||
@@ -46,7 +46,7 @@ use utils::{
|
||||
|
||||
/// Stream tar archive of timeline to tx.
|
||||
#[instrument(name = "snapshot", skip_all, fields(ttid = %tli.ttid))]
|
||||
pub async fn stream_snapshot(tli: FullAccessTimeline, tx: mpsc::Sender<Result<Bytes>>) {
|
||||
pub async fn stream_snapshot(tli: WalResidentTimeline, tx: mpsc::Sender<Result<Bytes>>) {
|
||||
if let Err(e) = stream_snapshot_guts(tli, tx.clone()).await {
|
||||
// Error type/contents don't matter as they won't can't reach the client
|
||||
// (hyper likely doesn't do anything with it), but http stream will be
|
||||
@@ -66,7 +66,7 @@ pub struct SnapshotContext {
|
||||
pub flush_lsn: Lsn,
|
||||
pub wal_seg_size: usize,
|
||||
// used to remove WAL hold off in Drop.
|
||||
pub tli: FullAccessTimeline,
|
||||
pub tli: WalResidentTimeline,
|
||||
}
|
||||
|
||||
impl Drop for SnapshotContext {
|
||||
@@ -80,7 +80,7 @@ impl Drop for SnapshotContext {
|
||||
}
|
||||
|
||||
pub async fn stream_snapshot_guts(
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
tx: mpsc::Sender<Result<Bytes>>,
|
||||
) -> Result<()> {
|
||||
// tokio-tar wants Write implementor, but we have mpsc tx <Result<Bytes>>;
|
||||
@@ -135,7 +135,7 @@ pub async fn stream_snapshot_guts(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
impl FullAccessTimeline {
|
||||
impl WalResidentTimeline {
|
||||
/// Start streaming tar archive with timeline:
|
||||
/// 1) stream control file under lock;
|
||||
/// 2) hold off WAL removal;
|
||||
@@ -160,6 +160,7 @@ impl FullAccessTimeline {
|
||||
ar: &mut tokio_tar::Builder<W>,
|
||||
) -> Result<SnapshotContext> {
|
||||
let mut shared_state = self.write_shared_state().await;
|
||||
let wal_seg_size = shared_state.get_wal_seg_size();
|
||||
|
||||
let cf_path = self.get_timeline_dir().join(CONTROL_FILE_NAME);
|
||||
let mut cf = File::open(cf_path).await?;
|
||||
@@ -173,19 +174,19 @@ impl FullAccessTimeline {
|
||||
// lock and setting `wal_removal_on_hold` later, it guarantees that WAL
|
||||
// won't be removed until we're done.
|
||||
let from_lsn = min(
|
||||
shared_state.sk.state.remote_consistent_lsn,
|
||||
shared_state.sk.state.backup_lsn,
|
||||
shared_state.sk.state().remote_consistent_lsn,
|
||||
shared_state.sk.state().backup_lsn,
|
||||
);
|
||||
if from_lsn == Lsn::INVALID {
|
||||
// this is possible if snapshot is called before handling first
|
||||
// elected message
|
||||
bail!("snapshot is called on uninitialized timeline");
|
||||
}
|
||||
let from_segno = from_lsn.segment_number(shared_state.get_wal_seg_size());
|
||||
let term = shared_state.sk.get_term();
|
||||
let last_log_term = shared_state.sk.get_last_log_term();
|
||||
let from_segno = from_lsn.segment_number(wal_seg_size);
|
||||
let term = shared_state.sk.state().acceptor_state.term;
|
||||
let last_log_term = shared_state.sk.last_log_term();
|
||||
let flush_lsn = shared_state.sk.flush_lsn();
|
||||
let upto_segno = flush_lsn.segment_number(shared_state.get_wal_seg_size());
|
||||
let upto_segno = flush_lsn.segment_number(wal_seg_size);
|
||||
// have some limit on max number of segments as a sanity check
|
||||
const MAX_ALLOWED_SEGS: u64 = 1000;
|
||||
let num_segs = upto_segno - from_segno + 1;
|
||||
@@ -206,14 +207,18 @@ impl FullAccessTimeline {
|
||||
}
|
||||
shared_state.wal_removal_on_hold = true;
|
||||
|
||||
// Drop shared_state to release the lock, before calling wal_residence_guard().
|
||||
drop(shared_state);
|
||||
|
||||
let tli_copy = self.wal_residence_guard().await?;
|
||||
let bctx = SnapshotContext {
|
||||
from_segno,
|
||||
upto_segno,
|
||||
term,
|
||||
last_log_term,
|
||||
flush_lsn,
|
||||
wal_seg_size: shared_state.get_wal_seg_size(),
|
||||
tli: self.clone(),
|
||||
wal_seg_size,
|
||||
tli: tli_copy,
|
||||
};
|
||||
|
||||
Ok(bctx)
|
||||
@@ -225,8 +230,8 @@ impl FullAccessTimeline {
|
||||
/// forget this if snapshotting fails mid the way.
|
||||
pub async fn finish_snapshot(&self, bctx: &SnapshotContext) -> Result<()> {
|
||||
let shared_state = self.read_shared_state().await;
|
||||
let term = shared_state.sk.get_term();
|
||||
let last_log_term = shared_state.sk.get_last_log_term();
|
||||
let term = shared_state.sk.state().acceptor_state.term;
|
||||
let last_log_term = shared_state.sk.last_log_term();
|
||||
// There are some cases to relax this check (e.g. last_log_term might
|
||||
// change, but as long as older history is strictly part of new that's
|
||||
// fine), but there is no need to do it.
|
||||
|
||||
@@ -6,7 +6,7 @@ use crate::handler::SafekeeperPostgresHandler;
|
||||
use crate::safekeeper::AcceptorProposerMessage;
|
||||
use crate::safekeeper::ProposerAcceptorMessage;
|
||||
use crate::safekeeper::ServerInfo;
|
||||
use crate::timeline::FullAccessTimeline;
|
||||
use crate::timeline::WalResidentTimeline;
|
||||
use crate::wal_service::ConnectionId;
|
||||
use crate::GlobalTimelines;
|
||||
use anyhow::{anyhow, Context};
|
||||
@@ -213,7 +213,7 @@ impl SafekeeperPostgresHandler {
|
||||
&mut self,
|
||||
pgb: &mut PostgresBackend<IO>,
|
||||
) -> Result<(), QueryError> {
|
||||
let mut tli: Option<FullAccessTimeline> = None;
|
||||
let mut tli: Option<WalResidentTimeline> = None;
|
||||
if let Err(end) = self.handle_start_wal_push_guts(pgb, &mut tli).await {
|
||||
// Log the result and probably send it to the client, closing the stream.
|
||||
let handle_end_fut = pgb.handle_copy_stream_end(end);
|
||||
@@ -233,7 +233,7 @@ impl SafekeeperPostgresHandler {
|
||||
pub async fn handle_start_wal_push_guts<IO: AsyncRead + AsyncWrite + Unpin>(
|
||||
&mut self,
|
||||
pgb: &mut PostgresBackend<IO>,
|
||||
tli: &mut Option<FullAccessTimeline>,
|
||||
tli: &mut Option<WalResidentTimeline>,
|
||||
) -> Result<(), CopyStreamHandlerEnd> {
|
||||
// Notify the libpq client that it's allowed to send `CopyData` messages
|
||||
pgb.write_message(&BeMessage::CopyBothResponse).await?;
|
||||
@@ -269,11 +269,11 @@ impl SafekeeperPostgresHandler {
|
||||
.get_walreceivers()
|
||||
.pageserver_feedback_tx
|
||||
.subscribe();
|
||||
*tli = Some(timeline.clone());
|
||||
*tli = Some(timeline.wal_residence_guard().await?);
|
||||
|
||||
tokio::select! {
|
||||
// todo: add read|write .context to these errors
|
||||
r = network_reader.run(msg_tx, msg_rx, reply_tx, timeline.clone(), next_msg) => r,
|
||||
r = network_reader.run(msg_tx, msg_rx, reply_tx, timeline, next_msg) => r,
|
||||
r = network_write(pgb, reply_rx, pageserver_feedback_rx) => r,
|
||||
}
|
||||
} else {
|
||||
@@ -323,7 +323,7 @@ struct NetworkReader<'a, IO> {
|
||||
impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> {
|
||||
async fn read_first_message(
|
||||
&mut self,
|
||||
) -> Result<(FullAccessTimeline, ProposerAcceptorMessage), CopyStreamHandlerEnd> {
|
||||
) -> Result<(WalResidentTimeline, ProposerAcceptorMessage), CopyStreamHandlerEnd> {
|
||||
// Receive information about server to create timeline, if not yet.
|
||||
let next_msg = read_message(self.pgb_reader).await?;
|
||||
let tli = match next_msg {
|
||||
@@ -340,7 +340,7 @@ impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> {
|
||||
let tli =
|
||||
GlobalTimelines::create(self.ttid, server_info, Lsn::INVALID, Lsn::INVALID)
|
||||
.await?;
|
||||
tli.full_access_guard().await?
|
||||
tli.wal_residence_guard().await?
|
||||
}
|
||||
_ => {
|
||||
return Err(CopyStreamHandlerEnd::Other(anyhow::anyhow!(
|
||||
@@ -356,7 +356,7 @@ impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> {
|
||||
msg_tx: Sender<ProposerAcceptorMessage>,
|
||||
msg_rx: Receiver<ProposerAcceptorMessage>,
|
||||
reply_tx: Sender<AcceptorProposerMessage>,
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
next_msg: ProposerAcceptorMessage,
|
||||
) -> Result<(), CopyStreamHandlerEnd> {
|
||||
*self.acceptor_handle = Some(WalAcceptor::spawn(
|
||||
@@ -451,7 +451,7 @@ const KEEPALIVE_INTERVAL: Duration = Duration::from_secs(1);
|
||||
/// replies to reply_tx; reading from socket and writing to disk in parallel is
|
||||
/// beneficial for performance, this struct provides writing to disk part.
|
||||
pub struct WalAcceptor {
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
msg_rx: Receiver<ProposerAcceptorMessage>,
|
||||
reply_tx: Sender<AcceptorProposerMessage>,
|
||||
conn_id: Option<ConnectionId>,
|
||||
@@ -464,7 +464,7 @@ impl WalAcceptor {
|
||||
///
|
||||
/// conn_id None means WalAcceptor is used by recovery initiated at this safekeeper.
|
||||
pub fn spawn(
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
msg_rx: Receiver<ProposerAcceptorMessage>,
|
||||
reply_tx: Sender<AcceptorProposerMessage>,
|
||||
conn_id: Option<ConnectionId>,
|
||||
|
||||
@@ -21,7 +21,7 @@ use utils::{id::NodeId, lsn::Lsn, postgres_client::wal_stream_connection_config}
|
||||
|
||||
use crate::receive_wal::{WalAcceptor, REPLY_QUEUE_SIZE};
|
||||
use crate::safekeeper::{AppendRequest, AppendRequestHeader};
|
||||
use crate::timeline::FullAccessTimeline;
|
||||
use crate::timeline::WalResidentTimeline;
|
||||
use crate::{
|
||||
http::routes::TimelineStatus,
|
||||
receive_wal::MSG_QUEUE_SIZE,
|
||||
@@ -36,7 +36,7 @@ use crate::{
|
||||
/// Entrypoint for per timeline task which always runs, checking whether
|
||||
/// recovery for this safekeeper is needed and starting it if so.
|
||||
#[instrument(name = "recovery task", skip_all, fields(ttid = %tli.ttid))]
|
||||
pub async fn recovery_main(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
pub async fn recovery_main(tli: WalResidentTimeline, conf: SafeKeeperConf) {
|
||||
info!("started");
|
||||
|
||||
let cancel = tli.cancel.clone();
|
||||
@@ -66,12 +66,12 @@ pub async fn recovery_main(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
/// depending on assembled quorum (e.g. classic picture 8 from Raft paper).
|
||||
/// Thus we don't try to predict it here.
|
||||
async fn recovery_needed(
|
||||
tli: &FullAccessTimeline,
|
||||
tli: &WalResidentTimeline,
|
||||
heartbeat_timeout: Duration,
|
||||
) -> RecoveryNeededInfo {
|
||||
let ss = tli.read_shared_state().await;
|
||||
let term = ss.sk.state.acceptor_state.term;
|
||||
let last_log_term = ss.sk.get_last_log_term();
|
||||
let term = ss.sk.state().acceptor_state.term;
|
||||
let last_log_term = ss.sk.last_log_term();
|
||||
let flush_lsn = ss.sk.flush_lsn();
|
||||
// note that peers contain myself, but that's ok -- we are interested only in peers which are strictly ahead of us.
|
||||
let mut peers = ss.get_peers(heartbeat_timeout);
|
||||
@@ -195,7 +195,7 @@ impl From<&PeerInfo> for Donor {
|
||||
const CHECK_INTERVAL_MS: u64 = 2000;
|
||||
|
||||
/// Check regularly whether we need to start recovery.
|
||||
async fn recovery_main_loop(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
async fn recovery_main_loop(tli: WalResidentTimeline, conf: SafeKeeperConf) {
|
||||
let check_duration = Duration::from_millis(CHECK_INTERVAL_MS);
|
||||
loop {
|
||||
let recovery_needed_info = recovery_needed(&tli, conf.heartbeat_timeout).await;
|
||||
@@ -205,7 +205,12 @@ async fn recovery_main_loop(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
"starting recovery from donor {}: {}",
|
||||
donor.sk_id, recovery_needed_info
|
||||
);
|
||||
match recover(tli.clone(), donor, &conf).await {
|
||||
let res = tli.wal_residence_guard().await;
|
||||
if let Err(e) = res {
|
||||
warn!("failed to obtain guard: {}", e);
|
||||
continue;
|
||||
}
|
||||
match recover(res.unwrap(), donor, &conf).await {
|
||||
// Note: 'write_wal rewrites WAL written before' error is
|
||||
// expected here and might happen if compute and recovery
|
||||
// concurrently write the same data. Eventually compute
|
||||
@@ -228,7 +233,7 @@ async fn recovery_main_loop(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
/// Recover from the specified donor. Returns message explaining normal finish
|
||||
/// reason or error.
|
||||
async fn recover(
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
donor: &Donor,
|
||||
conf: &SafeKeeperConf,
|
||||
) -> anyhow::Result<String> {
|
||||
@@ -314,7 +319,7 @@ async fn recover(
|
||||
|
||||
// Pull WAL from donor, assuming handshake is already done.
|
||||
async fn recovery_stream(
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
donor: &Donor,
|
||||
start_streaming_at: Lsn,
|
||||
conf: &SafeKeeperConf,
|
||||
@@ -364,10 +369,10 @@ async fn recovery_stream(
|
||||
// As in normal walreceiver, do networking and writing to disk in parallel.
|
||||
let (msg_tx, msg_rx) = channel(MSG_QUEUE_SIZE);
|
||||
let (reply_tx, reply_rx) = channel(REPLY_QUEUE_SIZE);
|
||||
let wa = WalAcceptor::spawn(tli.clone(), msg_rx, reply_tx, None);
|
||||
let wa = WalAcceptor::spawn(tli.wal_residence_guard().await?, msg_rx, reply_tx, None);
|
||||
|
||||
let res = tokio::select! {
|
||||
r = network_io(physical_stream, msg_tx, donor.clone(), tli.clone(), conf.clone()) => r,
|
||||
r = network_io(physical_stream, msg_tx, donor.clone(), tli, conf.clone()) => r,
|
||||
r = read_replies(reply_rx, donor.term) => r.map(|()| None),
|
||||
};
|
||||
|
||||
@@ -398,7 +403,7 @@ async fn network_io(
|
||||
physical_stream: ReplicationStream,
|
||||
msg_tx: Sender<ProposerAcceptorMessage>,
|
||||
donor: Donor,
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
conf: SafeKeeperConf,
|
||||
) -> anyhow::Result<Option<String>> {
|
||||
let mut physical_stream = pin!(physical_stream);
|
||||
|
||||
@@ -8,7 +8,7 @@ use crate::timeline_manager::StateSnapshot;
|
||||
/// While it is safe to use inmem values for determining horizon,
|
||||
/// we use persistent to make possible normal states less surprising.
|
||||
/// All segments covering LSNs before horizon_lsn can be removed.
|
||||
pub fn calc_horizon_lsn(state: &StateSnapshot, extra_horizon_lsn: Option<Lsn>) -> Lsn {
|
||||
pub(crate) fn calc_horizon_lsn(state: &StateSnapshot, extra_horizon_lsn: Option<Lsn>) -> Lsn {
|
||||
use std::cmp::min;
|
||||
|
||||
let mut horizon_lsn = min(
|
||||
|
||||
@@ -499,7 +499,11 @@ where
|
||||
/// Accepts a control file storage containing the safekeeper state.
|
||||
/// State must be initialized, i.e. contain filled `tenant_id`, `timeline_id`
|
||||
/// and `server` (`wal_seg_size` inside it) fields.
|
||||
pub fn new(state: CTRL, wal_store: WAL, node_id: NodeId) -> Result<SafeKeeper<CTRL, WAL>> {
|
||||
pub fn new(
|
||||
state: TimelineState<CTRL>,
|
||||
wal_store: WAL,
|
||||
node_id: NodeId,
|
||||
) -> Result<SafeKeeper<CTRL, WAL>> {
|
||||
if state.tenant_id == TenantId::from([0u8; 16])
|
||||
|| state.timeline_id == TimelineId::from([0u8; 16])
|
||||
{
|
||||
@@ -512,7 +516,7 @@ where
|
||||
|
||||
Ok(SafeKeeper {
|
||||
term_start_lsn: Lsn(0),
|
||||
state: TimelineState::new(state),
|
||||
state,
|
||||
wal_store,
|
||||
node_id,
|
||||
})
|
||||
@@ -526,11 +530,6 @@ where
|
||||
.up_to(self.flush_lsn())
|
||||
}
|
||||
|
||||
/// Get current term.
|
||||
pub fn get_term(&self) -> Term {
|
||||
self.state.acceptor_state.term
|
||||
}
|
||||
|
||||
pub fn get_last_log_term(&self) -> Term {
|
||||
self.state
|
||||
.acceptor_state
|
||||
@@ -912,10 +911,8 @@ where
|
||||
)))
|
||||
}
|
||||
|
||||
/// Update timeline state with peer safekeeper data.
|
||||
/// Update commit_lsn from peer safekeeper data.
|
||||
pub async fn record_safekeeper_info(&mut self, sk_info: &SafekeeperTimelineInfo) -> Result<()> {
|
||||
let mut sync_control_file = false;
|
||||
|
||||
if (Lsn(sk_info.commit_lsn) != Lsn::INVALID) && (sk_info.last_log_term != INVALID_TERM) {
|
||||
// Note: the check is too restrictive, generally we can update local
|
||||
// commit_lsn if our history matches (is part of) history of advanced
|
||||
@@ -924,29 +921,6 @@ where
|
||||
self.update_commit_lsn(Lsn(sk_info.commit_lsn)).await?;
|
||||
}
|
||||
}
|
||||
|
||||
self.state.inmem.backup_lsn = max(Lsn(sk_info.backup_lsn), self.state.inmem.backup_lsn);
|
||||
sync_control_file |= self.state.backup_lsn + (self.state.server.wal_seg_size as u64)
|
||||
< self.state.inmem.backup_lsn;
|
||||
|
||||
self.state.inmem.remote_consistent_lsn = max(
|
||||
Lsn(sk_info.remote_consistent_lsn),
|
||||
self.state.inmem.remote_consistent_lsn,
|
||||
);
|
||||
sync_control_file |= self.state.remote_consistent_lsn
|
||||
+ (self.state.server.wal_seg_size as u64)
|
||||
< self.state.inmem.remote_consistent_lsn;
|
||||
|
||||
self.state.inmem.peer_horizon_lsn = max(
|
||||
Lsn(sk_info.peer_horizon_lsn),
|
||||
self.state.inmem.peer_horizon_lsn,
|
||||
);
|
||||
sync_control_file |= self.state.peer_horizon_lsn + (self.state.server.wal_seg_size as u64)
|
||||
< self.state.inmem.peer_horizon_lsn;
|
||||
|
||||
if sync_control_file {
|
||||
self.state.flush().await?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -1039,7 +1013,7 @@ mod tests {
|
||||
persisted_state: test_sk_state(),
|
||||
};
|
||||
let wal_store = DummyWalStore { lsn: Lsn(0) };
|
||||
let mut sk = SafeKeeper::new(storage, wal_store, NodeId(0)).unwrap();
|
||||
let mut sk = SafeKeeper::new(TimelineState::new(storage), wal_store, NodeId(0)).unwrap();
|
||||
|
||||
// check voting for 1 is ok
|
||||
let vote_request = ProposerAcceptorMessage::VoteRequest(VoteRequest { term: 1 });
|
||||
@@ -1055,7 +1029,7 @@ mod tests {
|
||||
persisted_state: state,
|
||||
};
|
||||
|
||||
sk = SafeKeeper::new(storage, sk.wal_store, NodeId(0)).unwrap();
|
||||
sk = SafeKeeper::new(TimelineState::new(storage), sk.wal_store, NodeId(0)).unwrap();
|
||||
|
||||
// and ensure voting second time for 1 is not ok
|
||||
vote_resp = sk.process_msg(&vote_request).await;
|
||||
@@ -1072,7 +1046,7 @@ mod tests {
|
||||
};
|
||||
let wal_store = DummyWalStore { lsn: Lsn(0) };
|
||||
|
||||
let mut sk = SafeKeeper::new(storage, wal_store, NodeId(0)).unwrap();
|
||||
let mut sk = SafeKeeper::new(TimelineState::new(storage), wal_store, NodeId(0)).unwrap();
|
||||
|
||||
let mut ar_hdr = AppendRequestHeader {
|
||||
term: 1,
|
||||
|
||||
@@ -5,7 +5,7 @@ use crate::handler::SafekeeperPostgresHandler;
|
||||
use crate::metrics::RECEIVED_PS_FEEDBACKS;
|
||||
use crate::receive_wal::WalReceivers;
|
||||
use crate::safekeeper::{Term, TermLsn};
|
||||
use crate::timeline::FullAccessTimeline;
|
||||
use crate::timeline::WalResidentTimeline;
|
||||
use crate::wal_service::ConnectionId;
|
||||
use crate::wal_storage::WalReader;
|
||||
use crate::GlobalTimelines;
|
||||
@@ -387,10 +387,10 @@ impl SafekeeperPostgresHandler {
|
||||
term: Option<Term>,
|
||||
) -> Result<(), QueryError> {
|
||||
let tli = GlobalTimelines::get(self.ttid).map_err(|e| QueryError::Other(e.into()))?;
|
||||
let full_access = tli.full_access_guard().await?;
|
||||
let residence_guard = tli.wal_residence_guard().await?;
|
||||
|
||||
if let Err(end) = self
|
||||
.handle_start_replication_guts(pgb, start_pos, term, full_access)
|
||||
.handle_start_replication_guts(pgb, start_pos, term, residence_guard)
|
||||
.await
|
||||
{
|
||||
let info = tli.get_safekeeper_info(&self.conf).await;
|
||||
@@ -407,7 +407,7 @@ impl SafekeeperPostgresHandler {
|
||||
pgb: &mut PostgresBackend<IO>,
|
||||
start_pos: Lsn,
|
||||
term: Option<Term>,
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
) -> Result<(), CopyStreamHandlerEnd> {
|
||||
let appname = self.appname.clone();
|
||||
|
||||
@@ -458,7 +458,8 @@ impl SafekeeperPostgresHandler {
|
||||
|
||||
let mut sender = WalSender {
|
||||
pgb,
|
||||
tli: tli.clone(),
|
||||
// should succeed since we're already holding another guard
|
||||
tli: tli.wal_residence_guard().await?,
|
||||
appname,
|
||||
start_pos,
|
||||
end_pos,
|
||||
@@ -527,7 +528,7 @@ impl EndWatch {
|
||||
/// A half driving sending WAL.
|
||||
struct WalSender<'a, IO> {
|
||||
pgb: &'a mut PostgresBackend<IO>,
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
appname: Option<String>,
|
||||
// Position since which we are sending next chunk.
|
||||
start_pos: Lsn,
|
||||
@@ -736,7 +737,7 @@ impl<IO: AsyncRead + AsyncWrite + Unpin> WalSender<'_, IO> {
|
||||
struct ReplyReader<IO> {
|
||||
reader: PostgresBackendReader<IO>,
|
||||
ws_guard: Arc<WalSenderGuard>,
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
}
|
||||
|
||||
impl<IO: AsyncRead + AsyncWrite + Unpin> ReplyReader<IO> {
|
||||
|
||||
@@ -31,12 +31,15 @@ use crate::safekeeper::{
|
||||
INVALID_TERM,
|
||||
};
|
||||
use crate::send_wal::WalSenders;
|
||||
use crate::state::{TimelineMemState, TimelinePersistentState};
|
||||
use crate::state::{EvictionState, TimelineMemState, TimelinePersistentState, TimelineState};
|
||||
use crate::timeline_guard::ResidenceGuard;
|
||||
use crate::timeline_manager::{AtomicStatus, ManagerCtl};
|
||||
use crate::timelines_set::TimelinesSet;
|
||||
use crate::wal_backup::{self};
|
||||
use crate::wal_backup_partial::PartialRemoteSegment;
|
||||
use crate::{control_file, safekeeper::UNKNOWN_SERVER_VERSION};
|
||||
|
||||
use crate::metrics::FullTimelineInfo;
|
||||
use crate::metrics::{FullTimelineInfo, WalStorageMetrics};
|
||||
use crate::wal_storage::{Storage as wal_storage_iface, WalReader};
|
||||
use crate::{debug_dump, timeline_manager, wal_storage};
|
||||
use crate::{GlobalTimelines, SafeKeeperConf};
|
||||
@@ -132,8 +135,9 @@ impl<'a> DerefMut for WriteGuardSharedState<'a> {
|
||||
|
||||
impl<'a> Drop for WriteGuardSharedState<'a> {
|
||||
fn drop(&mut self) {
|
||||
let term_flush_lsn = TermLsn::from((self.guard.sk.get_term(), self.guard.sk.flush_lsn()));
|
||||
let commit_lsn = self.guard.sk.state.inmem.commit_lsn;
|
||||
let term_flush_lsn =
|
||||
TermLsn::from((self.guard.sk.last_log_term(), self.guard.sk.flush_lsn()));
|
||||
let commit_lsn = self.guard.sk.state().inmem.commit_lsn;
|
||||
|
||||
let _ = self.tli.term_flush_lsn_watch_tx.send_if_modified(|old| {
|
||||
if *old != term_flush_lsn {
|
||||
@@ -162,10 +166,150 @@ impl<'a> Drop for WriteGuardSharedState<'a> {
|
||||
}
|
||||
}
|
||||
|
||||
/// This structure is stored in shared state and represents the state of the timeline.
|
||||
/// Usually it holds SafeKeeper, but it also supports offloaded timeline state. In this
|
||||
/// case, SafeKeeper is not available (because WAL is not present on disk) and all
|
||||
/// operations can be done only with control file.
|
||||
pub enum StateSK {
|
||||
Loaded(SafeKeeper<control_file::FileStorage, wal_storage::PhysicalStorage>),
|
||||
Offloaded(Box<TimelineState<control_file::FileStorage>>),
|
||||
// Not used, required for moving between states.
|
||||
Empty,
|
||||
}
|
||||
|
||||
impl StateSK {
|
||||
pub fn flush_lsn(&self) -> Lsn {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk.wal_store.flush_lsn(),
|
||||
StateSK::Offloaded(state) => match state.eviction_state {
|
||||
EvictionState::Offloaded(flush_lsn) => flush_lsn,
|
||||
_ => panic!("StateSK::Offloaded mismatches with eviction_state from control_file"),
|
||||
},
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get a reference to the control file's timeline state.
|
||||
pub fn state(&self) -> &TimelineState<control_file::FileStorage> {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => &sk.state,
|
||||
StateSK::Offloaded(ref s) => s,
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn state_mut(&mut self) -> &mut TimelineState<control_file::FileStorage> {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => &mut sk.state,
|
||||
StateSK::Offloaded(ref mut s) => s,
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn last_log_term(&self) -> Term {
|
||||
self.state()
|
||||
.acceptor_state
|
||||
.get_last_log_term(self.flush_lsn())
|
||||
}
|
||||
|
||||
/// Close open WAL files to release FDs.
|
||||
fn close_wal_store(&mut self) {
|
||||
if let StateSK::Loaded(sk) = self {
|
||||
sk.wal_store.close();
|
||||
}
|
||||
}
|
||||
|
||||
/// Update timeline state with peer safekeeper data.
|
||||
pub async fn record_safekeeper_info(&mut self, sk_info: &SafekeeperTimelineInfo) -> Result<()> {
|
||||
// update commit_lsn if safekeeper is loaded
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk.record_safekeeper_info(sk_info).await?,
|
||||
StateSK::Offloaded(_) => {}
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
|
||||
// update everything else, including remote_consistent_lsn and backup_lsn
|
||||
let mut sync_control_file = false;
|
||||
let state = self.state_mut();
|
||||
let wal_seg_size = state.server.wal_seg_size as u64;
|
||||
|
||||
state.inmem.backup_lsn = max(Lsn(sk_info.backup_lsn), state.inmem.backup_lsn);
|
||||
sync_control_file |= state.backup_lsn + wal_seg_size < state.inmem.backup_lsn;
|
||||
|
||||
state.inmem.remote_consistent_lsn = max(
|
||||
Lsn(sk_info.remote_consistent_lsn),
|
||||
state.inmem.remote_consistent_lsn,
|
||||
);
|
||||
sync_control_file |=
|
||||
state.remote_consistent_lsn + wal_seg_size < state.inmem.remote_consistent_lsn;
|
||||
|
||||
state.inmem.peer_horizon_lsn =
|
||||
max(Lsn(sk_info.peer_horizon_lsn), state.inmem.peer_horizon_lsn);
|
||||
sync_control_file |= state.peer_horizon_lsn + wal_seg_size < state.inmem.peer_horizon_lsn;
|
||||
|
||||
if sync_control_file {
|
||||
state.flush().await?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Previously known as epoch_start_lsn. Needed only for reference in some APIs.
|
||||
pub fn term_start_lsn(&self) -> Lsn {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk.term_start_lsn,
|
||||
StateSK::Offloaded(_) => Lsn(0),
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Used for metrics only.
|
||||
pub fn wal_storage_metrics(&self) -> WalStorageMetrics {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk.wal_store.get_metrics(),
|
||||
StateSK::Offloaded(_) => WalStorageMetrics::default(),
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns WAL storage internal LSNs for debug dump.
|
||||
pub fn wal_storage_internal_state(&self) -> (Lsn, Lsn, Lsn, bool) {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk.wal_store.internal_state(),
|
||||
StateSK::Offloaded(_) => {
|
||||
let flush_lsn = self.flush_lsn();
|
||||
(flush_lsn, flush_lsn, flush_lsn, false)
|
||||
}
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Access to SafeKeeper object. Panics if offloaded, should be good to use from WalResidentTimeline.
|
||||
pub fn safekeeper(
|
||||
&mut self,
|
||||
) -> &mut SafeKeeper<control_file::FileStorage, wal_storage::PhysicalStorage> {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk,
|
||||
StateSK::Offloaded(_) => {
|
||||
panic!("safekeeper is offloaded, cannot be used")
|
||||
}
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Moves control file's state structure out of the enum. Used to switch states.
|
||||
fn take_state(self) -> TimelineState<control_file::FileStorage> {
|
||||
match self {
|
||||
StateSK::Loaded(sk) => sk.state,
|
||||
StateSK::Offloaded(state) => *state,
|
||||
StateSK::Empty => unreachable!(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Shared state associated with database instance
|
||||
pub struct SharedState {
|
||||
/// Safekeeper object
|
||||
pub(crate) sk: SafeKeeper<control_file::FileStorage, wal_storage::PhysicalStorage>,
|
||||
pub(crate) sk: StateSK,
|
||||
/// In memory list containing state of peers sent in latest messages from them.
|
||||
pub(crate) peers_info: PeersInfo,
|
||||
// True value hinders old WAL removal; this is used by snapshotting. We
|
||||
@@ -203,10 +347,10 @@ impl SharedState {
|
||||
control_file::FileStorage::create_new(timeline_dir.clone(), conf, state)?;
|
||||
let wal_store =
|
||||
wal_storage::PhysicalStorage::new(ttid, timeline_dir, conf, &control_store)?;
|
||||
let sk = SafeKeeper::new(control_store, wal_store, conf.my_id)?;
|
||||
let sk = SafeKeeper::new(TimelineState::new(control_store), wal_store, conf.my_id)?;
|
||||
|
||||
Ok(Self {
|
||||
sk,
|
||||
sk: StateSK::Loaded(sk),
|
||||
peers_info: PeersInfo(vec![]),
|
||||
wal_removal_on_hold: false,
|
||||
})
|
||||
@@ -220,18 +364,30 @@ impl SharedState {
|
||||
bail!(TimelineError::UninitializedWalSegSize(*ttid));
|
||||
}
|
||||
|
||||
let wal_store =
|
||||
wal_storage::PhysicalStorage::new(ttid, timeline_dir, conf, &control_store)?;
|
||||
let sk = match control_store.eviction_state {
|
||||
EvictionState::Present => {
|
||||
let wal_store =
|
||||
wal_storage::PhysicalStorage::new(ttid, timeline_dir, conf, &control_store)?;
|
||||
StateSK::Loaded(SafeKeeper::new(
|
||||
TimelineState::new(control_store),
|
||||
wal_store,
|
||||
conf.my_id,
|
||||
)?)
|
||||
}
|
||||
EvictionState::Offloaded(_) => {
|
||||
StateSK::Offloaded(Box::new(TimelineState::new(control_store)))
|
||||
}
|
||||
};
|
||||
|
||||
Ok(Self {
|
||||
sk: SafeKeeper::new(control_store, wal_store, conf.my_id)?,
|
||||
sk,
|
||||
peers_info: PeersInfo(vec![]),
|
||||
wal_removal_on_hold: false,
|
||||
})
|
||||
}
|
||||
|
||||
pub(crate) fn get_wal_seg_size(&self) -> usize {
|
||||
self.sk.state.server.wal_seg_size as usize
|
||||
self.sk.state().server.wal_seg_size as usize
|
||||
}
|
||||
|
||||
fn get_safekeeper_info(
|
||||
@@ -246,20 +402,20 @@ impl SharedState {
|
||||
tenant_id: ttid.tenant_id.as_ref().to_owned(),
|
||||
timeline_id: ttid.timeline_id.as_ref().to_owned(),
|
||||
}),
|
||||
term: self.sk.state.acceptor_state.term,
|
||||
last_log_term: self.sk.get_last_log_term(),
|
||||
term: self.sk.state().acceptor_state.term,
|
||||
last_log_term: self.sk.last_log_term(),
|
||||
flush_lsn: self.sk.flush_lsn().0,
|
||||
// note: this value is not flushed to control file yet and can be lost
|
||||
commit_lsn: self.sk.state.inmem.commit_lsn.0,
|
||||
remote_consistent_lsn: self.sk.state.inmem.remote_consistent_lsn.0,
|
||||
peer_horizon_lsn: self.sk.state.inmem.peer_horizon_lsn.0,
|
||||
commit_lsn: self.sk.state().inmem.commit_lsn.0,
|
||||
remote_consistent_lsn: self.sk.state().inmem.remote_consistent_lsn.0,
|
||||
peer_horizon_lsn: self.sk.state().inmem.peer_horizon_lsn.0,
|
||||
safekeeper_connstr: conf
|
||||
.advertise_pg_addr
|
||||
.to_owned()
|
||||
.unwrap_or(conf.listen_pg_addr.clone()),
|
||||
http_connstr: conf.listen_http_addr.to_owned(),
|
||||
backup_lsn: self.sk.state.inmem.backup_lsn.0,
|
||||
local_start_lsn: self.sk.state.local_start_lsn.0,
|
||||
backup_lsn: self.sk.state().inmem.backup_lsn.0,
|
||||
local_start_lsn: self.sk.state().local_start_lsn.0,
|
||||
availability_zone: conf.availability_zone.clone(),
|
||||
standby_horizon: standby_apply_lsn.0,
|
||||
}
|
||||
@@ -335,6 +491,7 @@ pub struct Timeline {
|
||||
walsenders: Arc<WalSenders>,
|
||||
walreceivers: Arc<WalReceivers>,
|
||||
timeline_dir: Utf8PathBuf,
|
||||
manager_ctl: ManagerCtl,
|
||||
|
||||
/// Delete/cancel will trigger this, background tasks should drop out as soon as it fires
|
||||
pub(crate) cancel: CancellationToken,
|
||||
@@ -343,6 +500,7 @@ pub struct Timeline {
|
||||
pub(crate) broker_active: AtomicBool,
|
||||
pub(crate) wal_backup_active: AtomicBool,
|
||||
pub(crate) last_removed_segno: AtomicU64,
|
||||
pub(crate) mgr_status: AtomicStatus,
|
||||
}
|
||||
|
||||
impl Timeline {
|
||||
@@ -352,9 +510,9 @@ impl Timeline {
|
||||
|
||||
let shared_state = SharedState::restore(conf, &ttid)?;
|
||||
let (commit_lsn_watch_tx, commit_lsn_watch_rx) =
|
||||
watch::channel(shared_state.sk.state.commit_lsn);
|
||||
watch::channel(shared_state.sk.state().commit_lsn);
|
||||
let (term_flush_lsn_watch_tx, term_flush_lsn_watch_rx) = watch::channel(TermLsn::from((
|
||||
shared_state.sk.get_term(),
|
||||
shared_state.sk.last_log_term(),
|
||||
shared_state.sk.flush_lsn(),
|
||||
)));
|
||||
let (shared_state_version_tx, shared_state_version_rx) = watch::channel(0);
|
||||
@@ -373,9 +531,11 @@ impl Timeline {
|
||||
walreceivers,
|
||||
cancel: CancellationToken::default(),
|
||||
timeline_dir: get_timeline_dir(conf, &ttid),
|
||||
manager_ctl: ManagerCtl::new(),
|
||||
broker_active: AtomicBool::new(false),
|
||||
wal_backup_active: AtomicBool::new(false),
|
||||
last_removed_segno: AtomicU64::new(0),
|
||||
mgr_status: AtomicStatus::new(),
|
||||
})
|
||||
}
|
||||
|
||||
@@ -409,9 +569,11 @@ impl Timeline {
|
||||
walreceivers,
|
||||
cancel: CancellationToken::default(),
|
||||
timeline_dir: get_timeline_dir(conf, &ttid),
|
||||
manager_ctl: ManagerCtl::new(),
|
||||
broker_active: AtomicBool::new(false),
|
||||
wal_backup_active: AtomicBool::new(false),
|
||||
last_removed_segno: AtomicU64::new(0),
|
||||
mgr_status: AtomicStatus::new(),
|
||||
})
|
||||
}
|
||||
|
||||
@@ -442,7 +604,7 @@ impl Timeline {
|
||||
fs::create_dir_all(&self.timeline_dir).await?;
|
||||
|
||||
// Write timeline to disk and start background tasks.
|
||||
if let Err(e) = shared_state.sk.state.flush().await {
|
||||
if let Err(e) = shared_state.sk.state_mut().flush().await {
|
||||
// Bootstrap failed, cancel timeline and remove timeline directory.
|
||||
self.cancel(shared_state);
|
||||
|
||||
@@ -465,12 +627,16 @@ impl Timeline {
|
||||
conf: &SafeKeeperConf,
|
||||
broker_active_set: Arc<TimelinesSet>,
|
||||
) {
|
||||
let (tx, rx) = self.manager_ctl.bootstrap_manager();
|
||||
|
||||
// Start manager task which will monitor timeline state and update
|
||||
// background tasks.
|
||||
tokio::spawn(timeline_manager::main_task(
|
||||
self.clone(),
|
||||
ManagerTimeline { tli: self.clone() },
|
||||
conf.clone(),
|
||||
broker_active_set,
|
||||
tx,
|
||||
rx,
|
||||
));
|
||||
}
|
||||
|
||||
@@ -507,7 +673,7 @@ impl Timeline {
|
||||
self.cancel.cancel();
|
||||
// Close associated FDs. Nobody will be able to touch timeline data once
|
||||
// it is cancelled, so WAL storage won't be opened again.
|
||||
shared_state.sk.wal_store.close();
|
||||
shared_state.sk.close_wal_store();
|
||||
}
|
||||
|
||||
/// Returns if timeline is cancelled.
|
||||
@@ -547,12 +713,15 @@ impl Timeline {
|
||||
/// Returns state of the timeline.
|
||||
pub async fn get_state(&self) -> (TimelineMemState, TimelinePersistentState) {
|
||||
let state = self.read_shared_state().await;
|
||||
(state.sk.state.inmem.clone(), state.sk.state.clone())
|
||||
(
|
||||
state.sk.state().inmem.clone(),
|
||||
TimelinePersistentState::clone(state.sk.state()),
|
||||
)
|
||||
}
|
||||
|
||||
/// Returns latest backup_lsn.
|
||||
pub async fn get_wal_backup_lsn(&self) -> Lsn {
|
||||
self.read_shared_state().await.sk.state.inmem.backup_lsn
|
||||
self.read_shared_state().await.sk.state().inmem.backup_lsn
|
||||
}
|
||||
|
||||
/// Sets backup_lsn to the given value.
|
||||
@@ -562,7 +731,7 @@ impl Timeline {
|
||||
}
|
||||
|
||||
let mut state = self.write_shared_state().await;
|
||||
state.sk.state.inmem.backup_lsn = max(state.sk.state.inmem.backup_lsn, backup_lsn);
|
||||
state.sk.state_mut().inmem.backup_lsn = max(state.sk.state().inmem.backup_lsn, backup_lsn);
|
||||
// we should check whether to shut down offloader, but this will be done
|
||||
// soon by peer communication anyway.
|
||||
Ok(())
|
||||
@@ -604,7 +773,7 @@ impl Timeline {
|
||||
|
||||
/// Returns flush_lsn.
|
||||
pub async fn get_flush_lsn(&self) -> Lsn {
|
||||
self.read_shared_state().await.sk.wal_store.flush_lsn()
|
||||
self.read_shared_state().await.sk.flush_lsn()
|
||||
}
|
||||
|
||||
/// Gather timeline data for metrics.
|
||||
@@ -623,11 +792,11 @@ impl Timeline {
|
||||
timeline_is_active: self.broker_active.load(Ordering::Relaxed),
|
||||
num_computes: self.walreceivers.get_num() as u32,
|
||||
last_removed_segno: self.last_removed_segno.load(Ordering::Relaxed),
|
||||
epoch_start_lsn: state.sk.term_start_lsn,
|
||||
mem_state: state.sk.state.inmem.clone(),
|
||||
persisted_state: state.sk.state.clone(),
|
||||
flush_lsn: state.sk.wal_store.flush_lsn(),
|
||||
wal_storage: state.sk.wal_store.get_metrics(),
|
||||
epoch_start_lsn: state.sk.term_start_lsn(),
|
||||
mem_state: state.sk.state().inmem.clone(),
|
||||
persisted_state: TimelinePersistentState::clone(state.sk.state()),
|
||||
flush_lsn: state.sk.flush_lsn(),
|
||||
wal_storage: state.sk.wal_storage_metrics(),
|
||||
})
|
||||
}
|
||||
|
||||
@@ -636,7 +805,7 @@ impl Timeline {
|
||||
let state = self.read_shared_state().await;
|
||||
|
||||
let (write_lsn, write_record_lsn, flush_lsn, file_open) =
|
||||
state.sk.wal_store.internal_state();
|
||||
state.sk.wal_storage_internal_state();
|
||||
|
||||
debug_dump::Memory {
|
||||
is_cancelled: self.is_cancelled(),
|
||||
@@ -646,8 +815,9 @@ impl Timeline {
|
||||
active: self.broker_active.load(Ordering::Relaxed),
|
||||
num_computes: self.walreceivers.get_num() as u32,
|
||||
last_removed_segno: self.last_removed_segno.load(Ordering::Relaxed),
|
||||
epoch_start_lsn: state.sk.term_start_lsn,
|
||||
mem_state: state.sk.state.inmem.clone(),
|
||||
epoch_start_lsn: state.sk.term_start_lsn(),
|
||||
mem_state: state.sk.state().inmem.clone(),
|
||||
mgr_status: self.mgr_status.get(),
|
||||
write_lsn,
|
||||
write_record_lsn,
|
||||
flush_lsn,
|
||||
@@ -661,34 +831,77 @@ impl Timeline {
|
||||
f: impl FnOnce(&mut TimelinePersistentState) -> Result<T>,
|
||||
) -> Result<T> {
|
||||
let mut state = self.write_shared_state().await;
|
||||
let mut persistent_state = state.sk.state.start_change();
|
||||
let mut persistent_state = state.sk.state_mut().start_change();
|
||||
// If f returns error, we abort the change and don't persist anything.
|
||||
let res = f(&mut persistent_state)?;
|
||||
// If persisting fails, we abort the change and return error.
|
||||
state.sk.state.finish_change(&persistent_state).await?;
|
||||
state
|
||||
.sk
|
||||
.state_mut()
|
||||
.finish_change(&persistent_state)
|
||||
.await?;
|
||||
Ok(res)
|
||||
}
|
||||
|
||||
/// Get the timeline guard for reading/writing WAL files.
|
||||
/// TODO: if WAL files are not present on disk (evicted), they will be
|
||||
/// downloaded from S3. Also there will logic for preventing eviction
|
||||
/// while someone is holding FullAccessTimeline guard.
|
||||
pub async fn full_access_guard(self: &Arc<Self>) -> Result<FullAccessTimeline> {
|
||||
/// If WAL files are not present on disk (evicted), they will be automatically
|
||||
/// downloaded from remote storage. This is done in the manager task, which is
|
||||
/// responsible for issuing all guards.
|
||||
///
|
||||
/// NB: don't use this function from timeline_manager, it will deadlock.
|
||||
/// NB: don't use this function while holding shared_state lock.
|
||||
pub async fn wal_residence_guard(self: &Arc<Self>) -> Result<WalResidentTimeline> {
|
||||
if self.is_cancelled() {
|
||||
bail!(TimelineError::Cancelled(self.ttid));
|
||||
}
|
||||
Ok(FullAccessTimeline { tli: self.clone() })
|
||||
|
||||
debug!("requesting WalResidentTimeline guard");
|
||||
|
||||
// Wait 5 seconds for the guard to be acquired, should be enough for uneviction.
|
||||
// If it times out, most likely there is a deadlock in the manager task.
|
||||
let res = tokio::time::timeout(
|
||||
Duration::from_secs(5),
|
||||
self.manager_ctl.wal_residence_guard(),
|
||||
)
|
||||
.await;
|
||||
|
||||
let guard = match res {
|
||||
Ok(Ok(guard)) => guard,
|
||||
Ok(Err(e)) => {
|
||||
warn!(
|
||||
"error while acquiring WalResidentTimeline guard (current state {:?}): {}",
|
||||
self.mgr_status.get(),
|
||||
e
|
||||
);
|
||||
return Err(e);
|
||||
}
|
||||
Err(_) => {
|
||||
warn!(
|
||||
"timeout while acquiring WalResidentTimeline guard (current state {:?})",
|
||||
self.mgr_status.get()
|
||||
);
|
||||
anyhow::bail!("timeout while acquiring WalResidentTimeline guard");
|
||||
}
|
||||
};
|
||||
|
||||
Ok(WalResidentTimeline::new(self.clone(), guard))
|
||||
}
|
||||
}
|
||||
|
||||
/// This is a guard that allows to read/write disk timeline state.
|
||||
/// All tasks that are using the disk should use this guard.
|
||||
#[derive(Clone)]
|
||||
pub struct FullAccessTimeline {
|
||||
/// All tasks that are trying to read/write WAL from disk should use this guard.
|
||||
pub struct WalResidentTimeline {
|
||||
pub tli: Arc<Timeline>,
|
||||
_guard: ResidenceGuard,
|
||||
}
|
||||
|
||||
impl Deref for FullAccessTimeline {
|
||||
impl WalResidentTimeline {
|
||||
pub fn new(tli: Arc<Timeline>, _guard: ResidenceGuard) -> Self {
|
||||
WalResidentTimeline { tli, _guard }
|
||||
}
|
||||
}
|
||||
|
||||
impl Deref for WalResidentTimeline {
|
||||
type Target = Arc<Timeline>;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
@@ -696,7 +909,7 @@ impl Deref for FullAccessTimeline {
|
||||
}
|
||||
}
|
||||
|
||||
impl FullAccessTimeline {
|
||||
impl WalResidentTimeline {
|
||||
/// Returns true if walsender should stop sending WAL to pageserver. We
|
||||
/// terminate it if remote_consistent_lsn reached commit_lsn and there is no
|
||||
/// computes. While there might be nothing to stream already, we learn about
|
||||
@@ -708,8 +921,8 @@ impl FullAccessTimeline {
|
||||
}
|
||||
let shared_state = self.read_shared_state().await;
|
||||
if self.walreceivers.get_num() == 0 {
|
||||
return shared_state.sk.state.inmem.commit_lsn == Lsn(0) || // no data at all yet
|
||||
reported_remote_consistent_lsn >= shared_state.sk.state.inmem.commit_lsn;
|
||||
return shared_state.sk.state().inmem.commit_lsn == Lsn(0) || // no data at all yet
|
||||
reported_remote_consistent_lsn >= shared_state.sk.state().inmem.commit_lsn;
|
||||
}
|
||||
false
|
||||
}
|
||||
@@ -717,11 +930,11 @@ impl FullAccessTimeline {
|
||||
/// Ensure that current term is t, erroring otherwise, and lock the state.
|
||||
pub async fn acquire_term(&self, t: Term) -> Result<ReadGuardSharedState> {
|
||||
let ss = self.read_shared_state().await;
|
||||
if ss.sk.state.acceptor_state.term != t {
|
||||
if ss.sk.state().acceptor_state.term != t {
|
||||
bail!(
|
||||
"failed to acquire term {}, current term {}",
|
||||
t,
|
||||
ss.sk.state.acceptor_state.term
|
||||
ss.sk.state().acceptor_state.term
|
||||
);
|
||||
}
|
||||
Ok(ss)
|
||||
@@ -739,7 +952,7 @@ impl FullAccessTimeline {
|
||||
let mut rmsg: Option<AcceptorProposerMessage>;
|
||||
{
|
||||
let mut shared_state = self.write_shared_state().await;
|
||||
rmsg = shared_state.sk.process_msg(msg).await?;
|
||||
rmsg = shared_state.sk.safekeeper().process_msg(msg).await?;
|
||||
|
||||
// if this is AppendResponse, fill in proper hot standby feedback.
|
||||
if let Some(AcceptorProposerMessage::AppendResponse(ref mut resp)) = rmsg {
|
||||
@@ -769,8 +982,141 @@ impl FullAccessTimeline {
|
||||
/// Update in memory remote consistent lsn.
|
||||
pub async fn update_remote_consistent_lsn(&self, candidate: Lsn) {
|
||||
let mut shared_state = self.write_shared_state().await;
|
||||
shared_state.sk.state.inmem.remote_consistent_lsn =
|
||||
max(shared_state.sk.state.inmem.remote_consistent_lsn, candidate);
|
||||
shared_state.sk.state_mut().inmem.remote_consistent_lsn = max(
|
||||
shared_state.sk.state().inmem.remote_consistent_lsn,
|
||||
candidate,
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/// This struct contains methods that are used by timeline manager task.
|
||||
pub(crate) struct ManagerTimeline {
|
||||
pub(crate) tli: Arc<Timeline>,
|
||||
}
|
||||
|
||||
impl Deref for ManagerTimeline {
|
||||
type Target = Arc<Timeline>;
|
||||
|
||||
fn deref(&self) -> &Self::Target {
|
||||
&self.tli
|
||||
}
|
||||
}
|
||||
|
||||
impl ManagerTimeline {
|
||||
pub(crate) fn timeline_dir(&self) -> &Utf8PathBuf {
|
||||
&self.tli.timeline_dir
|
||||
}
|
||||
|
||||
/// Manager requests this state on startup.
|
||||
pub(crate) async fn bootstrap_mgr(&self) -> (bool, Option<PartialRemoteSegment>) {
|
||||
let shared_state = self.read_shared_state().await;
|
||||
let is_offloaded = matches!(
|
||||
shared_state.sk.state().eviction_state,
|
||||
EvictionState::Offloaded(_)
|
||||
);
|
||||
let partial_backup_uploaded = shared_state.sk.state().partial_backup.uploaded_segment();
|
||||
|
||||
(is_offloaded, partial_backup_uploaded)
|
||||
}
|
||||
|
||||
/// Try to switch state Present->Offloaded.
|
||||
pub(crate) async fn switch_to_offloaded(
|
||||
&self,
|
||||
partial: &PartialRemoteSegment,
|
||||
) -> anyhow::Result<()> {
|
||||
let mut shared = self.write_shared_state().await;
|
||||
|
||||
// updating control file
|
||||
let mut pstate = shared.sk.state_mut().start_change();
|
||||
|
||||
if !matches!(pstate.eviction_state, EvictionState::Present) {
|
||||
bail!(
|
||||
"cannot switch to offloaded state, current state is {:?}",
|
||||
pstate.eviction_state
|
||||
);
|
||||
}
|
||||
|
||||
if partial.flush_lsn != shared.sk.flush_lsn() {
|
||||
bail!(
|
||||
"flush_lsn mismatch in partial backup, expected {}, got {}",
|
||||
shared.sk.flush_lsn(),
|
||||
partial.flush_lsn
|
||||
);
|
||||
}
|
||||
|
||||
if partial.commit_lsn != pstate.commit_lsn {
|
||||
bail!(
|
||||
"commit_lsn mismatch in partial backup, expected {}, got {}",
|
||||
pstate.commit_lsn,
|
||||
partial.commit_lsn
|
||||
);
|
||||
}
|
||||
|
||||
if partial.term != shared.sk.last_log_term() {
|
||||
bail!(
|
||||
"term mismatch in partial backup, expected {}, got {}",
|
||||
shared.sk.last_log_term(),
|
||||
partial.term
|
||||
);
|
||||
}
|
||||
|
||||
pstate.eviction_state = EvictionState::Offloaded(shared.sk.flush_lsn());
|
||||
shared.sk.state_mut().finish_change(&pstate).await?;
|
||||
// control file is now switched to Offloaded state
|
||||
|
||||
// now we can switch shared.sk to Offloaded, shouldn't fail
|
||||
let prev_sk = std::mem::replace(&mut shared.sk, StateSK::Empty);
|
||||
let cfile_state = prev_sk.take_state();
|
||||
shared.sk = StateSK::Offloaded(Box::new(cfile_state));
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Try to switch state Offloaded->Present.
|
||||
pub(crate) async fn switch_to_present(&self) -> anyhow::Result<()> {
|
||||
let conf = GlobalTimelines::get_global_config();
|
||||
let mut shared = self.write_shared_state().await;
|
||||
|
||||
// trying to restore WAL storage
|
||||
let wal_store = wal_storage::PhysicalStorage::new(
|
||||
&self.ttid,
|
||||
self.timeline_dir.clone(),
|
||||
&conf,
|
||||
shared.sk.state(),
|
||||
)?;
|
||||
|
||||
// updating control file
|
||||
let mut pstate = shared.sk.state_mut().start_change();
|
||||
|
||||
if !matches!(pstate.eviction_state, EvictionState::Offloaded(_)) {
|
||||
bail!(
|
||||
"cannot switch to present state, current state is {:?}",
|
||||
pstate.eviction_state
|
||||
);
|
||||
}
|
||||
|
||||
if wal_store.flush_lsn() != shared.sk.flush_lsn() {
|
||||
bail!(
|
||||
"flush_lsn mismatch in restored WAL, expected {}, got {}",
|
||||
shared.sk.flush_lsn(),
|
||||
wal_store.flush_lsn()
|
||||
);
|
||||
}
|
||||
|
||||
pstate.eviction_state = EvictionState::Present;
|
||||
shared.sk.state_mut().finish_change(&pstate).await?;
|
||||
|
||||
// now we can switch shared.sk to Present, shouldn't fail
|
||||
let prev_sk = std::mem::replace(&mut shared.sk, StateSK::Empty);
|
||||
let cfile_state = prev_sk.take_state();
|
||||
shared.sk = StateSK::Loaded(SafeKeeper::new(cfile_state, wal_store, conf.my_id)?);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Update current manager state, useful for debugging manager deadlocks.
|
||||
pub(crate) fn set_status(&self, status: timeline_manager::Status) {
|
||||
self.mgr_status.store(status, Ordering::Relaxed);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -784,13 +1130,13 @@ async fn delete_dir(path: &Utf8PathBuf) -> Result<bool> {
|
||||
}
|
||||
|
||||
/// Get a path to the tenant directory. If you just need to get a timeline directory,
|
||||
/// use FullAccessTimeline::get_timeline_dir instead.
|
||||
/// use WalResidentTimeline::get_timeline_dir instead.
|
||||
pub(crate) fn get_tenant_dir(conf: &SafeKeeperConf, tenant_id: &TenantId) -> Utf8PathBuf {
|
||||
conf.workdir.join(tenant_id.to_string())
|
||||
}
|
||||
|
||||
/// Get a path to the timeline directory. If you need to read WAL files from disk,
|
||||
/// use FullAccessTimeline::get_timeline_dir instead. This function does not check
|
||||
/// use WalResidentTimeline::get_timeline_dir instead. This function does not check
|
||||
/// timeline eviction status and WAL files might not be present on disk.
|
||||
pub(crate) fn get_timeline_dir(conf: &SafeKeeperConf, ttid: &TenantTimelineId) -> Utf8PathBuf {
|
||||
get_tenant_dir(conf, &ttid.tenant_id).join(ttid.timeline_id.to_string())
|
||||
|
||||
366
safekeeper/src/timeline_eviction.rs
Normal file
366
safekeeper/src/timeline_eviction.rs
Normal file
@@ -0,0 +1,366 @@
|
||||
//! Code related to evicting WAL files to remote storage. The actual upload is done by the
|
||||
//! partial WAL backup code. This file has code to delete and re-download WAL files,
|
||||
//! cross-validate with partial WAL backup if local file is still present.
|
||||
|
||||
use anyhow::Context;
|
||||
use camino::Utf8PathBuf;
|
||||
use remote_storage::RemotePath;
|
||||
use tokio::{
|
||||
fs::File,
|
||||
io::{AsyncRead, AsyncWriteExt},
|
||||
};
|
||||
use tracing::{debug, info, instrument, warn};
|
||||
use utils::crashsafe::durable_rename;
|
||||
|
||||
use crate::{
|
||||
timeline_manager::{Manager, StateSnapshot},
|
||||
wal_backup,
|
||||
wal_backup_partial::{self, PartialRemoteSegment},
|
||||
wal_storage::wal_file_paths,
|
||||
};
|
||||
|
||||
impl Manager {
|
||||
/// Returns true if the timeline is ready for eviction.
|
||||
/// Current criteria:
|
||||
/// - no active tasks
|
||||
/// - control file is flushed (no next event scheduled)
|
||||
/// - no WAL residence guards
|
||||
/// - no pushes to the broker
|
||||
/// - partial WAL backup is uploaded
|
||||
pub(crate) fn ready_for_eviction(
|
||||
&self,
|
||||
next_event: &Option<tokio::time::Instant>,
|
||||
state: &StateSnapshot,
|
||||
) -> bool {
|
||||
self.backup_task.is_none()
|
||||
&& self.recovery_task.is_none()
|
||||
&& self.wal_removal_task.is_none()
|
||||
&& self.partial_backup_task.is_none()
|
||||
&& self.partial_backup_uploaded.is_some()
|
||||
&& next_event.is_none()
|
||||
&& self.access_service.is_empty()
|
||||
&& !self.tli_broker_active.get()
|
||||
&& !wal_backup_partial::needs_uploading(state, &self.partial_backup_uploaded)
|
||||
&& self
|
||||
.partial_backup_uploaded
|
||||
.as_ref()
|
||||
.unwrap()
|
||||
.flush_lsn
|
||||
.segment_number(self.wal_seg_size)
|
||||
== self.last_removed_segno + 1
|
||||
}
|
||||
|
||||
/// Evict the timeline to remote storage.
|
||||
#[instrument(name = "evict_timeline", skip_all)]
|
||||
pub(crate) async fn evict_timeline(&mut self) {
|
||||
assert!(!self.is_offloaded);
|
||||
let partial_backup_uploaded = match &self.partial_backup_uploaded {
|
||||
Some(p) => p.clone(),
|
||||
None => {
|
||||
warn!("no partial backup uploaded, skipping eviction");
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
info!("starting eviction, using {:?}", partial_backup_uploaded);
|
||||
|
||||
if let Err(e) = do_eviction(self, &partial_backup_uploaded).await {
|
||||
warn!("failed to evict timeline: {:?}", e);
|
||||
return;
|
||||
}
|
||||
|
||||
info!("successfully evicted timeline");
|
||||
}
|
||||
|
||||
/// Restore evicted timeline from remote storage.
|
||||
#[instrument(name = "unevict_timeline", skip_all)]
|
||||
pub(crate) async fn unevict_timeline(&mut self) {
|
||||
assert!(self.is_offloaded);
|
||||
let partial_backup_uploaded = match &self.partial_backup_uploaded {
|
||||
Some(p) => p.clone(),
|
||||
None => {
|
||||
warn!("no partial backup uploaded, cannot unevict");
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
info!("starting uneviction, using {:?}", partial_backup_uploaded);
|
||||
|
||||
if let Err(e) = do_uneviction(self, &partial_backup_uploaded).await {
|
||||
warn!("failed to unevict timeline: {:?}", e);
|
||||
return;
|
||||
}
|
||||
|
||||
info!("successfully restored evicted timeline");
|
||||
}
|
||||
}
|
||||
|
||||
/// Ensure that content matches the remote partial backup, if local segment exists.
|
||||
/// Then change state in control file and in-memory. If `delete_offloaded_wal` is set,
|
||||
/// delete the local segment.
|
||||
async fn do_eviction(mgr: &mut Manager, partial: &PartialRemoteSegment) -> anyhow::Result<()> {
|
||||
compare_local_segment_with_remote(mgr, partial).await?;
|
||||
|
||||
mgr.tli.switch_to_offloaded(partial).await?;
|
||||
// switch manager state as soon as possible
|
||||
mgr.is_offloaded = true;
|
||||
|
||||
if mgr.conf.delete_offloaded_wal {
|
||||
delete_local_segment(mgr, partial).await?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Ensure that content matches the remote partial backup, if local segment exists.
|
||||
/// Then download segment to local disk and change state in control file and in-memory.
|
||||
async fn do_uneviction(mgr: &mut Manager, partial: &PartialRemoteSegment) -> anyhow::Result<()> {
|
||||
// if the local segment is present, validate it
|
||||
compare_local_segment_with_remote(mgr, partial).await?;
|
||||
|
||||
// atomically download the partial segment
|
||||
redownload_partial_segment(mgr, partial).await?;
|
||||
|
||||
mgr.tli.switch_to_present().await?;
|
||||
// switch manager state as soon as possible
|
||||
mgr.is_offloaded = false;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Delete local WAL segment.
|
||||
async fn delete_local_segment(mgr: &Manager, partial: &PartialRemoteSegment) -> anyhow::Result<()> {
|
||||
let local_path = local_segment_path(mgr, partial);
|
||||
|
||||
info!("deleting WAL file to evict: {}", local_path);
|
||||
tokio::fs::remove_file(&local_path).await?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Redownload partial segment from remote storage.
|
||||
/// The segment is downloaded to a temporary file and then renamed to the final path.
|
||||
async fn redownload_partial_segment(
|
||||
mgr: &Manager,
|
||||
partial: &PartialRemoteSegment,
|
||||
) -> anyhow::Result<()> {
|
||||
let tmp_file = mgr.tli.timeline_dir().join("remote_partial.tmp");
|
||||
let remote_segfile = remote_segment_path(mgr, partial)?;
|
||||
|
||||
debug!(
|
||||
"redownloading partial segment: {} -> {}",
|
||||
remote_segfile, tmp_file
|
||||
);
|
||||
|
||||
let mut reader = wal_backup::read_object(&remote_segfile, 0).await?;
|
||||
let mut file = File::create(&tmp_file).await?;
|
||||
|
||||
let actual_len = tokio::io::copy(&mut reader, &mut file).await?;
|
||||
let expected_len = partial.flush_lsn.segment_offset(mgr.wal_seg_size);
|
||||
|
||||
if actual_len != expected_len as u64 {
|
||||
anyhow::bail!(
|
||||
"partial downloaded {} bytes, expected {}",
|
||||
actual_len,
|
||||
expected_len
|
||||
);
|
||||
}
|
||||
|
||||
if actual_len > mgr.wal_seg_size as u64 {
|
||||
anyhow::bail!(
|
||||
"remote segment is too long: {} bytes, expected {}",
|
||||
actual_len,
|
||||
mgr.wal_seg_size
|
||||
);
|
||||
}
|
||||
file.set_len(mgr.wal_seg_size as u64).await?;
|
||||
file.flush().await?;
|
||||
|
||||
let final_path = local_segment_path(mgr, partial);
|
||||
info!(
|
||||
"downloaded {} bytes, renaming to {}",
|
||||
final_path, final_path,
|
||||
);
|
||||
if let Err(e) = durable_rename(&tmp_file, &final_path, !mgr.conf.no_sync).await {
|
||||
// Probably rename succeeded, but fsync of it failed. Remove
|
||||
// the file then to avoid using it.
|
||||
tokio::fs::remove_file(tmp_file)
|
||||
.await
|
||||
.or_else(utils::fs_ext::ignore_not_found)?;
|
||||
return Err(e.into());
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Compare local WAL segment with partial WAL backup in remote storage.
|
||||
/// If the local segment is not present, the function does nothing.
|
||||
/// If the local segment is present, it compares the local segment with the remote one.
|
||||
async fn compare_local_segment_with_remote(
|
||||
mgr: &Manager,
|
||||
partial: &PartialRemoteSegment,
|
||||
) -> anyhow::Result<()> {
|
||||
let local_path = local_segment_path(mgr, partial);
|
||||
|
||||
match File::open(&local_path).await {
|
||||
Ok(mut local_file) => do_validation(mgr, &mut local_file, mgr.wal_seg_size, partial)
|
||||
.await
|
||||
.context("validation failed"),
|
||||
Err(_) => {
|
||||
info!(
|
||||
"local WAL file {} is not present, skipping validation",
|
||||
local_path
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Compare opened local WAL segment with partial WAL backup in remote storage.
|
||||
/// Validate full content of both files.
|
||||
async fn do_validation(
|
||||
mgr: &Manager,
|
||||
file: &mut File,
|
||||
wal_seg_size: usize,
|
||||
partial: &PartialRemoteSegment,
|
||||
) -> anyhow::Result<()> {
|
||||
let local_size = file.metadata().await?.len() as usize;
|
||||
if local_size != wal_seg_size {
|
||||
anyhow::bail!(
|
||||
"local segment size is invalid: found {}, expected {}",
|
||||
local_size,
|
||||
wal_seg_size
|
||||
);
|
||||
}
|
||||
|
||||
let remote_segfile = remote_segment_path(mgr, partial)?;
|
||||
let mut remote_reader: std::pin::Pin<Box<dyn AsyncRead + Send + Sync>> =
|
||||
wal_backup::read_object(&remote_segfile, 0).await?;
|
||||
|
||||
// remote segment should have bytes excatly up to `flush_lsn`
|
||||
let expected_remote_size = partial.flush_lsn.segment_offset(mgr.wal_seg_size);
|
||||
// let's compare the first `expected_remote_size` bytes
|
||||
compare_n_bytes(&mut remote_reader, file, expected_remote_size).await?;
|
||||
// and check that the remote segment ends here
|
||||
check_end(&mut remote_reader).await?;
|
||||
|
||||
// if local segment is longer, the rest should be zeroes
|
||||
read_n_zeroes(file, mgr.wal_seg_size - expected_remote_size).await?;
|
||||
// and check that the local segment ends here
|
||||
check_end(file).await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn local_segment_path(mgr: &Manager, partial: &PartialRemoteSegment) -> Utf8PathBuf {
|
||||
let flush_lsn = partial.flush_lsn;
|
||||
let segno = flush_lsn.segment_number(mgr.wal_seg_size);
|
||||
let (_, local_partial_segfile) =
|
||||
wal_file_paths(mgr.tli.timeline_dir(), segno, mgr.wal_seg_size);
|
||||
local_partial_segfile
|
||||
}
|
||||
|
||||
fn remote_segment_path(
|
||||
mgr: &Manager,
|
||||
partial: &PartialRemoteSegment,
|
||||
) -> anyhow::Result<RemotePath> {
|
||||
let remote_timeline_path = wal_backup::remote_timeline_path(&mgr.tli.ttid)?;
|
||||
Ok(partial.remote_path(&remote_timeline_path))
|
||||
}
|
||||
|
||||
/// Compare first `n` bytes of two readers. If the bytes differ, return an error.
|
||||
/// If the readers are shorter than `n`, return an error.
|
||||
async fn compare_n_bytes<R1, R2>(reader1: &mut R1, reader2: &mut R2, n: usize) -> anyhow::Result<()>
|
||||
where
|
||||
R1: AsyncRead + Unpin,
|
||||
R2: AsyncRead + Unpin,
|
||||
{
|
||||
use tokio::io::AsyncReadExt;
|
||||
|
||||
const BUF_SIZE: usize = 32 * 1024;
|
||||
|
||||
let mut buffer1 = vec![0u8; BUF_SIZE];
|
||||
let mut buffer2 = vec![0u8; BUF_SIZE];
|
||||
|
||||
let mut offset = 0;
|
||||
|
||||
while offset < n {
|
||||
let bytes_to_read = std::cmp::min(BUF_SIZE, n - offset);
|
||||
|
||||
let bytes_read1 = reader1
|
||||
.read(&mut buffer1[..bytes_to_read])
|
||||
.await
|
||||
.with_context(|| format!("failed to read from reader1 at offset {}", offset))?;
|
||||
if bytes_read1 == 0 {
|
||||
anyhow::bail!("unexpected EOF from reader1 at offset {}", offset);
|
||||
}
|
||||
|
||||
let bytes_read2 = reader2
|
||||
.read_exact(&mut buffer2[..bytes_read1])
|
||||
.await
|
||||
.with_context(|| {
|
||||
format!(
|
||||
"failed to read {} bytes from reader2 at offset {}",
|
||||
bytes_read1, offset
|
||||
)
|
||||
})?;
|
||||
assert!(bytes_read2 == bytes_read1);
|
||||
|
||||
if buffer1[..bytes_read1] != buffer2[..bytes_read2] {
|
||||
let diff_offset = buffer1[..bytes_read1]
|
||||
.iter()
|
||||
.zip(buffer2[..bytes_read2].iter())
|
||||
.position(|(a, b)| a != b)
|
||||
.expect("mismatched buffers, but no difference found");
|
||||
anyhow::bail!("mismatch at offset {}", offset + diff_offset);
|
||||
}
|
||||
|
||||
offset += bytes_read1;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn check_end<R>(mut reader: R) -> anyhow::Result<()>
|
||||
where
|
||||
R: AsyncRead + Unpin,
|
||||
{
|
||||
use tokio::io::AsyncReadExt;
|
||||
|
||||
let mut buffer = [0u8; 1];
|
||||
let bytes_read = reader.read(&mut buffer).await?;
|
||||
if bytes_read != 0 {
|
||||
anyhow::bail!("expected EOF, found bytes");
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn read_n_zeroes<R>(reader: &mut R, n: usize) -> anyhow::Result<()>
|
||||
where
|
||||
R: AsyncRead + Unpin,
|
||||
{
|
||||
use tokio::io::AsyncReadExt;
|
||||
|
||||
const BUF_SIZE: usize = 32 * 1024;
|
||||
let mut buffer = vec![0u8; BUF_SIZE];
|
||||
let mut offset = 0;
|
||||
|
||||
while offset < n {
|
||||
let bytes_to_read = std::cmp::min(BUF_SIZE, n - offset);
|
||||
|
||||
let bytes_read = reader
|
||||
.read(&mut buffer[..bytes_to_read])
|
||||
.await
|
||||
.context("expected zeroes, got read error")?;
|
||||
if bytes_read == 0 {
|
||||
anyhow::bail!("expected zeroes, got EOF");
|
||||
}
|
||||
|
||||
if buffer[..bytes_read].iter().all(|&b| b == 0) {
|
||||
offset += bytes_read;
|
||||
} else {
|
||||
anyhow::bail!("non-zero byte found");
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
71
safekeeper/src/timeline_guard.rs
Normal file
71
safekeeper/src/timeline_guard.rs
Normal file
@@ -0,0 +1,71 @@
|
||||
//! Timeline residence guard is needed to ensure that WAL segments are present on disk,
|
||||
//! as long as the code is holding the guard. This file implements guard logic, to issue
|
||||
//! and drop guards, and to notify the manager when the guard is dropped.
|
||||
|
||||
use std::collections::HashSet;
|
||||
|
||||
use tracing::{debug, warn};
|
||||
|
||||
use crate::timeline_manager::ManagerCtlMessage;
|
||||
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub struct GuardId(u64);
|
||||
|
||||
pub struct ResidenceGuard {
|
||||
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
|
||||
guard_id: GuardId,
|
||||
}
|
||||
|
||||
impl Drop for ResidenceGuard {
|
||||
fn drop(&mut self) {
|
||||
// notify the manager that the guard is dropped
|
||||
let res = self
|
||||
.manager_tx
|
||||
.send(ManagerCtlMessage::GuardDrop(self.guard_id));
|
||||
if let Err(e) = res {
|
||||
warn!("failed to send GuardDrop message: {:?}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// AccessService is responsible for issuing and dropping residence guards.
|
||||
/// All guards are stored in the `guards` set.
|
||||
/// TODO: it's possible to add `String` name to each guard, for better observability.
|
||||
pub(crate) struct AccessService {
|
||||
next_guard_id: u64,
|
||||
guards: HashSet<u64>,
|
||||
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
|
||||
}
|
||||
|
||||
impl AccessService {
|
||||
pub(crate) fn new(manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>) -> Self {
|
||||
Self {
|
||||
next_guard_id: 0,
|
||||
guards: HashSet::new(),
|
||||
manager_tx,
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn is_empty(&self) -> bool {
|
||||
self.guards.is_empty()
|
||||
}
|
||||
|
||||
pub(crate) fn create_guard(&mut self) -> ResidenceGuard {
|
||||
let guard_id = self.next_guard_id;
|
||||
self.next_guard_id += 1;
|
||||
self.guards.insert(guard_id);
|
||||
|
||||
let guard_id = GuardId(guard_id);
|
||||
debug!("issued a new guard {:?}", guard_id);
|
||||
|
||||
ResidenceGuard {
|
||||
manager_tx: self.manager_tx.clone(),
|
||||
guard_id,
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn drop_guard(&mut self, guard_id: GuardId) {
|
||||
debug!("dropping guard {:?}", guard_id);
|
||||
assert!(self.guards.remove(&guard_id.0));
|
||||
}
|
||||
}
|
||||
@@ -2,66 +2,83 @@
|
||||
//! It is spawned alongside each timeline and exits when the timeline is deleted.
|
||||
//! It watches for changes in the timeline state and decides when to spawn or kill background tasks.
|
||||
//! It also can manage some reactive state, like should the timeline be active for broker pushes or not.
|
||||
//!
|
||||
//! Be aware that you need to be extra careful with manager code, because it is not respawned on panic.
|
||||
//! Also, if it will stuck in some branch, it will prevent any further progress in the timeline.
|
||||
|
||||
use std::{
|
||||
sync::Arc,
|
||||
time::{Duration, Instant},
|
||||
sync::{atomic::AtomicUsize, Arc},
|
||||
time::Duration,
|
||||
};
|
||||
|
||||
use postgres_ffi::XLogSegNo;
|
||||
use tokio::task::{JoinError, JoinHandle};
|
||||
use tracing::{info, info_span, instrument, warn, Instrument};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use tokio::{
|
||||
task::{JoinError, JoinHandle},
|
||||
time::Instant,
|
||||
};
|
||||
use tracing::{debug, info, info_span, instrument, warn, Instrument};
|
||||
use utils::lsn::Lsn;
|
||||
|
||||
use crate::{
|
||||
control_file::Storage,
|
||||
control_file::{FileStorage, Storage},
|
||||
metrics::{MANAGER_ACTIVE_CHANGES, MANAGER_ITERATIONS_TOTAL},
|
||||
recovery::recovery_main,
|
||||
remove_wal::calc_horizon_lsn,
|
||||
safekeeper::Term,
|
||||
send_wal::WalSenders,
|
||||
timeline::{PeerInfo, ReadGuardSharedState, Timeline},
|
||||
state::TimelineState,
|
||||
timeline::{ManagerTimeline, PeerInfo, ReadGuardSharedState, StateSK, WalResidentTimeline},
|
||||
timeline_guard::{AccessService, GuardId, ResidenceGuard},
|
||||
timelines_set::{TimelineSetGuard, TimelinesSet},
|
||||
wal_backup::{self, WalBackupTaskHandle},
|
||||
wal_backup_partial, SafeKeeperConf,
|
||||
wal_backup_partial::{self, PartialRemoteSegment},
|
||||
SafeKeeperConf,
|
||||
};
|
||||
|
||||
pub struct StateSnapshot {
|
||||
pub(crate) struct StateSnapshot {
|
||||
// inmem values
|
||||
pub commit_lsn: Lsn,
|
||||
pub backup_lsn: Lsn,
|
||||
pub remote_consistent_lsn: Lsn,
|
||||
pub(crate) commit_lsn: Lsn,
|
||||
pub(crate) backup_lsn: Lsn,
|
||||
pub(crate) remote_consistent_lsn: Lsn,
|
||||
|
||||
// persistent control file values
|
||||
pub cfile_peer_horizon_lsn: Lsn,
|
||||
pub cfile_remote_consistent_lsn: Lsn,
|
||||
pub cfile_backup_lsn: Lsn,
|
||||
pub(crate) cfile_peer_horizon_lsn: Lsn,
|
||||
pub(crate) cfile_remote_consistent_lsn: Lsn,
|
||||
pub(crate) cfile_backup_lsn: Lsn,
|
||||
|
||||
// latest state
|
||||
pub(crate) flush_lsn: Lsn,
|
||||
pub(crate) last_log_term: Term,
|
||||
|
||||
// misc
|
||||
pub cfile_last_persist_at: Instant,
|
||||
pub inmem_flush_pending: bool,
|
||||
pub wal_removal_on_hold: bool,
|
||||
pub peers: Vec<PeerInfo>,
|
||||
pub(crate) cfile_last_persist_at: std::time::Instant,
|
||||
pub(crate) inmem_flush_pending: bool,
|
||||
pub(crate) wal_removal_on_hold: bool,
|
||||
pub(crate) peers: Vec<PeerInfo>,
|
||||
}
|
||||
|
||||
impl StateSnapshot {
|
||||
/// Create a new snapshot of the timeline state.
|
||||
fn new(read_guard: ReadGuardSharedState, heartbeat_timeout: Duration) -> Self {
|
||||
let state = read_guard.sk.state();
|
||||
Self {
|
||||
commit_lsn: read_guard.sk.state.inmem.commit_lsn,
|
||||
backup_lsn: read_guard.sk.state.inmem.backup_lsn,
|
||||
remote_consistent_lsn: read_guard.sk.state.inmem.remote_consistent_lsn,
|
||||
cfile_peer_horizon_lsn: read_guard.sk.state.peer_horizon_lsn,
|
||||
cfile_remote_consistent_lsn: read_guard.sk.state.remote_consistent_lsn,
|
||||
cfile_backup_lsn: read_guard.sk.state.backup_lsn,
|
||||
cfile_last_persist_at: read_guard.sk.state.pers.last_persist_at(),
|
||||
inmem_flush_pending: Self::has_unflushed_inmem_state(&read_guard),
|
||||
commit_lsn: state.inmem.commit_lsn,
|
||||
backup_lsn: state.inmem.backup_lsn,
|
||||
remote_consistent_lsn: state.inmem.remote_consistent_lsn,
|
||||
cfile_peer_horizon_lsn: state.peer_horizon_lsn,
|
||||
cfile_remote_consistent_lsn: state.remote_consistent_lsn,
|
||||
cfile_backup_lsn: state.backup_lsn,
|
||||
flush_lsn: read_guard.sk.flush_lsn(),
|
||||
last_log_term: read_guard.sk.last_log_term(),
|
||||
cfile_last_persist_at: state.pers.last_persist_at(),
|
||||
inmem_flush_pending: Self::has_unflushed_inmem_state(state),
|
||||
wal_removal_on_hold: read_guard.wal_removal_on_hold,
|
||||
peers: read_guard.get_peers(heartbeat_timeout),
|
||||
}
|
||||
}
|
||||
|
||||
fn has_unflushed_inmem_state(read_guard: &ReadGuardSharedState) -> bool {
|
||||
let state = &read_guard.sk.state;
|
||||
fn has_unflushed_inmem_state(state: &TimelineState<FileStorage>) -> bool {
|
||||
state.inmem.commit_lsn > state.commit_lsn
|
||||
|| state.inmem.backup_lsn > state.backup_lsn
|
||||
|| state.inmem.peer_horizon_lsn > state.peer_horizon_lsn
|
||||
@@ -73,314 +90,560 @@ impl StateSnapshot {
|
||||
/// There is no need to check for updates more often than this.
|
||||
const REFRESH_INTERVAL: Duration = Duration::from_millis(300);
|
||||
|
||||
/// How often to save the control file if the is no other activity.
|
||||
const CF_SAVE_INTERVAL: Duration = Duration::from_secs(300);
|
||||
pub enum ManagerCtlMessage {
|
||||
/// Request to get a guard for WalResidentTimeline, with WAL files available locally.
|
||||
GuardRequest(tokio::sync::oneshot::Sender<anyhow::Result<ResidenceGuard>>),
|
||||
/// Request to drop the guard.
|
||||
GuardDrop(GuardId),
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for ManagerCtlMessage {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
match self {
|
||||
ManagerCtlMessage::GuardRequest(_) => write!(f, "GuardRequest"),
|
||||
ManagerCtlMessage::GuardDrop(id) => write!(f, "GuardDrop({:?})", id),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct ManagerCtl {
|
||||
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
|
||||
|
||||
// this is used to initialize manager, it will be moved out in bootstrap().
|
||||
init_manager_rx:
|
||||
std::sync::Mutex<Option<tokio::sync::mpsc::UnboundedReceiver<ManagerCtlMessage>>>,
|
||||
}
|
||||
|
||||
impl Default for ManagerCtl {
|
||||
fn default() -> Self {
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl ManagerCtl {
|
||||
pub fn new() -> Self {
|
||||
let (tx, rx) = tokio::sync::mpsc::unbounded_channel();
|
||||
Self {
|
||||
manager_tx: tx,
|
||||
init_manager_rx: std::sync::Mutex::new(Some(rx)),
|
||||
}
|
||||
}
|
||||
|
||||
/// Issue a new guard and wait for manager to prepare the timeline.
|
||||
/// Sends a message to the manager and waits for the response.
|
||||
/// Can be blocked indefinitely if the manager is stuck.
|
||||
pub async fn wal_residence_guard(&self) -> anyhow::Result<ResidenceGuard> {
|
||||
let (tx, rx) = tokio::sync::oneshot::channel();
|
||||
self.manager_tx.send(ManagerCtlMessage::GuardRequest(tx))?;
|
||||
|
||||
// wait for the manager to respond with the guard
|
||||
rx.await
|
||||
.map_err(|e| anyhow::anyhow!("response read fail: {:?}", e))
|
||||
.and_then(std::convert::identity)
|
||||
}
|
||||
|
||||
/// Must be called exactly once to bootstrap the manager.
|
||||
pub fn bootstrap_manager(
|
||||
&self,
|
||||
) -> (
|
||||
tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
|
||||
tokio::sync::mpsc::UnboundedReceiver<ManagerCtlMessage>,
|
||||
) {
|
||||
let rx = self
|
||||
.init_manager_rx
|
||||
.lock()
|
||||
.expect("mutex init_manager_rx poisoned")
|
||||
.take()
|
||||
.expect("manager already bootstrapped");
|
||||
|
||||
(self.manager_tx.clone(), rx)
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct Manager {
|
||||
// configuration & dependencies
|
||||
pub(crate) tli: ManagerTimeline,
|
||||
pub(crate) conf: SafeKeeperConf,
|
||||
pub(crate) wal_seg_size: usize,
|
||||
pub(crate) walsenders: Arc<WalSenders>,
|
||||
|
||||
// current state
|
||||
pub(crate) state_version_rx: tokio::sync::watch::Receiver<usize>,
|
||||
pub(crate) num_computes_rx: tokio::sync::watch::Receiver<usize>,
|
||||
pub(crate) tli_broker_active: TimelineSetGuard,
|
||||
pub(crate) last_removed_segno: XLogSegNo,
|
||||
pub(crate) is_offloaded: bool,
|
||||
|
||||
// background tasks
|
||||
pub(crate) backup_task: Option<WalBackupTaskHandle>,
|
||||
pub(crate) recovery_task: Option<JoinHandle<()>>,
|
||||
pub(crate) wal_removal_task: Option<JoinHandle<anyhow::Result<u64>>>,
|
||||
|
||||
// partial backup
|
||||
pub(crate) partial_backup_task: Option<JoinHandle<Option<PartialRemoteSegment>>>,
|
||||
pub(crate) partial_backup_uploaded: Option<PartialRemoteSegment>,
|
||||
|
||||
// misc
|
||||
pub(crate) access_service: AccessService,
|
||||
}
|
||||
|
||||
/// This task gets spawned alongside each timeline and is responsible for managing the timeline's
|
||||
/// background tasks.
|
||||
/// Be careful, this task is not respawned on panic, so it should not panic.
|
||||
#[instrument(name = "manager", skip_all, fields(ttid = %tli.ttid))]
|
||||
pub async fn main_task(
|
||||
tli: Arc<Timeline>,
|
||||
tli: ManagerTimeline,
|
||||
conf: SafeKeeperConf,
|
||||
broker_active_set: Arc<TimelinesSet>,
|
||||
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
|
||||
mut manager_rx: tokio::sync::mpsc::UnboundedReceiver<ManagerCtlMessage>,
|
||||
) {
|
||||
tli.set_status(Status::Started);
|
||||
|
||||
let defer_tli = tli.tli.clone();
|
||||
scopeguard::defer! {
|
||||
if tli.is_cancelled() {
|
||||
if defer_tli.is_cancelled() {
|
||||
info!("manager task finished");
|
||||
} else {
|
||||
warn!("manager task finished prematurely");
|
||||
}
|
||||
};
|
||||
|
||||
// configuration & dependencies
|
||||
let wal_seg_size = tli.get_wal_seg_size().await;
|
||||
let heartbeat_timeout = conf.heartbeat_timeout;
|
||||
let walsenders = tli.get_walsenders();
|
||||
let walreceivers = tli.get_walreceivers();
|
||||
|
||||
// current state
|
||||
let mut state_version_rx = tli.get_state_version_rx();
|
||||
let mut num_computes_rx = walreceivers.get_num_rx();
|
||||
let mut tli_broker_active = broker_active_set.guard(tli.clone());
|
||||
let mut last_removed_segno = 0 as XLogSegNo;
|
||||
|
||||
// list of background tasks
|
||||
let mut backup_task: Option<WalBackupTaskHandle> = None;
|
||||
let mut recovery_task: Option<JoinHandle<()>> = None;
|
||||
let mut partial_backup_task: Option<JoinHandle<()>> = None;
|
||||
let mut wal_removal_task: Option<JoinHandle<anyhow::Result<u64>>> = None;
|
||||
let mut mgr = Manager::new(tli, conf, broker_active_set, manager_tx).await;
|
||||
|
||||
// Start recovery task which always runs on the timeline.
|
||||
if conf.peer_recovery_enabled {
|
||||
match tli.full_access_guard().await {
|
||||
Ok(tli) => {
|
||||
recovery_task = Some(tokio::spawn(recovery_main(tli, conf.clone())));
|
||||
}
|
||||
Err(e) => {
|
||||
warn!("failed to start recovery task: {:?}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Start partial backup task which always runs on the timeline.
|
||||
if conf.is_wal_backup_enabled() && conf.partial_backup_enabled {
|
||||
match tli.full_access_guard().await {
|
||||
Ok(tli) => {
|
||||
partial_backup_task = Some(tokio::spawn(wal_backup_partial::main_task(
|
||||
tli,
|
||||
conf.clone(),
|
||||
)));
|
||||
}
|
||||
Err(e) => {
|
||||
warn!("failed to start partial backup task: {:?}", e);
|
||||
}
|
||||
}
|
||||
if !mgr.is_offloaded && mgr.conf.peer_recovery_enabled {
|
||||
let tli = mgr.wal_resident_timeline();
|
||||
mgr.recovery_task = Some(tokio::spawn(recovery_main(tli, mgr.conf.clone())));
|
||||
}
|
||||
|
||||
let last_state = 'outer: loop {
|
||||
MANAGER_ITERATIONS_TOTAL.inc();
|
||||
|
||||
let state_snapshot = StateSnapshot::new(tli.read_shared_state().await, heartbeat_timeout);
|
||||
let num_computes = *num_computes_rx.borrow();
|
||||
mgr.set_status(Status::StateSnapshot);
|
||||
let state_snapshot = mgr.state_snapshot().await;
|
||||
|
||||
let is_wal_backup_required = update_backup(
|
||||
&conf,
|
||||
&tli,
|
||||
wal_seg_size,
|
||||
num_computes,
|
||||
&state_snapshot,
|
||||
&mut backup_task,
|
||||
)
|
||||
.await;
|
||||
let mut next_event: Option<Instant> = None;
|
||||
if !mgr.is_offloaded {
|
||||
let num_computes = *mgr.num_computes_rx.borrow();
|
||||
|
||||
let _is_active = update_is_active(
|
||||
is_wal_backup_required,
|
||||
num_computes,
|
||||
&state_snapshot,
|
||||
&mut tli_broker_active,
|
||||
&tli,
|
||||
);
|
||||
mgr.set_status(Status::UpdateBackup);
|
||||
let is_wal_backup_required = mgr.update_backup(num_computes, &state_snapshot).await;
|
||||
mgr.update_is_active(is_wal_backup_required, num_computes, &state_snapshot);
|
||||
|
||||
let next_cfile_save = update_control_file_save(&state_snapshot, &tli).await;
|
||||
mgr.set_status(Status::UpdateControlFile);
|
||||
mgr.update_control_file_save(&state_snapshot, &mut next_event)
|
||||
.await;
|
||||
|
||||
update_wal_removal(
|
||||
&conf,
|
||||
walsenders,
|
||||
&tli,
|
||||
wal_seg_size,
|
||||
&state_snapshot,
|
||||
last_removed_segno,
|
||||
&mut wal_removal_task,
|
||||
)
|
||||
.await;
|
||||
mgr.set_status(Status::UpdateWalRemoval);
|
||||
mgr.update_wal_removal(&state_snapshot).await;
|
||||
|
||||
mgr.set_status(Status::UpdatePartialBackup);
|
||||
mgr.update_partial_backup(&state_snapshot).await;
|
||||
|
||||
if mgr.conf.enable_offload && mgr.ready_for_eviction(&next_event, &state_snapshot) {
|
||||
mgr.set_status(Status::EvictTimeline);
|
||||
mgr.evict_timeline().await;
|
||||
}
|
||||
}
|
||||
|
||||
mgr.set_status(Status::Wait);
|
||||
// wait until something changes. tx channels are stored under Arc, so they will not be
|
||||
// dropped until the manager task is finished.
|
||||
tokio::select! {
|
||||
_ = tli.cancel.cancelled() => {
|
||||
_ = mgr.tli.cancel.cancelled() => {
|
||||
// timeline was deleted
|
||||
break 'outer state_snapshot;
|
||||
}
|
||||
_ = async {
|
||||
// don't wake up on every state change, but at most every REFRESH_INTERVAL
|
||||
tokio::time::sleep(REFRESH_INTERVAL).await;
|
||||
let _ = state_version_rx.changed().await;
|
||||
let _ = mgr.state_version_rx.changed().await;
|
||||
} => {
|
||||
// state was updated
|
||||
}
|
||||
_ = num_computes_rx.changed() => {
|
||||
_ = mgr.num_computes_rx.changed() => {
|
||||
// number of connected computes was updated
|
||||
}
|
||||
_ = async {
|
||||
if let Some(timeout) = next_cfile_save {
|
||||
tokio::time::sleep_until(timeout).await
|
||||
} else {
|
||||
futures::future::pending().await
|
||||
}
|
||||
} => {
|
||||
// it's time to save the control file
|
||||
_ = sleep_until(&next_event) => {
|
||||
// we were waiting for some event (e.g. cfile save)
|
||||
}
|
||||
res = async {
|
||||
if let Some(task) = &mut wal_removal_task {
|
||||
task.await
|
||||
} else {
|
||||
futures::future::pending().await
|
||||
}
|
||||
} => {
|
||||
res = await_task_finish(&mut mgr.wal_removal_task) => {
|
||||
// WAL removal task finished
|
||||
wal_removal_task = None;
|
||||
update_wal_removal_end(res, &tli, &mut last_removed_segno);
|
||||
mgr.wal_removal_task = None;
|
||||
mgr.update_wal_removal_end(res);
|
||||
}
|
||||
res = await_task_finish(&mut mgr.partial_backup_task) => {
|
||||
// partial backup task finished
|
||||
mgr.partial_backup_task = None;
|
||||
mgr.update_partial_backup_end(res);
|
||||
}
|
||||
|
||||
msg = manager_rx.recv() => {
|
||||
mgr.set_status(Status::HandleMessage);
|
||||
mgr.handle_message(msg).await;
|
||||
}
|
||||
}
|
||||
};
|
||||
mgr.set_status(Status::Exiting);
|
||||
|
||||
// remove timeline from the broker active set sooner, before waiting for background tasks
|
||||
tli_broker_active.set(false);
|
||||
mgr.tli_broker_active.set(false);
|
||||
|
||||
// shutdown background tasks
|
||||
if conf.is_wal_backup_enabled() {
|
||||
wal_backup::update_task(&conf, &tli, false, &last_state, &mut backup_task).await;
|
||||
if mgr.conf.is_wal_backup_enabled() {
|
||||
wal_backup::update_task(&mut mgr, false, &last_state).await;
|
||||
}
|
||||
|
||||
if let Some(recovery_task) = recovery_task {
|
||||
if let Some(recovery_task) = &mut mgr.recovery_task {
|
||||
if let Err(e) = recovery_task.await {
|
||||
warn!("recovery task failed: {:?}", e);
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(partial_backup_task) = partial_backup_task {
|
||||
if let Some(partial_backup_task) = &mut mgr.partial_backup_task {
|
||||
if let Err(e) = partial_backup_task.await {
|
||||
warn!("partial backup task failed: {:?}", e);
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(wal_removal_task) = wal_removal_task {
|
||||
if let Some(wal_removal_task) = &mut mgr.wal_removal_task {
|
||||
let res = wal_removal_task.await;
|
||||
update_wal_removal_end(res, &tli, &mut last_removed_segno);
|
||||
mgr.update_wal_removal_end(res);
|
||||
}
|
||||
|
||||
mgr.set_status(Status::Finished);
|
||||
}
|
||||
|
||||
/// Spawns/kills backup task and returns true if backup is required.
|
||||
async fn update_backup(
|
||||
conf: &SafeKeeperConf,
|
||||
tli: &Arc<Timeline>,
|
||||
wal_seg_size: usize,
|
||||
num_computes: usize,
|
||||
state: &StateSnapshot,
|
||||
backup_task: &mut Option<WalBackupTaskHandle>,
|
||||
) -> bool {
|
||||
let is_wal_backup_required =
|
||||
wal_backup::is_wal_backup_required(wal_seg_size, num_computes, state);
|
||||
|
||||
if conf.is_wal_backup_enabled() {
|
||||
wal_backup::update_task(conf, tli, is_wal_backup_required, state, backup_task).await;
|
||||
impl Manager {
|
||||
async fn new(
|
||||
tli: ManagerTimeline,
|
||||
conf: SafeKeeperConf,
|
||||
broker_active_set: Arc<TimelinesSet>,
|
||||
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
|
||||
) -> Manager {
|
||||
let (is_offloaded, partial_backup_uploaded) = tli.bootstrap_mgr().await;
|
||||
Manager {
|
||||
conf,
|
||||
wal_seg_size: tli.get_wal_seg_size().await,
|
||||
walsenders: tli.get_walsenders().clone(),
|
||||
state_version_rx: tli.get_state_version_rx(),
|
||||
num_computes_rx: tli.get_walreceivers().get_num_rx(),
|
||||
tli_broker_active: broker_active_set.guard(tli.clone()),
|
||||
last_removed_segno: 0,
|
||||
is_offloaded,
|
||||
backup_task: None,
|
||||
recovery_task: None,
|
||||
wal_removal_task: None,
|
||||
partial_backup_task: None,
|
||||
partial_backup_uploaded,
|
||||
access_service: AccessService::new(manager_tx),
|
||||
tli,
|
||||
}
|
||||
}
|
||||
|
||||
// update the state in Arc<Timeline>
|
||||
tli.wal_backup_active
|
||||
.store(backup_task.is_some(), std::sync::atomic::Ordering::Relaxed);
|
||||
is_wal_backup_required
|
||||
}
|
||||
|
||||
/// Update is_active flag and returns its value.
|
||||
fn update_is_active(
|
||||
is_wal_backup_required: bool,
|
||||
num_computes: usize,
|
||||
state: &StateSnapshot,
|
||||
tli_broker_active: &mut TimelineSetGuard,
|
||||
tli: &Arc<Timeline>,
|
||||
) -> bool {
|
||||
let is_active = is_wal_backup_required
|
||||
|| num_computes > 0
|
||||
|| state.remote_consistent_lsn < state.commit_lsn;
|
||||
|
||||
// update the broker timeline set
|
||||
if tli_broker_active.set(is_active) {
|
||||
// write log if state has changed
|
||||
info!(
|
||||
"timeline active={} now, remote_consistent_lsn={}, commit_lsn={}",
|
||||
is_active, state.remote_consistent_lsn, state.commit_lsn,
|
||||
);
|
||||
|
||||
MANAGER_ACTIVE_CHANGES.inc();
|
||||
fn set_status(&self, status: Status) {
|
||||
self.tli.set_status(status);
|
||||
}
|
||||
|
||||
// update the state in Arc<Timeline>
|
||||
tli.broker_active
|
||||
.store(is_active, std::sync::atomic::Ordering::Relaxed);
|
||||
is_active
|
||||
}
|
||||
|
||||
/// Save control file if needed. Returns Instant if we should persist the control file in the future.
|
||||
async fn update_control_file_save(
|
||||
state: &StateSnapshot,
|
||||
tli: &Arc<Timeline>,
|
||||
) -> Option<tokio::time::Instant> {
|
||||
if !state.inmem_flush_pending {
|
||||
return None;
|
||||
/// Get a WalResidentTimeline.
|
||||
/// Manager code must use this function instead of one from `Timeline`
|
||||
/// directly, because it will deadlock.
|
||||
pub(crate) fn wal_resident_timeline(&mut self) -> WalResidentTimeline {
|
||||
assert!(!self.is_offloaded);
|
||||
let guard = self.access_service.create_guard();
|
||||
WalResidentTimeline::new(self.tli.clone(), guard)
|
||||
}
|
||||
|
||||
if state.cfile_last_persist_at.elapsed() > CF_SAVE_INTERVAL {
|
||||
let mut write_guard = tli.write_shared_state().await;
|
||||
// this can be done in the background because it blocks manager task, but flush() should
|
||||
// be fast enough not to be a problem now
|
||||
if let Err(e) = write_guard.sk.state.flush().await {
|
||||
warn!("failed to save control file: {:?}", e);
|
||||
/// Get a snapshot of the timeline state.
|
||||
async fn state_snapshot(&self) -> StateSnapshot {
|
||||
StateSnapshot::new(
|
||||
self.tli.read_shared_state().await,
|
||||
self.conf.heartbeat_timeout,
|
||||
)
|
||||
}
|
||||
|
||||
/// Spawns/kills backup task and returns true if backup is required.
|
||||
async fn update_backup(&mut self, num_computes: usize, state: &StateSnapshot) -> bool {
|
||||
let is_wal_backup_required =
|
||||
wal_backup::is_wal_backup_required(self.wal_seg_size, num_computes, state);
|
||||
|
||||
if self.conf.is_wal_backup_enabled() {
|
||||
wal_backup::update_task(self, is_wal_backup_required, state).await;
|
||||
}
|
||||
|
||||
None
|
||||
} else {
|
||||
// we should wait until next CF_SAVE_INTERVAL
|
||||
Some((state.cfile_last_persist_at + CF_SAVE_INTERVAL).into())
|
||||
}
|
||||
}
|
||||
|
||||
/// Spawns WAL removal task if needed.
|
||||
async fn update_wal_removal(
|
||||
conf: &SafeKeeperConf,
|
||||
walsenders: &Arc<WalSenders>,
|
||||
tli: &Arc<Timeline>,
|
||||
wal_seg_size: usize,
|
||||
state: &StateSnapshot,
|
||||
last_removed_segno: u64,
|
||||
wal_removal_task: &mut Option<JoinHandle<anyhow::Result<u64>>>,
|
||||
) {
|
||||
if wal_removal_task.is_some() || state.wal_removal_on_hold {
|
||||
// WAL removal is already in progress or hold off
|
||||
return;
|
||||
}
|
||||
|
||||
// If enabled, we use LSN of the most lagging walsender as a WAL removal horizon.
|
||||
// This allows to get better read speed for pageservers that are lagging behind,
|
||||
// at the cost of keeping more WAL on disk.
|
||||
let replication_horizon_lsn = if conf.walsenders_keep_horizon {
|
||||
walsenders.laggard_lsn()
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
let removal_horizon_lsn = calc_horizon_lsn(state, replication_horizon_lsn);
|
||||
let removal_horizon_segno = removal_horizon_lsn
|
||||
.segment_number(wal_seg_size)
|
||||
.saturating_sub(1);
|
||||
|
||||
if removal_horizon_segno > last_removed_segno {
|
||||
// we need to remove WAL
|
||||
let remover = crate::wal_storage::Storage::remove_up_to(
|
||||
&tli.read_shared_state().await.sk.wal_store,
|
||||
removal_horizon_segno,
|
||||
// update the state in Arc<Timeline>
|
||||
self.tli.wal_backup_active.store(
|
||||
self.backup_task.is_some(),
|
||||
std::sync::atomic::Ordering::Relaxed,
|
||||
);
|
||||
*wal_removal_task = Some(tokio::spawn(
|
||||
async move {
|
||||
remover.await?;
|
||||
Ok(removal_horizon_segno)
|
||||
is_wal_backup_required
|
||||
}
|
||||
|
||||
/// Update is_active flag and returns its value.
|
||||
fn update_is_active(
|
||||
&mut self,
|
||||
is_wal_backup_required: bool,
|
||||
num_computes: usize,
|
||||
state: &StateSnapshot,
|
||||
) {
|
||||
let is_active = is_wal_backup_required
|
||||
|| num_computes > 0
|
||||
|| state.remote_consistent_lsn < state.commit_lsn;
|
||||
|
||||
// update the broker timeline set
|
||||
if self.tli_broker_active.set(is_active) {
|
||||
// write log if state has changed
|
||||
info!(
|
||||
"timeline active={} now, remote_consistent_lsn={}, commit_lsn={}",
|
||||
is_active, state.remote_consistent_lsn, state.commit_lsn,
|
||||
);
|
||||
|
||||
MANAGER_ACTIVE_CHANGES.inc();
|
||||
}
|
||||
|
||||
// update the state in Arc<Timeline>
|
||||
self.tli
|
||||
.broker_active
|
||||
.store(is_active, std::sync::atomic::Ordering::Relaxed);
|
||||
}
|
||||
|
||||
/// Save control file if needed. Returns Instant if we should persist the control file in the future.
|
||||
async fn update_control_file_save(
|
||||
&self,
|
||||
state: &StateSnapshot,
|
||||
next_event: &mut Option<Instant>,
|
||||
) {
|
||||
if !state.inmem_flush_pending {
|
||||
return;
|
||||
}
|
||||
|
||||
if state.cfile_last_persist_at.elapsed() > self.conf.control_file_save_interval {
|
||||
let mut write_guard = self.tli.write_shared_state().await;
|
||||
// it should be done in the background because it blocks manager task, but flush() should
|
||||
// be fast enough not to be a problem now
|
||||
if let Err(e) = write_guard.sk.state_mut().flush().await {
|
||||
warn!("failed to save control file: {:?}", e);
|
||||
}
|
||||
.instrument(info_span!("WAL removal", ttid=%tli.ttid)),
|
||||
));
|
||||
} else {
|
||||
// we should wait until some time passed until the next save
|
||||
update_next_event(
|
||||
next_event,
|
||||
(state.cfile_last_persist_at + self.conf.control_file_save_interval).into(),
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/// Spawns WAL removal task if needed.
|
||||
async fn update_wal_removal(&mut self, state: &StateSnapshot) {
|
||||
if self.wal_removal_task.is_some() || state.wal_removal_on_hold {
|
||||
// WAL removal is already in progress or hold off
|
||||
return;
|
||||
}
|
||||
|
||||
// If enabled, we use LSN of the most lagging walsender as a WAL removal horizon.
|
||||
// This allows to get better read speed for pageservers that are lagging behind,
|
||||
// at the cost of keeping more WAL on disk.
|
||||
let replication_horizon_lsn = if self.conf.walsenders_keep_horizon {
|
||||
self.walsenders.laggard_lsn()
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
let removal_horizon_lsn = calc_horizon_lsn(state, replication_horizon_lsn);
|
||||
let removal_horizon_segno = removal_horizon_lsn
|
||||
.segment_number(self.wal_seg_size)
|
||||
.saturating_sub(1);
|
||||
|
||||
if removal_horizon_segno > self.last_removed_segno {
|
||||
// we need to remove WAL
|
||||
let remover = match self.tli.read_shared_state().await.sk {
|
||||
StateSK::Loaded(ref sk) => {
|
||||
crate::wal_storage::Storage::remove_up_to(&sk.wal_store, removal_horizon_segno)
|
||||
}
|
||||
StateSK::Offloaded(_) => {
|
||||
// we can't remove WAL if it's not loaded
|
||||
warn!("unexpectedly trying to run WAL removal on offloaded timeline");
|
||||
return;
|
||||
}
|
||||
StateSK::Empty => unreachable!(),
|
||||
};
|
||||
|
||||
self.wal_removal_task = Some(tokio::spawn(
|
||||
async move {
|
||||
remover.await?;
|
||||
Ok(removal_horizon_segno)
|
||||
}
|
||||
.instrument(info_span!("WAL removal", ttid=%self.tli.ttid)),
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
/// Update the state after WAL removal task finished.
|
||||
fn update_wal_removal_end(&mut self, res: Result<anyhow::Result<u64>, JoinError>) {
|
||||
let new_last_removed_segno = match res {
|
||||
Ok(Ok(segno)) => segno,
|
||||
Err(e) => {
|
||||
warn!("WAL removal task failed: {:?}", e);
|
||||
return;
|
||||
}
|
||||
Ok(Err(e)) => {
|
||||
warn!("WAL removal task failed: {:?}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
self.last_removed_segno = new_last_removed_segno;
|
||||
// update the state in Arc<Timeline>
|
||||
self.tli
|
||||
.last_removed_segno
|
||||
.store(new_last_removed_segno, std::sync::atomic::Ordering::Relaxed);
|
||||
}
|
||||
|
||||
/// Spawns partial WAL backup task if needed.
|
||||
async fn update_partial_backup(&mut self, state: &StateSnapshot) {
|
||||
// check if partial backup is enabled and should be started
|
||||
if !self.conf.is_wal_backup_enabled() || !self.conf.partial_backup_enabled {
|
||||
return;
|
||||
}
|
||||
|
||||
if self.partial_backup_task.is_some() {
|
||||
// partial backup is already running
|
||||
return;
|
||||
}
|
||||
|
||||
if !wal_backup_partial::needs_uploading(state, &self.partial_backup_uploaded) {
|
||||
// nothing to upload
|
||||
return;
|
||||
}
|
||||
|
||||
// Get WalResidentTimeline and start partial backup task.
|
||||
self.partial_backup_task = Some(tokio::spawn(wal_backup_partial::main_task(
|
||||
self.wal_resident_timeline(),
|
||||
self.conf.clone(),
|
||||
)));
|
||||
}
|
||||
|
||||
/// Update the state after partial WAL backup task finished.
|
||||
fn update_partial_backup_end(&mut self, res: Result<Option<PartialRemoteSegment>, JoinError>) {
|
||||
match res {
|
||||
Ok(new_upload_state) => {
|
||||
self.partial_backup_uploaded = new_upload_state;
|
||||
}
|
||||
Err(e) => {
|
||||
warn!("partial backup task panicked: {:?}", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Handle message arrived from ManagerCtl.
|
||||
async fn handle_message(&mut self, msg: Option<ManagerCtlMessage>) {
|
||||
debug!("received manager message: {:?}", msg);
|
||||
match msg {
|
||||
Some(ManagerCtlMessage::GuardRequest(tx)) => {
|
||||
if self.is_offloaded {
|
||||
// trying to unevict timeline, but without gurarantee that it will be successful
|
||||
self.unevict_timeline().await;
|
||||
}
|
||||
|
||||
let guard = if self.is_offloaded {
|
||||
Err(anyhow::anyhow!("timeline is offloaded, can't get a guard"))
|
||||
} else {
|
||||
Ok(self.access_service.create_guard())
|
||||
};
|
||||
|
||||
if tx.send(guard).is_err() {
|
||||
warn!("failed to reply with a guard, receiver dropped");
|
||||
}
|
||||
}
|
||||
Some(ManagerCtlMessage::GuardDrop(guard_id)) => {
|
||||
self.access_service.drop_guard(guard_id);
|
||||
}
|
||||
None => {
|
||||
// can't happen, we're holding the sender
|
||||
unreachable!();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Update the state after WAL removal task finished.
|
||||
fn update_wal_removal_end(
|
||||
res: Result<anyhow::Result<u64>, JoinError>,
|
||||
tli: &Arc<Timeline>,
|
||||
last_removed_segno: &mut u64,
|
||||
) {
|
||||
let new_last_removed_segno = match res {
|
||||
Ok(Ok(segno)) => segno,
|
||||
Err(e) => {
|
||||
warn!("WAL removal task failed: {:?}", e);
|
||||
return;
|
||||
}
|
||||
Ok(Err(e)) => {
|
||||
warn!("WAL removal task failed: {:?}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
*last_removed_segno = new_last_removed_segno;
|
||||
// update the state in Arc<Timeline>
|
||||
tli.last_removed_segno
|
||||
.store(new_last_removed_segno, std::sync::atomic::Ordering::Relaxed);
|
||||
// utility functions
|
||||
async fn sleep_until(option: &Option<tokio::time::Instant>) {
|
||||
if let Some(timeout) = option {
|
||||
tokio::time::sleep_until(*timeout).await;
|
||||
} else {
|
||||
futures::future::pending::<()>().await;
|
||||
}
|
||||
}
|
||||
|
||||
async fn await_task_finish<T>(option: &mut Option<JoinHandle<T>>) -> Result<T, JoinError> {
|
||||
if let Some(task) = option {
|
||||
task.await
|
||||
} else {
|
||||
futures::future::pending().await
|
||||
}
|
||||
}
|
||||
|
||||
/// Update next_event if candidate is earlier.
|
||||
fn update_next_event(next_event: &mut Option<Instant>, candidate: Instant) {
|
||||
if let Some(next) = next_event {
|
||||
if candidate < *next {
|
||||
*next = candidate;
|
||||
}
|
||||
} else {
|
||||
*next_event = Some(candidate);
|
||||
}
|
||||
}
|
||||
|
||||
#[repr(usize)]
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)]
|
||||
pub enum Status {
|
||||
NotStarted,
|
||||
Started,
|
||||
StateSnapshot,
|
||||
UpdateBackup,
|
||||
UpdateControlFile,
|
||||
UpdateWalRemoval,
|
||||
UpdatePartialBackup,
|
||||
EvictTimeline,
|
||||
Wait,
|
||||
HandleMessage,
|
||||
Exiting,
|
||||
Finished,
|
||||
}
|
||||
|
||||
/// AtomicStatus is a wrapper around AtomicUsize adapted for the Status enum.
|
||||
pub struct AtomicStatus {
|
||||
inner: AtomicUsize,
|
||||
}
|
||||
|
||||
impl Default for AtomicStatus {
|
||||
fn default() -> Self {
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
impl AtomicStatus {
|
||||
pub fn new() -> Self {
|
||||
AtomicStatus {
|
||||
inner: AtomicUsize::new(Status::NotStarted as usize),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn load(&self, order: std::sync::atomic::Ordering) -> Status {
|
||||
// Safety: This line of code uses `std::mem::transmute` to reinterpret the loaded value as `Status`.
|
||||
// It is safe to use `transmute` in this context because `Status` is a repr(usize) enum,
|
||||
// which means it has the same memory layout as usize.
|
||||
// However, it is important to ensure that the loaded value is a valid variant of `Status`,
|
||||
// otherwise, the behavior will be undefined.
|
||||
unsafe { std::mem::transmute(self.inner.load(order)) }
|
||||
}
|
||||
|
||||
pub fn get(&self) -> Status {
|
||||
self.load(std::sync::atomic::Ordering::Relaxed)
|
||||
}
|
||||
|
||||
pub fn store(&self, val: Status, order: std::sync::atomic::Ordering) {
|
||||
self.inner.store(val as usize, order);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -80,6 +80,10 @@ impl TimelineSetGuard {
|
||||
self.timelines_set.set_present(self.tli.clone(), present);
|
||||
true
|
||||
}
|
||||
|
||||
pub fn get(&self) -> bool {
|
||||
self.is_present
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for TimelineSetGuard {
|
||||
|
||||
@@ -12,7 +12,6 @@ use std::cmp::min;
|
||||
use std::collections::HashSet;
|
||||
use std::num::NonZeroU32;
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use postgres_ffi::v14::xlog_utils::XLogSegNoOffsetToRecPtr;
|
||||
@@ -30,8 +29,8 @@ use tracing::*;
|
||||
use utils::{id::TenantTimelineId, lsn::Lsn};
|
||||
|
||||
use crate::metrics::{BACKED_UP_SEGMENTS, BACKUP_ERRORS, WAL_BACKUP_TASKS};
|
||||
use crate::timeline::{FullAccessTimeline, PeerInfo, Timeline};
|
||||
use crate::timeline_manager::StateSnapshot;
|
||||
use crate::timeline::{PeerInfo, WalResidentTimeline};
|
||||
use crate::timeline_manager::{Manager, StateSnapshot};
|
||||
use crate::{SafeKeeperConf, WAL_BACKUP_RUNTIME};
|
||||
|
||||
use once_cell::sync::OnceCell;
|
||||
@@ -48,7 +47,7 @@ pub struct WalBackupTaskHandle {
|
||||
}
|
||||
|
||||
/// Do we have anything to upload to S3, i.e. should safekeepers run backup activity?
|
||||
pub fn is_wal_backup_required(
|
||||
pub(crate) fn is_wal_backup_required(
|
||||
wal_seg_size: usize,
|
||||
num_computes: usize,
|
||||
state: &StateSnapshot,
|
||||
@@ -61,35 +60,33 @@ pub fn is_wal_backup_required(
|
||||
/// Based on peer information determine which safekeeper should offload; if it
|
||||
/// is me, run (per timeline) task, if not yet. OTOH, if it is not me and task
|
||||
/// is running, kill it.
|
||||
pub async fn update_task(
|
||||
conf: &SafeKeeperConf,
|
||||
tli: &Arc<Timeline>,
|
||||
need_backup: bool,
|
||||
state: &StateSnapshot,
|
||||
entry: &mut Option<WalBackupTaskHandle>,
|
||||
) {
|
||||
pub(crate) async fn update_task(mgr: &mut Manager, need_backup: bool, state: &StateSnapshot) {
|
||||
let (offloader, election_dbg_str) =
|
||||
determine_offloader(&state.peers, state.backup_lsn, tli.ttid, conf);
|
||||
let elected_me = Some(conf.my_id) == offloader;
|
||||
determine_offloader(&state.peers, state.backup_lsn, mgr.tli.ttid, &mgr.conf);
|
||||
let elected_me = Some(mgr.conf.my_id) == offloader;
|
||||
|
||||
let should_task_run = need_backup && elected_me;
|
||||
|
||||
// start or stop the task
|
||||
if should_task_run != (entry.is_some()) {
|
||||
if should_task_run != (mgr.backup_task.is_some()) {
|
||||
if should_task_run {
|
||||
info!("elected for backup: {}", election_dbg_str);
|
||||
|
||||
let (shutdown_tx, shutdown_rx) = mpsc::channel(1);
|
||||
|
||||
let async_task = backup_task_main(tli.clone(), conf.backup_parallel_jobs, shutdown_rx);
|
||||
let async_task = backup_task_main(
|
||||
mgr.wal_resident_timeline(),
|
||||
mgr.conf.backup_parallel_jobs,
|
||||
shutdown_rx,
|
||||
);
|
||||
|
||||
let handle = if conf.current_thread_runtime {
|
||||
let handle = if mgr.conf.current_thread_runtime {
|
||||
tokio::spawn(async_task)
|
||||
} else {
|
||||
WAL_BACKUP_RUNTIME.spawn(async_task)
|
||||
};
|
||||
|
||||
*entry = Some(WalBackupTaskHandle {
|
||||
mgr.backup_task = Some(WalBackupTaskHandle {
|
||||
shutdown_tx,
|
||||
handle,
|
||||
});
|
||||
@@ -101,7 +98,7 @@ pub async fn update_task(
|
||||
// someone else has been elected
|
||||
info!("stepping down from backup: {}", election_dbg_str);
|
||||
}
|
||||
shut_down_task(entry).await;
|
||||
shut_down_task(&mut mgr.backup_task).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -191,7 +188,7 @@ pub fn init_remote_storage(conf: &SafeKeeperConf) {
|
||||
}
|
||||
|
||||
struct WalBackupTask {
|
||||
timeline: FullAccessTimeline,
|
||||
timeline: WalResidentTimeline,
|
||||
timeline_dir: Utf8PathBuf,
|
||||
wal_seg_size: usize,
|
||||
parallel_jobs: usize,
|
||||
@@ -200,16 +197,12 @@ struct WalBackupTask {
|
||||
|
||||
/// Offload single timeline.
|
||||
#[instrument(name = "WAL backup", skip_all, fields(ttid = %tli.ttid))]
|
||||
async fn backup_task_main(tli: Arc<Timeline>, parallel_jobs: usize, mut shutdown_rx: Receiver<()>) {
|
||||
async fn backup_task_main(
|
||||
tli: WalResidentTimeline,
|
||||
parallel_jobs: usize,
|
||||
mut shutdown_rx: Receiver<()>,
|
||||
) {
|
||||
let _guard = WAL_BACKUP_TASKS.guard();
|
||||
|
||||
let tli = match tli.full_access_guard().await {
|
||||
Ok(tli) => tli,
|
||||
Err(e) => {
|
||||
error!("backup error: {}", e);
|
||||
return;
|
||||
}
|
||||
};
|
||||
info!("started");
|
||||
|
||||
let mut wb = WalBackupTask {
|
||||
@@ -304,7 +297,7 @@ impl WalBackupTask {
|
||||
}
|
||||
|
||||
async fn backup_lsn_range(
|
||||
timeline: &FullAccessTimeline,
|
||||
timeline: &WalResidentTimeline,
|
||||
backup_lsn: &mut Lsn,
|
||||
end_lsn: Lsn,
|
||||
wal_seg_size: usize,
|
||||
|
||||
@@ -29,18 +29,22 @@ use utils::lsn::Lsn;
|
||||
use crate::{
|
||||
metrics::{PARTIAL_BACKUP_UPLOADED_BYTES, PARTIAL_BACKUP_UPLOADS},
|
||||
safekeeper::Term,
|
||||
timeline::FullAccessTimeline,
|
||||
timeline::WalResidentTimeline,
|
||||
timeline_manager::StateSnapshot,
|
||||
wal_backup::{self, remote_timeline_path},
|
||||
SafeKeeperConf,
|
||||
};
|
||||
|
||||
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
|
||||
pub enum UploadStatus {
|
||||
/// Upload is in progress
|
||||
/// Upload is in progress. This status should be used only for garbage collection,
|
||||
/// don't read data from the remote storage with this status.
|
||||
InProgress,
|
||||
/// Upload is finished
|
||||
/// Upload is finished. There is always at most one segment with this status.
|
||||
/// It means that the segment is actual and can be used.
|
||||
Uploaded,
|
||||
/// Deletion is in progress
|
||||
/// Deletion is in progress. This status should be used only for garbage collection,
|
||||
/// don't read data from the remote storage with this status.
|
||||
Deleting,
|
||||
}
|
||||
|
||||
@@ -50,6 +54,10 @@ pub struct PartialRemoteSegment {
|
||||
pub name: String,
|
||||
pub commit_lsn: Lsn,
|
||||
pub flush_lsn: Lsn,
|
||||
// We should use last_log_term here, otherwise it's possible to have inconsistent data in the
|
||||
// remote storage.
|
||||
//
|
||||
// More info here: https://github.com/neondatabase/neon/pull/8022#discussion_r1654738405
|
||||
pub term: Term,
|
||||
}
|
||||
|
||||
@@ -60,6 +68,10 @@ impl PartialRemoteSegment {
|
||||
&& self.flush_lsn == other.flush_lsn
|
||||
&& self.term == other.term
|
||||
}
|
||||
|
||||
pub(crate) fn remote_path(&self, remote_timeline_path: &RemotePath) -> RemotePath {
|
||||
remote_timeline_path.join(&self.name)
|
||||
}
|
||||
}
|
||||
|
||||
// NB: these structures are a part of a control_file, you can't change them without
|
||||
@@ -71,7 +83,7 @@ pub struct State {
|
||||
|
||||
impl State {
|
||||
/// Find an Uploaded segment. There should be only one Uploaded segment at a time.
|
||||
fn uploaded_segment(&self) -> Option<PartialRemoteSegment> {
|
||||
pub(crate) fn uploaded_segment(&self) -> Option<PartialRemoteSegment> {
|
||||
self.segments
|
||||
.iter()
|
||||
.find(|seg| seg.status == UploadStatus::Uploaded)
|
||||
@@ -81,7 +93,7 @@ impl State {
|
||||
|
||||
struct PartialBackup {
|
||||
wal_seg_size: usize,
|
||||
tli: FullAccessTimeline,
|
||||
tli: WalResidentTimeline,
|
||||
conf: SafeKeeperConf,
|
||||
local_prefix: Utf8PathBuf,
|
||||
remote_timeline_path: RemotePath,
|
||||
@@ -128,17 +140,17 @@ impl PartialBackup {
|
||||
let sk_info = self.tli.get_safekeeper_info(&self.conf).await;
|
||||
let flush_lsn = Lsn(sk_info.flush_lsn);
|
||||
let commit_lsn = Lsn(sk_info.commit_lsn);
|
||||
let term = sk_info.term;
|
||||
let last_log_term = sk_info.last_log_term;
|
||||
let segno = self.segno(flush_lsn);
|
||||
|
||||
let name = self.remote_segment_name(segno, term, commit_lsn, flush_lsn);
|
||||
let name = self.remote_segment_name(segno, last_log_term, commit_lsn, flush_lsn);
|
||||
|
||||
PartialRemoteSegment {
|
||||
status: UploadStatus::InProgress,
|
||||
name,
|
||||
commit_lsn,
|
||||
flush_lsn,
|
||||
term,
|
||||
term: last_log_term,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -151,7 +163,7 @@ impl PartialBackup {
|
||||
let backup_bytes = flush_lsn.segment_offset(self.wal_seg_size);
|
||||
|
||||
let local_path = self.local_prefix.join(self.local_segment_name(segno));
|
||||
let remote_path = self.remote_timeline_path.join(&prepared.name);
|
||||
let remote_path = prepared.remote_path(&self.remote_timeline_path);
|
||||
|
||||
// Upload first `backup_bytes` bytes of the segment to the remote storage.
|
||||
wal_backup::backup_partial_segment(&local_path, &remote_path, backup_bytes).await?;
|
||||
@@ -161,7 +173,7 @@ impl PartialBackup {
|
||||
// If the term changed, we cannot guarantee the validity of the uploaded data.
|
||||
// If the term is the same, we know the data is not corrupted.
|
||||
let sk_info = self.tli.get_safekeeper_info(&self.conf).await;
|
||||
if sk_info.term != prepared.term {
|
||||
if sk_info.last_log_term != prepared.term {
|
||||
anyhow::bail!("term changed during upload");
|
||||
}
|
||||
assert!(prepared.commit_lsn <= Lsn(sk_info.commit_lsn));
|
||||
@@ -270,8 +282,32 @@ impl PartialBackup {
|
||||
}
|
||||
}
|
||||
|
||||
/// Check if everything is uploaded and partial backup task doesn't need to run.
|
||||
pub(crate) fn needs_uploading(
|
||||
state: &StateSnapshot,
|
||||
uploaded: &Option<PartialRemoteSegment>,
|
||||
) -> bool {
|
||||
match uploaded {
|
||||
Some(uploaded) => {
|
||||
uploaded.status != UploadStatus::Uploaded
|
||||
|| uploaded.flush_lsn != state.flush_lsn
|
||||
|| uploaded.commit_lsn != state.commit_lsn
|
||||
|| uploaded.term != state.last_log_term
|
||||
}
|
||||
None => true,
|
||||
}
|
||||
}
|
||||
|
||||
/// Main task for partial backup. It waits for the flush_lsn to change and then uploads the
|
||||
/// partial segment to the remote storage. It also does garbage collection of old segments.
|
||||
///
|
||||
/// When there is nothing more to do and the last segment was successfully uploaded, the task
|
||||
/// returns PartialRemoteSegment, to signal readiness for offloading the timeline.
|
||||
#[instrument(name = "Partial backup", skip_all, fields(ttid = %tli.ttid))]
|
||||
pub async fn main_task(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
pub async fn main_task(
|
||||
tli: WalResidentTimeline,
|
||||
conf: SafeKeeperConf,
|
||||
) -> Option<PartialRemoteSegment> {
|
||||
debug!("started");
|
||||
let await_duration = conf.partial_backup_timeout;
|
||||
|
||||
@@ -285,7 +321,7 @@ pub async fn main_task(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
Ok(path) => path,
|
||||
Err(e) => {
|
||||
error!("failed to create remote path: {:?}", e);
|
||||
return;
|
||||
return None;
|
||||
}
|
||||
};
|
||||
|
||||
@@ -320,19 +356,13 @@ pub async fn main_task(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
// wait until we have something to upload
|
||||
let uploaded_segment = backup.state.uploaded_segment();
|
||||
if let Some(seg) = &uploaded_segment {
|
||||
// if we already uploaded something, wait until we have something new
|
||||
while flush_lsn_rx.borrow().lsn == seg.flush_lsn
|
||||
// check if uploaded segment matches the current state
|
||||
if flush_lsn_rx.borrow().lsn == seg.flush_lsn
|
||||
&& *commit_lsn_rx.borrow() == seg.commit_lsn
|
||||
&& flush_lsn_rx.borrow().term == seg.term
|
||||
{
|
||||
tokio::select! {
|
||||
_ = backup.tli.cancel.cancelled() => {
|
||||
info!("timeline canceled");
|
||||
return;
|
||||
}
|
||||
_ = commit_lsn_rx.changed() => {}
|
||||
_ = flush_lsn_rx.changed() => {}
|
||||
}
|
||||
// we have nothing to do, the last segment is already uploaded
|
||||
return Some(seg.clone());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -341,7 +371,7 @@ pub async fn main_task(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
tokio::select! {
|
||||
_ = backup.tli.cancel.cancelled() => {
|
||||
info!("timeline canceled");
|
||||
return;
|
||||
return None;
|
||||
}
|
||||
_ = flush_lsn_rx.changed() => {}
|
||||
}
|
||||
@@ -358,7 +388,7 @@ pub async fn main_task(tli: FullAccessTimeline, conf: SafeKeeperConf) {
|
||||
tokio::select! {
|
||||
_ = backup.tli.cancel.cancelled() => {
|
||||
info!("timeline canceled");
|
||||
return;
|
||||
return None;
|
||||
}
|
||||
_ = commit_lsn_rx.changed() => {}
|
||||
_ = flush_lsn_rx.changed() => {
|
||||
|
||||
@@ -211,7 +211,7 @@ impl PhysicalStorage {
|
||||
/// Returns `file` and `is_partial`.
|
||||
async fn open_or_create(&mut self, segno: XLogSegNo) -> Result<(File, bool)> {
|
||||
let (wal_file_path, wal_file_partial_path) =
|
||||
wal_file_paths(&self.timeline_dir, segno, self.wal_seg_size)?;
|
||||
wal_file_paths(&self.timeline_dir, segno, self.wal_seg_size);
|
||||
|
||||
// Try to open already completed segment
|
||||
if let Ok(file) = OpenOptions::new().write(true).open(&wal_file_path).await {
|
||||
@@ -276,7 +276,7 @@ impl PhysicalStorage {
|
||||
|
||||
// Rename partial file to completed file
|
||||
let (wal_file_path, wal_file_partial_path) =
|
||||
wal_file_paths(&self.timeline_dir, segno, self.wal_seg_size)?;
|
||||
wal_file_paths(&self.timeline_dir, segno, self.wal_seg_size);
|
||||
fs::rename(wal_file_partial_path, wal_file_path).await?;
|
||||
} else {
|
||||
// otherwise, file can be reused later
|
||||
@@ -461,7 +461,7 @@ impl Storage for PhysicalStorage {
|
||||
if !is_partial {
|
||||
// Make segment partial once again
|
||||
let (wal_file_path, wal_file_partial_path) =
|
||||
wal_file_paths(&self.timeline_dir, segno, self.wal_seg_size)?;
|
||||
wal_file_paths(&self.timeline_dir, segno, self.wal_seg_size);
|
||||
fs::rename(wal_file_path, wal_file_partial_path).await?;
|
||||
}
|
||||
|
||||
@@ -741,7 +741,7 @@ pub(crate) async fn open_wal_file(
|
||||
segno: XLogSegNo,
|
||||
wal_seg_size: usize,
|
||||
) -> Result<(tokio::fs::File, bool)> {
|
||||
let (wal_file_path, wal_file_partial_path) = wal_file_paths(timeline_dir, segno, wal_seg_size)?;
|
||||
let (wal_file_path, wal_file_partial_path) = wal_file_paths(timeline_dir, segno, wal_seg_size);
|
||||
|
||||
// First try to open the .partial file.
|
||||
let mut partial_path = wal_file_path.to_owned();
|
||||
@@ -767,9 +767,9 @@ pub fn wal_file_paths(
|
||||
timeline_dir: &Utf8Path,
|
||||
segno: XLogSegNo,
|
||||
wal_seg_size: usize,
|
||||
) -> Result<(Utf8PathBuf, Utf8PathBuf)> {
|
||||
) -> (Utf8PathBuf, Utf8PathBuf) {
|
||||
let wal_file_name = XLogFileName(PG_TLI, segno, wal_seg_size);
|
||||
let wal_file_path = timeline_dir.join(wal_file_name.clone());
|
||||
let wal_file_partial_path = timeline_dir.join(wal_file_name + ".partial");
|
||||
Ok((wal_file_path, wal_file_partial_path))
|
||||
(wal_file_path, wal_file_partial_path)
|
||||
}
|
||||
|
||||
@@ -16,7 +16,7 @@ use desim::{
|
||||
use hyper::Uri;
|
||||
use safekeeper::{
|
||||
safekeeper::{ProposerAcceptorMessage, SafeKeeper, ServerInfo, UNKNOWN_SERVER_VERSION},
|
||||
state::TimelinePersistentState,
|
||||
state::{TimelinePersistentState, TimelineState},
|
||||
timeline::TimelineError,
|
||||
wal_storage::Storage,
|
||||
SafeKeeperConf,
|
||||
@@ -68,7 +68,7 @@ impl GlobalMap {
|
||||
let control_store = DiskStateStorage::new(disk.clone());
|
||||
let wal_store = DiskWALStorage::new(disk.clone(), &control_store)?;
|
||||
|
||||
let sk = SafeKeeper::new(control_store, wal_store, conf.my_id)?;
|
||||
let sk = SafeKeeper::new(TimelineState::new(control_store), wal_store, conf.my_id)?;
|
||||
timelines.insert(
|
||||
ttid,
|
||||
SharedState {
|
||||
@@ -118,7 +118,11 @@ impl GlobalMap {
|
||||
let control_store = DiskStateStorage::new(disk_timeline.clone());
|
||||
let wal_store = DiskWALStorage::new(disk_timeline.clone(), &control_store)?;
|
||||
|
||||
let sk = SafeKeeper::new(control_store, wal_store, self.conf.my_id)?;
|
||||
let sk = SafeKeeper::new(
|
||||
TimelineState::new(control_store),
|
||||
wal_store,
|
||||
self.conf.my_id,
|
||||
)?;
|
||||
|
||||
self.timelines.insert(
|
||||
ttid,
|
||||
@@ -180,6 +184,9 @@ pub fn run_server(os: NodeOs, disk: Arc<SafekeeperDisk>) -> Result<()> {
|
||||
partial_backup_enabled: false,
|
||||
partial_backup_timeout: Duration::from_secs(0),
|
||||
disable_periodic_broker_push: false,
|
||||
enable_offload: false,
|
||||
delete_offloaded_wal: false,
|
||||
control_file_save_interval: Duration::from_secs(1),
|
||||
};
|
||||
|
||||
let mut global = GlobalMap::new(disk, conf.clone())?;
|
||||
|
||||
@@ -3916,6 +3916,8 @@ class Safekeeper(LogUtils):
|
||||
|
||||
def assert_no_errors(self):
|
||||
assert not self.log_contains("manager task finished prematurely")
|
||||
assert not self.log_contains("error while acquiring WalResidentTimeline guard")
|
||||
assert not self.log_contains("timeout while acquiring WalResidentTimeline guard")
|
||||
|
||||
def append_logical_message(
|
||||
self, tenant_id: TenantId, timeline_id: TimelineId, request: Dict[str, Any]
|
||||
|
||||
@@ -1,4 +1,5 @@
|
||||
import filecmp
|
||||
import logging
|
||||
import os
|
||||
import random
|
||||
import shutil
|
||||
@@ -2178,3 +2179,102 @@ def test_broker_discovery(neon_env_builder: NeonEnvBuilder):
|
||||
|
||||
do_something()
|
||||
do_something()
|
||||
|
||||
|
||||
# Test creates 5 endpoints and tries to wake them up randomly. All timeouts are
|
||||
# configured to be very short, so that we expect that:
|
||||
# - pageserver will update remote_consistent_lsn very often
|
||||
# - safekeepers will upload partial WAL segments very often
|
||||
# - safekeeper will try to evict and unevict timelines
|
||||
#
|
||||
# Test checks that there are no critical errors while doing this. Also it checks
|
||||
# that every safekeeper has at least one successful eviction.
|
||||
@pytest.mark.parametrize("delete_offloaded_wal", [False, True])
|
||||
@pytest.mark.parametrize("restart_chance", [0.0, 0.2])
|
||||
def test_s3_eviction(
|
||||
neon_env_builder: NeonEnvBuilder, delete_offloaded_wal: bool, restart_chance: float
|
||||
):
|
||||
neon_env_builder.num_safekeepers = 3
|
||||
neon_env_builder.enable_safekeeper_remote_storage(RemoteStorageKind.LOCAL_FS)
|
||||
env = neon_env_builder.init_start(
|
||||
initial_tenant_conf={
|
||||
"checkpoint_timeout": "100ms",
|
||||
}
|
||||
)
|
||||
|
||||
extra_opts = [
|
||||
"--enable-offload",
|
||||
"--partial-backup-timeout",
|
||||
"50ms",
|
||||
"--control-file-save-interval",
|
||||
"1s",
|
||||
]
|
||||
if delete_offloaded_wal:
|
||||
extra_opts.append("--delete-offloaded-wal")
|
||||
|
||||
for sk in env.safekeepers:
|
||||
sk.stop().start(extra_opts=extra_opts)
|
||||
|
||||
n_timelines = 5
|
||||
|
||||
branch_names = [f"branch{tlin}" for tlin in range(n_timelines)]
|
||||
timelines = []
|
||||
ps_client = env.pageservers[0].http_client()
|
||||
|
||||
# start postgres on each timeline
|
||||
endpoints: list[Endpoint] = []
|
||||
for branch_name in branch_names:
|
||||
timeline_id = env.neon_cli.create_branch(branch_name)
|
||||
timelines.append(timeline_id)
|
||||
|
||||
endpoints.append(env.endpoints.create_start(branch_name))
|
||||
endpoints[-1].safe_psql("CREATE TABLE t(i int)")
|
||||
endpoints[-1].safe_psql("INSERT INTO t VALUES (0)")
|
||||
|
||||
lsn = endpoints[-1].safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0]
|
||||
log.info(f"{branch_name}: LSN={lsn}")
|
||||
|
||||
endpoints[-1].stop()
|
||||
|
||||
# update remote_consistent_lsn on pageserver
|
||||
ps_client.timeline_checkpoint(env.initial_tenant, timelines[-1], wait_until_uploaded=True)
|
||||
|
||||
check_values = [0] * n_timelines
|
||||
|
||||
n_iters = 20
|
||||
for _ in range(n_iters):
|
||||
if log.isEnabledFor(logging.DEBUG):
|
||||
for j in range(n_timelines):
|
||||
detail = ps_client.timeline_detail(env.initial_tenant, timelines[j])
|
||||
log.debug(
|
||||
f'{branch_names[j]}: RCL={detail["remote_consistent_lsn"]}, LRL={detail["last_record_lsn"]}'
|
||||
)
|
||||
|
||||
i = random.randint(0, n_timelines - 1)
|
||||
log.info(f"Starting endpoint {i}")
|
||||
endpoints[i].start()
|
||||
check_values[i] += 1
|
||||
res = endpoints[i].safe_psql("UPDATE t SET i = i + 1 RETURNING i")
|
||||
assert res[0][0] == check_values[i]
|
||||
|
||||
lsn = endpoints[i].safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0]
|
||||
log.info(f"{branch_names[i]}: LSN={lsn}")
|
||||
|
||||
endpoints[i].stop()
|
||||
|
||||
# update remote_consistent_lsn on pageserver
|
||||
ps_client.timeline_checkpoint(env.initial_tenant, timelines[i], wait_until_uploaded=True)
|
||||
|
||||
# restarting random safekeepers
|
||||
for sk in env.safekeepers:
|
||||
if random.random() < restart_chance:
|
||||
sk.stop().start(extra_opts=extra_opts)
|
||||
time.sleep(0.5)
|
||||
|
||||
# require at least one successful eviction in at least one safekeeper
|
||||
# TODO: require eviction in each safekeeper after https://github.com/neondatabase/neon/issues/8148 is fixed
|
||||
assert any(
|
||||
sk.log_contains("successfully evicted timeline")
|
||||
and sk.log_contains("successfully restored evicted timeline")
|
||||
for sk in env.safekeepers
|
||||
)
|
||||
|
||||
@@ -200,9 +200,8 @@ async def run_restarts_under_load(
|
||||
# assert that at least one transaction has completed in every worker
|
||||
stats.check_progress()
|
||||
|
||||
# testing #6530, temporary here
|
||||
# TODO: remove afer partial backup is enabled by default
|
||||
victim.start(extra_opts=["--partial-backup-enabled", "--partial-backup-timeout=2s"])
|
||||
# testing #6530
|
||||
victim.start(extra_opts=["--partial-backup-timeout=2s"])
|
||||
|
||||
log.info("Iterations are finished, exiting coroutines...")
|
||||
stats.running = False
|
||||
|
||||
Reference in New Issue
Block a user