mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-05 12:32:54 +00:00
safekeeper: refactor timeline initialization (#9362)
Always do timeline init through atomic rename of temp directory. Add GlobalTimelines::load_temp_timeline which does this, and use it from both pull_timeline and basic timeline creation. Fixes a collection of issues: - previously timeline creation didn't really flushed cfile to disk due to 'nothing to do if state didn't change' check; - even if it did, without tmp dir it is possible to lose the cfile but leave timeline dir in place, making it look corrupted; - tenant directory creation fsync was missing in timeline creation; - pull_timeline is now protected from concurrent both itself and timeline creation; - now global timelines map entry got special CreationInProgress entry type which prevents from anyone getting access to timeline while it is being created (previously one could get access to it, but it was locked during creation, which is valid but confusing if creation failed). fixes #8927
This commit is contained in:
@@ -66,22 +66,25 @@ impl FileStorage {
|
||||
})
|
||||
}
|
||||
|
||||
/// Create file storage for a new timeline, but don't persist it yet.
|
||||
pub fn create_new(
|
||||
timeline_dir: Utf8PathBuf,
|
||||
/// Create and reliably persist new control file at given location.
|
||||
///
|
||||
/// Note: we normally call this in temp directory for atomic init, so
|
||||
/// interested in FileStorage as a result only in tests.
|
||||
pub async fn create_new(
|
||||
dir: Utf8PathBuf,
|
||||
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,
|
||||
let mut store = FileStorage {
|
||||
timeline_dir: dir,
|
||||
no_sync: conf.no_sync,
|
||||
state,
|
||||
state: state.clone(),
|
||||
last_persist_at: Instant::now(),
|
||||
};
|
||||
|
||||
store.persist(&state).await?;
|
||||
Ok(store)
|
||||
}
|
||||
|
||||
@@ -190,8 +193,6 @@ impl TimelinePersistentState {
|
||||
|
||||
impl Storage for FileStorage {
|
||||
/// Persists state durably to the underlying storage.
|
||||
///
|
||||
/// For a description, see <https://lwn.net/Articles/457667/>.
|
||||
async fn persist(&mut self, s: &TimelinePersistentState) -> Result<()> {
|
||||
let _timer = PERSIST_CONTROL_FILE_SECONDS.start_timer();
|
||||
|
||||
@@ -269,7 +270,7 @@ mod test {
|
||||
.await
|
||||
.expect("failed to create timeline dir");
|
||||
let state = TimelinePersistentState::empty();
|
||||
let storage = FileStorage::create_new(timeline_dir, conf, state.clone())?;
|
||||
let storage = FileStorage::create_new(timeline_dir, conf, state.clone()).await?;
|
||||
Ok((storage, state))
|
||||
}
|
||||
|
||||
|
||||
@@ -12,10 +12,10 @@ use tracing::{info, warn};
|
||||
use utils::{id::TenantTimelineId, lsn::Lsn};
|
||||
|
||||
use crate::{
|
||||
control_file::{FileStorage, Storage},
|
||||
pull_timeline::{create_temp_timeline_dir, load_temp_timeline, validate_temp_timeline},
|
||||
control_file::FileStorage,
|
||||
state::TimelinePersistentState,
|
||||
timeline::{Timeline, TimelineError, WalResidentTimeline},
|
||||
timelines_global_map::{create_temp_timeline_dir, validate_temp_timeline},
|
||||
wal_backup::copy_s3_segments,
|
||||
wal_storage::{wal_file_paths, WalReader},
|
||||
GlobalTimelines,
|
||||
@@ -149,17 +149,16 @@ pub async fn handle_request(request: Request) -> Result<()> {
|
||||
vec![],
|
||||
request.until_lsn,
|
||||
start_lsn,
|
||||
);
|
||||
)?;
|
||||
new_state.timeline_start_lsn = start_lsn;
|
||||
new_state.peer_horizon_lsn = request.until_lsn;
|
||||
new_state.backup_lsn = new_backup_lsn;
|
||||
|
||||
let mut file_storage = FileStorage::create_new(tli_dir_path.clone(), conf, new_state.clone())?;
|
||||
file_storage.persist(&new_state).await?;
|
||||
FileStorage::create_new(tli_dir_path.clone(), conf, new_state.clone()).await?;
|
||||
|
||||
// now we have a ready timeline in a temp directory
|
||||
validate_temp_timeline(conf, request.destination_ttid, &tli_dir_path).await?;
|
||||
load_temp_timeline(conf, request.destination_ttid, &tli_dir_path).await?;
|
||||
GlobalTimelines::load_temp_timeline(request.destination_ttid, &tli_dir_path, true).await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -1,7 +1,6 @@
|
||||
use anyhow::{anyhow, bail, Context, Result};
|
||||
use bytes::Bytes;
|
||||
use camino::Utf8PathBuf;
|
||||
use camino_tempfile::Utf8TempDir;
|
||||
use chrono::{DateTime, Utc};
|
||||
use futures::{SinkExt, StreamExt, TryStreamExt};
|
||||
use postgres_ffi::{XLogFileName, XLogSegNo, PG_TLI};
|
||||
@@ -9,7 +8,6 @@ use serde::{Deserialize, Serialize};
|
||||
use std::{
|
||||
cmp::min,
|
||||
io::{self, ErrorKind},
|
||||
sync::Arc,
|
||||
};
|
||||
use tokio::{fs::OpenOptions, io::AsyncWrite, sync::mpsc, task};
|
||||
use tokio_tar::{Archive, Builder, Header};
|
||||
@@ -20,7 +18,7 @@ use tokio_util::{
|
||||
use tracing::{error, info, instrument};
|
||||
|
||||
use crate::{
|
||||
control_file::{self, CONTROL_FILE_NAME},
|
||||
control_file::CONTROL_FILE_NAME,
|
||||
debug_dump,
|
||||
http::{
|
||||
client::{self, Client},
|
||||
@@ -28,13 +26,14 @@ use crate::{
|
||||
},
|
||||
safekeeper::Term,
|
||||
state::TimelinePersistentState,
|
||||
timeline::{get_tenant_dir, get_timeline_dir, Timeline, TimelineError, WalResidentTimeline},
|
||||
timeline::WalResidentTimeline,
|
||||
timelines_global_map::{create_temp_timeline_dir, validate_temp_timeline},
|
||||
wal_backup,
|
||||
wal_storage::{self, open_wal_file, Storage},
|
||||
GlobalTimelines, SafeKeeperConf,
|
||||
wal_storage::open_wal_file,
|
||||
GlobalTimelines,
|
||||
};
|
||||
use utils::{
|
||||
crashsafe::{durable_rename, fsync_async_opt},
|
||||
crashsafe::fsync_async_opt,
|
||||
id::{NodeId, TenantId, TenantTimelineId, TimelineId},
|
||||
logging::SecretString,
|
||||
lsn::Lsn,
|
||||
@@ -428,100 +427,9 @@ async fn pull_timeline(
|
||||
assert!(status.commit_lsn <= status.flush_lsn);
|
||||
|
||||
// Finally, load the timeline.
|
||||
let _tli = load_temp_timeline(conf, ttid, &tli_dir_path).await?;
|
||||
let _tli = GlobalTimelines::load_temp_timeline(ttid, &tli_dir_path, false).await?;
|
||||
|
||||
Ok(Response {
|
||||
safekeeper_host: host,
|
||||
})
|
||||
}
|
||||
|
||||
/// Create temp directory for a new timeline. It needs to be located on the same
|
||||
/// filesystem as the rest of the timelines. It will be automatically deleted when
|
||||
/// Utf8TempDir goes out of scope.
|
||||
pub async fn create_temp_timeline_dir(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: TenantTimelineId,
|
||||
) -> Result<(Utf8TempDir, Utf8PathBuf)> {
|
||||
// conf.workdir is usually /storage/safekeeper/data
|
||||
// will try to transform it into /storage/safekeeper/tmp
|
||||
let temp_base = conf
|
||||
.workdir
|
||||
.parent()
|
||||
.ok_or(anyhow::anyhow!("workdir has no parent"))?
|
||||
.join("tmp");
|
||||
|
||||
tokio::fs::create_dir_all(&temp_base).await?;
|
||||
|
||||
let tli_dir = camino_tempfile::Builder::new()
|
||||
.suffix("_temptli")
|
||||
.prefix(&format!("{}_{}_", ttid.tenant_id, ttid.timeline_id))
|
||||
.tempdir_in(temp_base)?;
|
||||
|
||||
let tli_dir_path = tli_dir.path().to_path_buf();
|
||||
|
||||
Ok((tli_dir, tli_dir_path))
|
||||
}
|
||||
|
||||
/// Do basic validation of a temp timeline, before moving it to the global map.
|
||||
pub async fn validate_temp_timeline(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: TenantTimelineId,
|
||||
path: &Utf8PathBuf,
|
||||
) -> Result<(Lsn, Lsn)> {
|
||||
let control_path = path.join("safekeeper.control");
|
||||
|
||||
let control_store = control_file::FileStorage::load_control_file(control_path)?;
|
||||
if control_store.server.wal_seg_size == 0 {
|
||||
bail!("wal_seg_size is not set");
|
||||
}
|
||||
|
||||
let wal_store = wal_storage::PhysicalStorage::new(&ttid, path.clone(), conf, &control_store)?;
|
||||
|
||||
let commit_lsn = control_store.commit_lsn;
|
||||
let flush_lsn = wal_store.flush_lsn();
|
||||
|
||||
Ok((commit_lsn, flush_lsn))
|
||||
}
|
||||
|
||||
/// Move timeline from a temp directory to the main storage, and load it to the global map.
|
||||
///
|
||||
/// This operation is done under a lock to prevent bugs if several concurrent requests are
|
||||
/// trying to load the same timeline. Note that it doesn't guard against creating the
|
||||
/// timeline with the same ttid, but no one should be doing this anyway.
|
||||
pub async fn load_temp_timeline(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: TenantTimelineId,
|
||||
tmp_path: &Utf8PathBuf,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
// Take a lock to prevent concurrent loadings
|
||||
let load_lock = GlobalTimelines::loading_lock().await;
|
||||
let guard = load_lock.lock().await;
|
||||
|
||||
if !matches!(GlobalTimelines::get(ttid), Err(TimelineError::NotFound(_))) {
|
||||
bail!("timeline already exists, cannot overwrite it")
|
||||
}
|
||||
|
||||
// Move timeline dir to the correct location
|
||||
let timeline_path = get_timeline_dir(conf, &ttid);
|
||||
|
||||
info!(
|
||||
"moving timeline {} from {} to {}",
|
||||
ttid, tmp_path, timeline_path
|
||||
);
|
||||
tokio::fs::create_dir_all(get_tenant_dir(conf, &ttid.tenant_id)).await?;
|
||||
// fsync tenant dir creation
|
||||
fsync_async_opt(&conf.workdir, !conf.no_sync).await?;
|
||||
durable_rename(tmp_path, &timeline_path, !conf.no_sync).await?;
|
||||
|
||||
let tli = GlobalTimelines::load_timeline(&guard, ttid)
|
||||
.await
|
||||
.context("Failed to load timeline after copy")?;
|
||||
|
||||
info!(
|
||||
"loaded timeline {}, flush_lsn={}",
|
||||
ttid,
|
||||
tli.get_flush_lsn().await
|
||||
);
|
||||
|
||||
Ok(tli)
|
||||
}
|
||||
|
||||
@@ -339,7 +339,8 @@ impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> {
|
||||
};
|
||||
let tli =
|
||||
GlobalTimelines::create(self.ttid, server_info, Lsn::INVALID, Lsn::INVALID)
|
||||
.await?;
|
||||
.await
|
||||
.context("create timeline")?;
|
||||
tli.wal_residence_guard().await?
|
||||
}
|
||||
_ => {
|
||||
|
||||
@@ -3,7 +3,7 @@
|
||||
|
||||
use std::{cmp::max, ops::Deref};
|
||||
|
||||
use anyhow::Result;
|
||||
use anyhow::{bail, Result};
|
||||
use safekeeper_api::models::TimelineTermBumpResponse;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use utils::{
|
||||
@@ -13,7 +13,11 @@ use utils::{
|
||||
|
||||
use crate::{
|
||||
control_file,
|
||||
safekeeper::{AcceptorState, PersistedPeerInfo, PgUuid, ServerInfo, Term, TermHistory},
|
||||
safekeeper::{
|
||||
AcceptorState, PersistedPeerInfo, PgUuid, ServerInfo, Term, TermHistory,
|
||||
UNKNOWN_SERVER_VERSION,
|
||||
},
|
||||
timeline::TimelineError,
|
||||
wal_backup_partial::{self},
|
||||
};
|
||||
|
||||
@@ -91,8 +95,24 @@ impl TimelinePersistentState {
|
||||
peers: Vec<NodeId>,
|
||||
commit_lsn: Lsn,
|
||||
local_start_lsn: Lsn,
|
||||
) -> TimelinePersistentState {
|
||||
TimelinePersistentState {
|
||||
) -> anyhow::Result<TimelinePersistentState> {
|
||||
if server_info.wal_seg_size == 0 {
|
||||
bail!(TimelineError::UninitializedWalSegSize(*ttid));
|
||||
}
|
||||
|
||||
if server_info.pg_version == UNKNOWN_SERVER_VERSION {
|
||||
bail!(TimelineError::UninitialinzedPgVersion(*ttid));
|
||||
}
|
||||
|
||||
if commit_lsn < local_start_lsn {
|
||||
bail!(
|
||||
"commit_lsn {} is smaller than local_start_lsn {}",
|
||||
commit_lsn,
|
||||
local_start_lsn
|
||||
);
|
||||
}
|
||||
|
||||
Ok(TimelinePersistentState {
|
||||
tenant_id: ttid.tenant_id,
|
||||
timeline_id: ttid.timeline_id,
|
||||
acceptor_state: AcceptorState {
|
||||
@@ -115,24 +135,23 @@ impl TimelinePersistentState {
|
||||
),
|
||||
partial_backup: wal_backup_partial::State::default(),
|
||||
eviction_state: EvictionState::Present,
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub fn empty() -> Self {
|
||||
use crate::safekeeper::UNKNOWN_SERVER_VERSION;
|
||||
|
||||
TimelinePersistentState::new(
|
||||
&TenantTimelineId::empty(),
|
||||
ServerInfo {
|
||||
pg_version: UNKNOWN_SERVER_VERSION, /* Postgres server version */
|
||||
system_id: 0, /* Postgres system identifier */
|
||||
wal_seg_size: 0,
|
||||
pg_version: 17, /* Postgres server version */
|
||||
system_id: 0, /* Postgres system identifier */
|
||||
wal_seg_size: 16 * 1024 * 1024,
|
||||
},
|
||||
vec![],
|
||||
Lsn::INVALID,
|
||||
Lsn::INVALID,
|
||||
)
|
||||
.unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -27,11 +27,11 @@ use utils::{
|
||||
use storage_broker::proto::SafekeeperTimelineInfo;
|
||||
use storage_broker::proto::TenantTimelineId as ProtoTenantTimelineId;
|
||||
|
||||
use crate::control_file;
|
||||
use crate::rate_limit::RateLimiter;
|
||||
use crate::receive_wal::WalReceivers;
|
||||
use crate::safekeeper::{
|
||||
AcceptorProposerMessage, ProposerAcceptorMessage, SafeKeeper, ServerInfo, Term, TermLsn,
|
||||
INVALID_TERM,
|
||||
AcceptorProposerMessage, ProposerAcceptorMessage, SafeKeeper, Term, TermLsn,
|
||||
};
|
||||
use crate::send_wal::WalSenders;
|
||||
use crate::state::{EvictionState, TimelineMemState, TimelinePersistentState, TimelineState};
|
||||
@@ -40,7 +40,6 @@ use crate::timeline_manager::{AtomicStatus, ManagerCtl};
|
||||
use crate::timelines_set::TimelinesSet;
|
||||
use crate::wal_backup::{self, remote_timeline_path};
|
||||
use crate::wal_backup_partial::PartialRemoteSegment;
|
||||
use crate::{control_file, safekeeper::UNKNOWN_SERVER_VERSION};
|
||||
|
||||
use crate::metrics::{FullTimelineInfo, WalStorageMetrics, MISC_OPERATION_SECONDS};
|
||||
use crate::wal_storage::{Storage as wal_storage_iface, WalReader};
|
||||
@@ -326,44 +325,6 @@ pub struct SharedState {
|
||||
}
|
||||
|
||||
impl SharedState {
|
||||
/// Initialize fresh timeline state without persisting anything to disk.
|
||||
fn create_new(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: &TenantTimelineId,
|
||||
state: TimelinePersistentState,
|
||||
) -> Result<Self> {
|
||||
if state.server.wal_seg_size == 0 {
|
||||
bail!(TimelineError::UninitializedWalSegSize(*ttid));
|
||||
}
|
||||
|
||||
if state.server.pg_version == UNKNOWN_SERVER_VERSION {
|
||||
bail!(TimelineError::UninitialinzedPgVersion(*ttid));
|
||||
}
|
||||
|
||||
if state.commit_lsn < state.local_start_lsn {
|
||||
bail!(
|
||||
"commit_lsn {} is higher than local_start_lsn {}",
|
||||
state.commit_lsn,
|
||||
state.local_start_lsn
|
||||
);
|
||||
}
|
||||
|
||||
// We don't want to write anything to disk, because we may have existing timeline there.
|
||||
// These functions should not change anything on disk.
|
||||
let timeline_dir = get_timeline_dir(conf, ttid);
|
||||
let control_store =
|
||||
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(TimelineState::new(control_store), wal_store, conf.my_id)?;
|
||||
|
||||
Ok(Self {
|
||||
sk: StateSK::Loaded(sk),
|
||||
peers_info: PeersInfo(vec![]),
|
||||
wal_removal_on_hold: false,
|
||||
})
|
||||
}
|
||||
|
||||
/// Restore SharedState from control file. If file doesn't exist, bails out.
|
||||
fn restore(conf: &SafeKeeperConf, ttid: &TenantTimelineId) -> Result<Self> {
|
||||
let timeline_dir = get_timeline_dir(conf, ttid);
|
||||
@@ -450,6 +411,8 @@ pub enum TimelineError {
|
||||
Cancelled(TenantTimelineId),
|
||||
#[error("Timeline {0} was not found in global map")]
|
||||
NotFound(TenantTimelineId),
|
||||
#[error("Timeline {0} creation is in progress")]
|
||||
CreationInProgress(TenantTimelineId),
|
||||
#[error("Timeline {0} exists on disk, but wasn't loaded on startup")]
|
||||
Invalid(TenantTimelineId),
|
||||
#[error("Timeline {0} is already exists")]
|
||||
@@ -514,7 +477,7 @@ pub struct Timeline {
|
||||
|
||||
impl Timeline {
|
||||
/// Load existing timeline from disk.
|
||||
pub fn load_timeline(conf: &SafeKeeperConf, ttid: TenantTimelineId) -> Result<Timeline> {
|
||||
pub fn load_timeline(conf: &SafeKeeperConf, ttid: TenantTimelineId) -> Result<Arc<Timeline>> {
|
||||
let _enter = info_span!("load_timeline", timeline = %ttid.timeline_id).entered();
|
||||
|
||||
let shared_state = SharedState::restore(conf, &ttid)?;
|
||||
@@ -528,7 +491,7 @@ impl Timeline {
|
||||
|
||||
let walreceivers = WalReceivers::new();
|
||||
let remote_path = remote_timeline_path(&ttid)?;
|
||||
Ok(Timeline {
|
||||
Ok(Arc::new(Timeline {
|
||||
ttid,
|
||||
remote_path,
|
||||
commit_lsn_watch_tx,
|
||||
@@ -547,47 +510,7 @@ impl Timeline {
|
||||
wal_backup_active: AtomicBool::new(false),
|
||||
last_removed_segno: AtomicU64::new(0),
|
||||
mgr_status: AtomicStatus::new(),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new timeline, which is not yet persisted to disk.
|
||||
pub fn create_empty(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: TenantTimelineId,
|
||||
server_info: ServerInfo,
|
||||
commit_lsn: Lsn,
|
||||
local_start_lsn: Lsn,
|
||||
) -> Result<Timeline> {
|
||||
let (commit_lsn_watch_tx, commit_lsn_watch_rx) = watch::channel(Lsn::INVALID);
|
||||
let (term_flush_lsn_watch_tx, term_flush_lsn_watch_rx) =
|
||||
watch::channel(TermLsn::from((INVALID_TERM, Lsn::INVALID)));
|
||||
let (shared_state_version_tx, shared_state_version_rx) = watch::channel(0);
|
||||
|
||||
let state =
|
||||
TimelinePersistentState::new(&ttid, server_info, vec![], commit_lsn, local_start_lsn);
|
||||
|
||||
let walreceivers = WalReceivers::new();
|
||||
let remote_path = remote_timeline_path(&ttid)?;
|
||||
Ok(Timeline {
|
||||
ttid,
|
||||
remote_path,
|
||||
commit_lsn_watch_tx,
|
||||
commit_lsn_watch_rx,
|
||||
term_flush_lsn_watch_tx,
|
||||
term_flush_lsn_watch_rx,
|
||||
shared_state_version_tx,
|
||||
shared_state_version_rx,
|
||||
mutex: RwLock::new(SharedState::create_new(conf, &ttid, state)?),
|
||||
walsenders: WalSenders::new(walreceivers.clone()),
|
||||
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(),
|
||||
})
|
||||
}))
|
||||
}
|
||||
|
||||
/// Initialize fresh timeline on disk and start background tasks. If init
|
||||
|
||||
@@ -5,11 +5,14 @@
|
||||
use crate::defaults::DEFAULT_EVICTION_CONCURRENCY;
|
||||
use crate::rate_limit::RateLimiter;
|
||||
use crate::safekeeper::ServerInfo;
|
||||
use crate::state::TimelinePersistentState;
|
||||
use crate::timeline::{get_tenant_dir, get_timeline_dir, Timeline, TimelineError};
|
||||
use crate::timelines_set::TimelinesSet;
|
||||
use crate::SafeKeeperConf;
|
||||
use crate::wal_storage::Storage;
|
||||
use crate::{control_file, wal_storage, SafeKeeperConf};
|
||||
use anyhow::{bail, Context, Result};
|
||||
use camino::Utf8PathBuf;
|
||||
use camino_tempfile::Utf8TempDir;
|
||||
use once_cell::sync::Lazy;
|
||||
use serde::Serialize;
|
||||
use std::collections::HashMap;
|
||||
@@ -17,12 +20,22 @@ use std::str::FromStr;
|
||||
use std::sync::atomic::Ordering;
|
||||
use std::sync::{Arc, Mutex};
|
||||
use std::time::{Duration, Instant};
|
||||
use tokio::fs;
|
||||
use tracing::*;
|
||||
use utils::crashsafe::{durable_rename, fsync_async_opt};
|
||||
use utils::id::{TenantId, TenantTimelineId, TimelineId};
|
||||
use utils::lsn::Lsn;
|
||||
|
||||
// Timeline entry in the global map: either a ready timeline, or mark that it is
|
||||
// being created.
|
||||
#[derive(Clone)]
|
||||
enum GlobalMapTimeline {
|
||||
CreationInProgress,
|
||||
Timeline(Arc<Timeline>),
|
||||
}
|
||||
|
||||
struct GlobalTimelinesState {
|
||||
timelines: HashMap<TenantTimelineId, Arc<Timeline>>,
|
||||
timelines: HashMap<TenantTimelineId, GlobalMapTimeline>,
|
||||
|
||||
// A tombstone indicates this timeline used to exist has been deleted. These are used to prevent
|
||||
// on-demand timeline creation from recreating deleted timelines. This is only soft-enforced, as
|
||||
@@ -31,13 +44,9 @@ struct GlobalTimelinesState {
|
||||
|
||||
conf: Option<SafeKeeperConf>,
|
||||
broker_active_set: Arc<TimelinesSet>,
|
||||
load_lock: Arc<tokio::sync::Mutex<TimelineLoadLock>>,
|
||||
global_rate_limiter: RateLimiter,
|
||||
}
|
||||
|
||||
// Used to prevent concurrent timeline loading.
|
||||
pub struct TimelineLoadLock;
|
||||
|
||||
impl GlobalTimelinesState {
|
||||
/// Get configuration, which must be set once during init.
|
||||
fn get_conf(&self) -> &SafeKeeperConf {
|
||||
@@ -55,22 +64,16 @@ impl GlobalTimelinesState {
|
||||
)
|
||||
}
|
||||
|
||||
/// Insert timeline into the map. Returns error if timeline with the same id already exists.
|
||||
fn try_insert(&mut self, timeline: Arc<Timeline>) -> Result<()> {
|
||||
let ttid = timeline.ttid;
|
||||
if self.timelines.contains_key(&ttid) {
|
||||
bail!(TimelineError::AlreadyExists(ttid));
|
||||
}
|
||||
self.timelines.insert(ttid, timeline);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Get timeline from the map. Returns error if timeline doesn't exist.
|
||||
/// Get timeline from the map. Returns error if timeline doesn't exist or
|
||||
/// creation is in progress.
|
||||
fn get(&self, ttid: &TenantTimelineId) -> Result<Arc<Timeline>, TimelineError> {
|
||||
self.timelines
|
||||
.get(ttid)
|
||||
.cloned()
|
||||
.ok_or(TimelineError::NotFound(*ttid))
|
||||
match self.timelines.get(ttid).cloned() {
|
||||
Some(GlobalMapTimeline::Timeline(tli)) => Ok(tli),
|
||||
Some(GlobalMapTimeline::CreationInProgress) => {
|
||||
Err(TimelineError::CreationInProgress(*ttid))
|
||||
}
|
||||
None => Err(TimelineError::NotFound(*ttid)),
|
||||
}
|
||||
}
|
||||
|
||||
fn delete(&mut self, ttid: TenantTimelineId) {
|
||||
@@ -85,7 +88,6 @@ static TIMELINES_STATE: Lazy<Mutex<GlobalTimelinesState>> = Lazy::new(|| {
|
||||
tombstones: HashMap::new(),
|
||||
conf: None,
|
||||
broker_active_set: Arc::new(TimelinesSet::default()),
|
||||
load_lock: Arc::new(tokio::sync::Mutex::new(TimelineLoadLock)),
|
||||
global_rate_limiter: RateLimiter::new(1, 1),
|
||||
})
|
||||
});
|
||||
@@ -141,11 +143,10 @@ impl GlobalTimelines {
|
||||
/// Loads all timelines for the given tenant to memory. Returns fs::read_dir
|
||||
/// errors if any.
|
||||
///
|
||||
/// It is async for update_status_notify sake. Since TIMELINES_STATE lock is
|
||||
/// sync and there is no important reason to make it async (it is always
|
||||
/// held for a short while) we just lock and unlock it for each timeline --
|
||||
/// this function is called during init when nothing else is running, so
|
||||
/// this is fine.
|
||||
/// It is async, but TIMELINES_STATE lock is sync and there is no important
|
||||
/// reason to make it async (it is always held for a short while), so we
|
||||
/// just lock and unlock it for each timeline -- this function is called
|
||||
/// during init when nothing else is running, so this is fine.
|
||||
async fn load_tenant_timelines(tenant_id: TenantId) -> Result<()> {
|
||||
let (conf, broker_active_set, partial_backup_rate_limiter) = {
|
||||
let state = TIMELINES_STATE.lock().unwrap();
|
||||
@@ -163,14 +164,13 @@ impl GlobalTimelines {
|
||||
{
|
||||
let ttid = TenantTimelineId::new(tenant_id, timeline_id);
|
||||
match Timeline::load_timeline(&conf, ttid) {
|
||||
Ok(timeline) => {
|
||||
let tli = Arc::new(timeline);
|
||||
Ok(tli) => {
|
||||
let mut shared_state = tli.write_shared_state().await;
|
||||
TIMELINES_STATE
|
||||
.lock()
|
||||
.unwrap()
|
||||
.timelines
|
||||
.insert(ttid, tli.clone());
|
||||
.insert(ttid, GlobalMapTimeline::Timeline(tli.clone()));
|
||||
tli.bootstrap(
|
||||
&mut shared_state,
|
||||
&conf,
|
||||
@@ -199,51 +199,6 @@ impl GlobalTimelines {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Take a lock for timeline loading.
|
||||
pub async fn loading_lock() -> Arc<tokio::sync::Mutex<TimelineLoadLock>> {
|
||||
TIMELINES_STATE.lock().unwrap().load_lock.clone()
|
||||
}
|
||||
|
||||
/// Load timeline from disk to the memory.
|
||||
pub async fn load_timeline<'a>(
|
||||
_guard: &tokio::sync::MutexGuard<'a, TimelineLoadLock>,
|
||||
ttid: TenantTimelineId,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
let (conf, broker_active_set, partial_backup_rate_limiter) =
|
||||
TIMELINES_STATE.lock().unwrap().get_dependencies();
|
||||
|
||||
match Timeline::load_timeline(&conf, ttid) {
|
||||
Ok(timeline) => {
|
||||
let tli = Arc::new(timeline);
|
||||
let mut shared_state = tli.write_shared_state().await;
|
||||
|
||||
// TODO: prevent concurrent timeline creation/loading
|
||||
{
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
|
||||
// We may be have been asked to load a timeline that was previously deleted (e.g. from `pull_timeline.rs`). We trust
|
||||
// that the human doing this manual intervention knows what they are doing, and remove its tombstone.
|
||||
if state.tombstones.remove(&ttid).is_some() {
|
||||
warn!("Un-deleted timeline {ttid}");
|
||||
}
|
||||
|
||||
state.timelines.insert(ttid, tli.clone());
|
||||
}
|
||||
|
||||
tli.bootstrap(
|
||||
&mut shared_state,
|
||||
&conf,
|
||||
broker_active_set,
|
||||
partial_backup_rate_limiter,
|
||||
);
|
||||
drop(shared_state);
|
||||
Ok(tli)
|
||||
}
|
||||
// If we can't load a timeline, it's bad. Caller will figure it out.
|
||||
Err(e) => bail!("failed to load timeline {}, reason: {:?}", ttid, e),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the number of timelines in the map.
|
||||
pub fn timelines_count() -> usize {
|
||||
TIMELINES_STATE.lock().unwrap().timelines.len()
|
||||
@@ -266,7 +221,7 @@ impl GlobalTimelines {
|
||||
commit_lsn: Lsn,
|
||||
local_start_lsn: Lsn,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
let (conf, broker_active_set, partial_backup_rate_limiter) = {
|
||||
let (conf, _, _) = {
|
||||
let state = TIMELINES_STATE.lock().unwrap();
|
||||
if let Ok(timeline) = state.get(&ttid) {
|
||||
// Timeline already exists, return it.
|
||||
@@ -282,55 +237,146 @@ impl GlobalTimelines {
|
||||
|
||||
info!("creating new timeline {}", ttid);
|
||||
|
||||
let timeline = Arc::new(Timeline::create_empty(
|
||||
&conf,
|
||||
ttid,
|
||||
server_info,
|
||||
commit_lsn,
|
||||
local_start_lsn,
|
||||
)?);
|
||||
// Do on disk initialization in tmp dir.
|
||||
let (_tmp_dir, tmp_dir_path) = create_temp_timeline_dir(&conf, ttid).await?;
|
||||
|
||||
// Take a lock and finish the initialization holding this mutex. No other threads
|
||||
// can interfere with creation after we will insert timeline into the map.
|
||||
{
|
||||
let mut shared_state = timeline.write_shared_state().await;
|
||||
// TODO: currently we create only cfile. It would be reasonable to
|
||||
// immediately initialize first WAL segment as well.
|
||||
let state =
|
||||
TimelinePersistentState::new(&ttid, server_info, vec![], commit_lsn, local_start_lsn)?;
|
||||
control_file::FileStorage::create_new(tmp_dir_path.clone(), &conf, state).await?;
|
||||
let timeline = GlobalTimelines::load_temp_timeline(ttid, &tmp_dir_path, true).await?;
|
||||
Ok(timeline)
|
||||
}
|
||||
|
||||
// We can get a race condition here in case of concurrent create calls, but only
|
||||
// in theory. create() will return valid timeline on the next try.
|
||||
TIMELINES_STATE
|
||||
.lock()
|
||||
.unwrap()
|
||||
.try_insert(timeline.clone())?;
|
||||
/// Move timeline from a temp directory to the main storage, and load it to
|
||||
/// the global map. Creating timeline in this way ensures atomicity: rename
|
||||
/// is atomic, so either move of the whole datadir succeeds or it doesn't,
|
||||
/// but corrupted data dir shouldn't be possible.
|
||||
///
|
||||
/// We'd like to avoid holding map lock while doing IO, so it's a 3 step
|
||||
/// process:
|
||||
/// 1) check the global map that timeline doesn't exist and mark that we're
|
||||
/// creating it;
|
||||
/// 2) move the directory and load the timeline
|
||||
/// 3) take lock again and insert the timeline into the global map.
|
||||
pub async fn load_temp_timeline(
|
||||
ttid: TenantTimelineId,
|
||||
tmp_path: &Utf8PathBuf,
|
||||
check_tombstone: bool,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
// Check for existence and mark that we're creating it.
|
||||
let (conf, broker_active_set, partial_backup_rate_limiter) = {
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
match state.timelines.get(&ttid) {
|
||||
Some(GlobalMapTimeline::CreationInProgress) => {
|
||||
bail!(TimelineError::CreationInProgress(ttid));
|
||||
}
|
||||
Some(GlobalMapTimeline::Timeline(_)) => {
|
||||
bail!(TimelineError::AlreadyExists(ttid));
|
||||
}
|
||||
_ => {}
|
||||
}
|
||||
if check_tombstone {
|
||||
if state.tombstones.contains_key(&ttid) {
|
||||
anyhow::bail!("timeline {ttid} is deleted, refusing to recreate");
|
||||
}
|
||||
} else {
|
||||
// We may be have been asked to load a timeline that was previously deleted (e.g. from `pull_timeline.rs`). We trust
|
||||
// that the human doing this manual intervention knows what they are doing, and remove its tombstone.
|
||||
if state.tombstones.remove(&ttid).is_some() {
|
||||
warn!("un-deleted timeline {ttid}");
|
||||
}
|
||||
}
|
||||
state
|
||||
.timelines
|
||||
.insert(ttid, GlobalMapTimeline::CreationInProgress);
|
||||
state.get_dependencies()
|
||||
};
|
||||
|
||||
// Write the new timeline to the disk and start background workers.
|
||||
// Bootstrap is transactional, so if it fails, the timeline will be deleted,
|
||||
// and the state on disk should remain unchanged.
|
||||
if let Err(e) = timeline
|
||||
.init_new(
|
||||
&mut shared_state,
|
||||
// Do the actual move and reflect the result in the map.
|
||||
match GlobalTimelines::install_temp_timeline(ttid, tmp_path, &conf).await {
|
||||
Ok(timeline) => {
|
||||
let mut timeline_shared_state = timeline.write_shared_state().await;
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
assert!(matches!(
|
||||
state.timelines.get(&ttid),
|
||||
Some(GlobalMapTimeline::CreationInProgress)
|
||||
));
|
||||
|
||||
state
|
||||
.timelines
|
||||
.insert(ttid, GlobalMapTimeline::Timeline(timeline.clone()));
|
||||
drop(state);
|
||||
timeline.bootstrap(
|
||||
&mut timeline_shared_state,
|
||||
&conf,
|
||||
broker_active_set,
|
||||
partial_backup_rate_limiter,
|
||||
)
|
||||
.await
|
||||
{
|
||||
// Note: the most likely reason for init failure is that the timeline
|
||||
// directory already exists on disk. This happens when timeline is corrupted
|
||||
// and wasn't loaded from disk on startup because of that. We want to preserve
|
||||
// the timeline directory in this case, for further inspection.
|
||||
|
||||
// TODO: this is an unusual error, perhaps we should send it to sentry
|
||||
// TODO: compute will try to create timeline every second, we should add backoff
|
||||
error!("failed to init new timeline {}: {}", ttid, e);
|
||||
|
||||
// Timeline failed to init, it cannot be used. Remove it from the map.
|
||||
TIMELINES_STATE.lock().unwrap().timelines.remove(&ttid);
|
||||
return Err(e);
|
||||
);
|
||||
drop(timeline_shared_state);
|
||||
Ok(timeline)
|
||||
}
|
||||
Err(e) => {
|
||||
// Init failed, remove the marker from the map
|
||||
let mut state = TIMELINES_STATE.lock().unwrap();
|
||||
assert!(matches!(
|
||||
state.timelines.get(&ttid),
|
||||
Some(GlobalMapTimeline::CreationInProgress)
|
||||
));
|
||||
state.timelines.remove(&ttid);
|
||||
Err(e)
|
||||
}
|
||||
// We are done with bootstrap, release the lock, return the timeline.
|
||||
// {} block forces release before .await
|
||||
}
|
||||
Ok(timeline)
|
||||
}
|
||||
|
||||
/// Main part of load_temp_timeline: do the move and load.
|
||||
async fn install_temp_timeline(
|
||||
ttid: TenantTimelineId,
|
||||
tmp_path: &Utf8PathBuf,
|
||||
conf: &SafeKeeperConf,
|
||||
) -> Result<Arc<Timeline>> {
|
||||
let tenant_path = get_tenant_dir(conf, &ttid.tenant_id);
|
||||
let timeline_path = get_timeline_dir(conf, &ttid);
|
||||
|
||||
// We must have already checked that timeline doesn't exist in the map,
|
||||
// but there might be existing datadir: if timeline is corrupted it is
|
||||
// not loaded. We don't want to overwrite such a dir, so check for its
|
||||
// existence.
|
||||
match fs::metadata(&timeline_path).await {
|
||||
Ok(_) => {
|
||||
// Timeline directory exists on disk, we should leave state unchanged
|
||||
// and return error.
|
||||
bail!(TimelineError::Invalid(ttid));
|
||||
}
|
||||
Err(e) if e.kind() == std::io::ErrorKind::NotFound => {}
|
||||
Err(e) => {
|
||||
return Err(e.into());
|
||||
}
|
||||
}
|
||||
|
||||
info!(
|
||||
"moving timeline {} from {} to {}",
|
||||
ttid, tmp_path, timeline_path
|
||||
);
|
||||
|
||||
// Now it is safe to move the timeline directory to the correct
|
||||
// location. First, create tenant directory. Ignore error if it already
|
||||
// exists.
|
||||
if let Err(e) = tokio::fs::create_dir(&tenant_path).await {
|
||||
if e.kind() != std::io::ErrorKind::AlreadyExists {
|
||||
return Err(e.into());
|
||||
}
|
||||
}
|
||||
// fsync it
|
||||
fsync_async_opt(&tenant_path, !conf.no_sync).await?;
|
||||
// and its creation
|
||||
fsync_async_opt(&conf.workdir, !conf.no_sync).await?;
|
||||
|
||||
// Do the move.
|
||||
durable_rename(tmp_path, &timeline_path, !conf.no_sync).await?;
|
||||
|
||||
Timeline::load_timeline(conf, ttid)
|
||||
}
|
||||
|
||||
/// Get a timeline from the global map. If it's not present, it doesn't exist on disk,
|
||||
@@ -358,8 +404,16 @@ impl GlobalTimelines {
|
||||
global_lock
|
||||
.timelines
|
||||
.values()
|
||||
.filter(|t| !t.is_cancelled())
|
||||
.cloned()
|
||||
.filter_map(|t| match t {
|
||||
GlobalMapTimeline::Timeline(t) => {
|
||||
if t.is_cancelled() {
|
||||
None
|
||||
} else {
|
||||
Some(t.clone())
|
||||
}
|
||||
}
|
||||
_ => None,
|
||||
})
|
||||
.collect()
|
||||
}
|
||||
|
||||
@@ -370,8 +424,11 @@ impl GlobalTimelines {
|
||||
global_lock
|
||||
.timelines
|
||||
.values()
|
||||
.filter_map(|t| match t {
|
||||
GlobalMapTimeline::Timeline(t) => Some(t.clone()),
|
||||
_ => None,
|
||||
})
|
||||
.filter(|t| t.ttid.tenant_id == tenant_id)
|
||||
.cloned()
|
||||
.collect()
|
||||
}
|
||||
|
||||
@@ -504,3 +561,45 @@ fn delete_dir(path: Utf8PathBuf) -> Result<bool> {
|
||||
Err(e) => Err(e.into()),
|
||||
}
|
||||
}
|
||||
|
||||
/// Create temp directory for a new timeline. It needs to be located on the same
|
||||
/// filesystem as the rest of the timelines. It will be automatically deleted when
|
||||
/// Utf8TempDir goes out of scope.
|
||||
pub async fn create_temp_timeline_dir(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: TenantTimelineId,
|
||||
) -> Result<(Utf8TempDir, Utf8PathBuf)> {
|
||||
let temp_base = conf.workdir.join("tmp");
|
||||
|
||||
tokio::fs::create_dir_all(&temp_base).await?;
|
||||
|
||||
let tli_dir = camino_tempfile::Builder::new()
|
||||
.suffix("_temptli")
|
||||
.prefix(&format!("{}_{}_", ttid.tenant_id, ttid.timeline_id))
|
||||
.tempdir_in(temp_base)?;
|
||||
|
||||
let tli_dir_path = tli_dir.path().to_path_buf();
|
||||
|
||||
Ok((tli_dir, tli_dir_path))
|
||||
}
|
||||
|
||||
/// Do basic validation of a temp timeline, before moving it to the global map.
|
||||
pub async fn validate_temp_timeline(
|
||||
conf: &SafeKeeperConf,
|
||||
ttid: TenantTimelineId,
|
||||
path: &Utf8PathBuf,
|
||||
) -> Result<(Lsn, Lsn)> {
|
||||
let control_path = path.join("safekeeper.control");
|
||||
|
||||
let control_store = control_file::FileStorage::load_control_file(control_path)?;
|
||||
if control_store.server.wal_seg_size == 0 {
|
||||
bail!("wal_seg_size is not set");
|
||||
}
|
||||
|
||||
let wal_store = wal_storage::PhysicalStorage::new(&ttid, path.clone(), conf, &control_store)?;
|
||||
|
||||
let commit_lsn = control_store.commit_lsn;
|
||||
let flush_lsn = wal_store.flush_lsn();
|
||||
|
||||
Ok((commit_lsn, flush_lsn))
|
||||
}
|
||||
|
||||
@@ -186,8 +186,14 @@ impl PhysicalStorage {
|
||||
"initialized storage for timeline {}, flush_lsn={}, commit_lsn={}, peer_horizon_lsn={}",
|
||||
ttid.timeline_id, flush_lsn, state.commit_lsn, state.peer_horizon_lsn,
|
||||
);
|
||||
if flush_lsn < state.commit_lsn || flush_lsn < state.peer_horizon_lsn {
|
||||
warn!("timeline {} potential data loss: flush_lsn by find_end_of_wal is less than either commit_lsn or peer_horizon_lsn from control file", ttid.timeline_id);
|
||||
if flush_lsn < state.commit_lsn {
|
||||
bail!("timeline {} potential data loss: flush_lsn {} by find_end_of_wal is less than commit_lsn {} from control file", ttid.timeline_id, flush_lsn, state.commit_lsn);
|
||||
}
|
||||
if flush_lsn < state.peer_horizon_lsn {
|
||||
warn!(
|
||||
"timeline {}: flush_lsn {} is less than cfile peer_horizon_lsn {}",
|
||||
ttid.timeline_id, flush_lsn, state.peer_horizon_lsn
|
||||
);
|
||||
}
|
||||
|
||||
Ok(PhysicalStorage {
|
||||
|
||||
@@ -59,7 +59,7 @@ impl GlobalMap {
|
||||
|
||||
if state.commit_lsn < state.local_start_lsn {
|
||||
bail!(
|
||||
"commit_lsn {} is higher than local_start_lsn {}",
|
||||
"commit_lsn {} is smaller than local_start_lsn {}",
|
||||
state.commit_lsn,
|
||||
state.local_start_lsn
|
||||
);
|
||||
@@ -96,23 +96,7 @@ impl GlobalMap {
|
||||
let local_start_lsn = Lsn::INVALID;
|
||||
|
||||
let state =
|
||||
TimelinePersistentState::new(&ttid, server_info, vec![], commit_lsn, local_start_lsn);
|
||||
|
||||
if state.server.wal_seg_size == 0 {
|
||||
bail!(TimelineError::UninitializedWalSegSize(ttid));
|
||||
}
|
||||
|
||||
if state.server.pg_version == UNKNOWN_SERVER_VERSION {
|
||||
bail!(TimelineError::UninitialinzedPgVersion(ttid));
|
||||
}
|
||||
|
||||
if state.commit_lsn < state.local_start_lsn {
|
||||
bail!(
|
||||
"commit_lsn {} is higher than local_start_lsn {}",
|
||||
state.commit_lsn,
|
||||
state.local_start_lsn
|
||||
);
|
||||
}
|
||||
TimelinePersistentState::new(&ttid, server_info, vec![], commit_lsn, local_start_lsn)?;
|
||||
|
||||
let disk_timeline = self.disk.put_state(&ttid, state);
|
||||
let control_store = DiskStateStorage::new(disk_timeline.clone());
|
||||
|
||||
Reference in New Issue
Block a user