From 7e817789d5deaf25d63e4d36eea0e510c1d2d45d Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 2 Sep 2023 10:06:14 +0200 Subject: [PATCH] VirtualFile: add crash-safe overwrite abstraction & use it (#5186) (part of #4743) (preliminary to #5180) This PR adds a special-purpose API to `VirtualFile` for write-once files. It adopts it for `save_metadata` and `persist_tenant_conf`. This is helpful for the asyncification efforts (#4743) and specifically asyncification of `VirtualFile` because above two functions were the only ones that needed the VirtualFile to be an `std::io::Write`. (There was also `manifest.rs` that needed the `std::io::Write`, but, it isn't used right now, and likely won't be used because we're taking a different route for crash consistency, see #5172. So, let's remove it. It'll be in Git history if we need to re-introduce it when picking up the compaction work again; that's why it was introduced in the first place). We can't remove the `impl std::io::Write for VirtualFile` just yet because of the `BufWriter` in ```rust struct DeltaLayerWriterInner { ... blob_writer: WriteBlobWriter>, } ``` But, @arpad-m and I have a plan to get rid of that by extracting the append-only-ness-on-top-of-VirtualFile that #4994 added to `EphemeralFile` into an abstraction that can be re-used in the `DeltaLayerWriterInner` and `ImageLayerWriterInner`. That'll be another PR. ### Performance Impact This PR adds more fsyncs compared to before because we fsync the parent directory every time. 1. For `save_metadata`, the additional fsyncs are unnecessary because we know that `metadata` fits into a kernel page, and hence the write won't be torn on the way into the kernel. However, the `metadata` file in general is going to lose signficance very soon (=> see #5172), and the NVMes in prod can definitely handle the additional fsync. So, let's not worry about it. 2. For `persist_tenant_conf`, which we don't check to fit into a single kernel page, this PR makes it actually crash-consistent. Before, we could crash while writing out the tenant conf, leaving a prefix of the tenant conf on disk. --- pageserver/src/tenant.rs | 97 ++------- pageserver/src/tenant/manifest.rs | 325 ------------------------------ pageserver/src/tenant/metadata.rs | 35 +--- pageserver/src/tenant/mgr.rs | 2 +- pageserver/src/tenant/timeline.rs | 10 +- pageserver/src/virtual_file.rs | 79 ++++++++ 6 files changed, 112 insertions(+), 436 deletions(-) delete mode 100644 pageserver/src/tenant/manifest.rs diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index ce8520853773..855301cd1d1b 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -32,9 +32,7 @@ use std::fmt::Debug; use std::fmt::Display; use std::fs; use std::fs::File; -use std::fs::OpenOptions; use std::io; -use std::io::Write; use std::ops::Bound::Included; use std::path::Path; use std::path::PathBuf; @@ -115,7 +113,6 @@ pub mod block_io; pub mod disk_btree; pub(crate) mod ephemeral_file; pub mod layer_map; -pub mod manifest; mod span; pub mod metadata; @@ -407,7 +404,6 @@ impl Tenant { remote_startup_data: Option, local_metadata: Option, ancestor: Option>, - first_save: bool, init_order: Option<&InitializationOrder>, _ctx: &RequestContext, ) -> anyhow::Result<()> { @@ -441,14 +437,8 @@ impl Tenant { // Save the metadata file to local disk. if !picked_local { - save_metadata( - self.conf, - &tenant_id, - &timeline_id, - up_to_date_metadata, - first_save, - ) - .context("save_metadata")?; + save_metadata(self.conf, &tenant_id, &timeline_id, up_to_date_metadata) + .context("save_metadata")?; } let index_part = remote_startup_data.as_ref().map(|x| &x.index_part); @@ -833,7 +823,6 @@ impl Tenant { }), local_metadata, ancestor, - true, None, ctx, ) @@ -1386,7 +1375,6 @@ impl Tenant { remote_startup_data, Some(local_metadata), ancestor, - false, init_order, ctx, ) @@ -2378,68 +2366,33 @@ impl Tenant { tenant_id: &TenantId, target_config_path: &Path, tenant_conf: TenantConfOpt, - creating_tenant: bool, ) -> anyhow::Result<()> { let _enter = info_span!("saving tenantconf").entered(); // imitate a try-block with a closure - let do_persist = |target_config_path: &Path| -> anyhow::Result<()> { - let target_config_parent = target_config_path.parent().with_context(|| { - format!( - "Config path does not have a parent: {}", - target_config_path.display() - ) - })?; - - info!("persisting tenantconf to {}", target_config_path.display()); + info!("persisting tenantconf to {}", target_config_path.display()); - let mut conf_content = r#"# This file contains a specific per-tenant's config. + let mut conf_content = r#"# This file contains a specific per-tenant's config. # It is read in case of pageserver restart. [tenant_config] "# - .to_string(); - - // Convert the config to a toml file. - conf_content += &toml_edit::ser::to_string(&tenant_conf)?; - - let mut target_config_file = VirtualFile::open_with_options( - target_config_path, - OpenOptions::new() - .truncate(true) // This needed for overwriting with small config files - .write(true) - .create_new(creating_tenant) - // when creating a new tenant, first_save will be true and `.create(true)` will be - // ignored (per rust std docs). - // - // later when updating the config of created tenant, or persisting config for the - // first time for attached tenant, the `.create(true)` is used. - .create(true), - )?; - - target_config_file - .write(conf_content.as_bytes()) - .context("write toml bytes into file") - .and_then(|_| target_config_file.sync_all().context("fsync config file")) - .context("write config file")?; - - // fsync the parent directory to ensure the directory entry is durable. - // before this was done conditionally on creating_tenant, but these management actions are rare - // enough to just fsync it always. - - crashsafe::fsync(target_config_parent)?; - // XXX we're not fsyncing the parent dir, need to do that in case `creating_tenant` - Ok(()) - }; + .to_string(); - // this function is called from creating the tenant and updating the tenant config, which - // would otherwise share this context, so keep it here in one place. - do_persist(target_config_path).with_context(|| { - format!( - "write tenant {tenant_id} config to {}", - target_config_path.display() - ) - }) + // Convert the config to a toml file. + conf_content += &toml_edit::ser::to_string(&tenant_conf)?; + + let conf_content = conf_content.as_bytes(); + + let temp_path = path_with_suffix_extension(target_config_path, TEMP_FILE_SUFFIX); + VirtualFile::crashsafe_overwrite(target_config_path, &temp_path, conf_content) + .with_context(|| { + format!( + "write tenant {tenant_id} config to {}", + target_config_path.display() + ) + })?; + Ok(()) } // @@ -2968,14 +2921,8 @@ impl Tenant { anyhow::bail!("failpoint after-timeline-uninit-mark-creation"); }); - save_metadata( - self.conf, - &self.tenant_id, - new_timeline_id, - new_metadata, - true, - ) - .context("Failed to create timeline metadata")?; + save_metadata(self.conf, &self.tenant_id, new_timeline_id, new_metadata) + .context("Failed to create timeline metadata")?; Ok(()) } @@ -3215,7 +3162,7 @@ fn try_create_target_tenant_dir( ) .with_context(|| format!("resolve tenant {tenant_id} temporary config path"))?; - Tenant::persist_tenant_config(tenant_id, &temporary_tenant_config_path, tenant_conf, true)?; + Tenant::persist_tenant_config(tenant_id, &temporary_tenant_config_path, tenant_conf)?; crashsafe::create_dir(&temporary_tenant_timelines_dir).with_context(|| { format!( diff --git a/pageserver/src/tenant/manifest.rs b/pageserver/src/tenant/manifest.rs deleted file mode 100644 index 1d2835114f48..000000000000 --- a/pageserver/src/tenant/manifest.rs +++ /dev/null @@ -1,325 +0,0 @@ -//! This module contains the encoding and decoding of the local manifest file. -//! -//! MANIFEST is a write-ahead log which is stored locally to each timeline. It -//! records the state of the storage engine. It contains a snapshot of the -//! state and all operations proceeding that snapshot. The file begins with a -//! header recording MANIFEST version number. After that, it contains a snapshot. -//! The snapshot is followed by a list of operations. Each operation is a list -//! of records. Each record is either an addition or a removal of a layer. -//! -//! With MANIFEST, we can: -//! -//! 1. recover state quickly by reading the file, potentially boosting the -//! startup speed. -//! 2. ensure all operations are atomic and avoid corruption, solving issues -//! like redundant image layer and preparing us for future compaction -//! strategies. -//! -//! There is also a format for storing all layer files on S3, called -//! `index_part.json`. Compared with index_part, MANIFEST is an WAL which -//! records all operations as logs, and therefore we can easily replay the -//! operations when recovering from crash, while ensuring those operations -//! are atomic upon restart. -//! -//! Currently, this is not used in the system. Future refactors will ensure -//! the storage state will be recorded in this file, and the system can be -//! recovered from this file. This is tracked in -//! - -use std::io::{self, Read, Write}; - -use crate::virtual_file::VirtualFile; -use anyhow::Result; -use bytes::{Buf, BufMut, Bytes, BytesMut}; -use crc32c::crc32c; -use serde::{Deserialize, Serialize}; -use tracing::log::warn; -use utils::lsn::Lsn; - -use super::storage_layer::PersistentLayerDesc; - -pub struct Manifest { - file: VirtualFile, -} - -#[derive(Clone, Serialize, Deserialize, PartialEq, Eq, Debug)] -pub struct Snapshot { - pub layers: Vec, -} - -/// serde by default encode this in tagged enum, and therefore it will be something -/// like `{ "AddLayer": { ... } }`. -#[derive(Clone, Serialize, Deserialize, PartialEq, Eq, Debug)] -pub enum Record { - AddLayer(PersistentLayerDesc), - RemoveLayer(PersistentLayerDesc), -} - -/// `echo neon.manifest | sha1sum` and take the leading 8 bytes. -const MANIFEST_MAGIC_NUMBER: u64 = 0xf5c44592b806109c; -const MANIFEST_VERSION: u64 = 1; - -#[derive(Clone, Serialize, Deserialize, PartialEq, Eq, Debug)] -pub struct ManifestHeader { - magic_number: u64, - version: u64, -} - -const MANIFEST_HEADER_LEN: usize = 16; - -impl ManifestHeader { - fn encode(&self) -> BytesMut { - let mut buf = BytesMut::with_capacity(MANIFEST_HEADER_LEN); - buf.put_u64(self.magic_number); - buf.put_u64(self.version); - buf - } - - fn decode(mut buf: &[u8]) -> Self { - assert!(buf.len() == MANIFEST_HEADER_LEN, "invalid header"); - Self { - magic_number: buf.get_u64(), - version: buf.get_u64(), - } - } -} - -#[derive(Clone, Serialize, Deserialize, PartialEq, Eq, Debug)] -pub enum Operation { - /// A snapshot of the current state. - /// - /// Lsn field represents the LSN that is persisted to disk for this snapshot. - Snapshot(Snapshot, Lsn), - /// An atomic operation that changes the state. - /// - /// Lsn field represents the LSN that is persisted to disk after the operation is done. - /// This will only change when new L0 is flushed to the disk. - Operation(Vec, Lsn), -} - -struct RecordHeader { - size: u32, - checksum: u32, -} - -const RECORD_HEADER_LEN: usize = 8; - -impl RecordHeader { - fn encode(&self) -> BytesMut { - let mut buf = BytesMut::with_capacity(RECORD_HEADER_LEN); - buf.put_u32(self.size); - buf.put_u32(self.checksum); - buf - } - - fn decode(mut buf: &[u8]) -> Self { - assert!(buf.len() == RECORD_HEADER_LEN, "invalid header"); - Self { - size: buf.get_u32(), - checksum: buf.get_u32(), - } - } -} - -#[derive(Debug, thiserror::Error)] -pub enum ManifestLoadError { - #[error("manifest header is corrupted")] - CorruptedManifestHeader, - #[error("unsupported manifest version: got {0}, expected {1}")] - UnsupportedVersion(u64, u64), - #[error("error when decoding record: {0}")] - DecodeRecord(serde_json::Error), - #[error("I/O error: {0}")] - Io(io::Error), -} - -#[must_use = "Should check if the manifest is partially corrupted"] -pub struct ManifestPartiallyCorrupted(bool); - -impl Manifest { - /// Create a new manifest by writing the manifest header and a snapshot record to the given file. - pub fn init(file: VirtualFile, snapshot: Snapshot, lsn: Lsn) -> Result { - let mut manifest = Self { file }; - manifest.append_manifest_header(ManifestHeader { - magic_number: MANIFEST_MAGIC_NUMBER, - version: MANIFEST_VERSION, - })?; - manifest.append_operation(Operation::Snapshot(snapshot, lsn))?; - Ok(manifest) - } - - /// Load a manifest. Returns the manifest and a list of operations. If the manifest is corrupted, - /// the bool flag will be set to true and the user is responsible to reconstruct a new manifest and - /// backup the current one. - pub fn load( - mut file: VirtualFile, - ) -> Result<(Self, Vec, ManifestPartiallyCorrupted), ManifestLoadError> { - let mut buf = vec![]; - file.read_to_end(&mut buf).map_err(ManifestLoadError::Io)?; - - // Read manifest header - let mut buf = Bytes::from(buf); - if buf.remaining() < MANIFEST_HEADER_LEN { - return Err(ManifestLoadError::CorruptedManifestHeader); - } - let header = ManifestHeader::decode(&buf[..MANIFEST_HEADER_LEN]); - buf.advance(MANIFEST_HEADER_LEN); - if header.version != MANIFEST_VERSION { - return Err(ManifestLoadError::UnsupportedVersion( - header.version, - MANIFEST_VERSION, - )); - } - - // Read operations - let mut operations = Vec::new(); - let corrupted = loop { - if buf.remaining() == 0 { - break false; - } - if buf.remaining() < RECORD_HEADER_LEN { - warn!("incomplete header when decoding manifest, could be corrupted"); - break true; - } - let RecordHeader { size, checksum } = RecordHeader::decode(&buf[..RECORD_HEADER_LEN]); - let size = size as usize; - buf.advance(RECORD_HEADER_LEN); - if buf.remaining() < size { - warn!("incomplete data when decoding manifest, could be corrupted"); - break true; - } - let data = &buf[..size]; - if crc32c(data) != checksum { - warn!("checksum mismatch when decoding manifest, could be corrupted"); - break true; - } - // if the following decode fails, we cannot use the manifest or safely ignore any record. - operations.push(serde_json::from_slice(data).map_err(ManifestLoadError::DecodeRecord)?); - buf.advance(size); - }; - Ok(( - Self { file }, - operations, - ManifestPartiallyCorrupted(corrupted), - )) - } - - fn append_data(&mut self, data: &[u8]) -> Result<()> { - if data.len() >= u32::MAX as usize { - panic!("data too large"); - } - let header = RecordHeader { - size: data.len() as u32, - checksum: crc32c(data), - }; - let header = header.encode(); - self.file.write_all(&header)?; - self.file.write_all(data)?; - self.file.sync_all()?; - Ok(()) - } - - fn append_manifest_header(&mut self, header: ManifestHeader) -> Result<()> { - let encoded = header.encode(); - self.file.write_all(&encoded)?; - Ok(()) - } - - /// Add an operation to the manifest. The operation will be appended to the end of the file, - /// and the file will fsync. - pub fn append_operation(&mut self, operation: Operation) -> Result<()> { - let encoded = Vec::from(serde_json::to_string(&operation)?); - self.append_data(&encoded) - } -} - -#[cfg(test)] -mod tests { - use std::fs::OpenOptions; - - use crate::repository::Key; - - use super::*; - - #[test] - fn test_read_manifest() { - let testdir = crate::config::PageServerConf::test_repo_dir("test_read_manifest"); - std::fs::create_dir_all(&testdir).unwrap(); - let file = VirtualFile::create(&testdir.join("MANIFEST")).unwrap(); - let layer1 = PersistentLayerDesc::new_test(Key::from_i128(0)..Key::from_i128(233)); - let layer2 = PersistentLayerDesc::new_test(Key::from_i128(233)..Key::from_i128(2333)); - let layer3 = PersistentLayerDesc::new_test(Key::from_i128(2333)..Key::from_i128(23333)); - let layer4 = PersistentLayerDesc::new_test(Key::from_i128(23333)..Key::from_i128(233333)); - - // Write a manifest with a snapshot and some operations - let snapshot = Snapshot { - layers: vec![layer1, layer2], - }; - let mut manifest = Manifest::init(file, snapshot.clone(), Lsn::from(0)).unwrap(); - manifest - .append_operation(Operation::Operation( - vec![Record::AddLayer(layer3.clone())], - Lsn::from(1), - )) - .unwrap(); - drop(manifest); - - // Open the second time and write - let file = VirtualFile::open_with_options( - &testdir.join("MANIFEST"), - OpenOptions::new() - .read(true) - .write(true) - .create_new(false) - .truncate(false), - ) - .unwrap(); - let (mut manifest, operations, corrupted) = Manifest::load(file).unwrap(); - assert!(!corrupted.0); - assert_eq!(operations.len(), 2); - assert_eq!( - &operations[0], - &Operation::Snapshot(snapshot.clone(), Lsn::from(0)) - ); - assert_eq!( - &operations[1], - &Operation::Operation(vec![Record::AddLayer(layer3.clone())], Lsn::from(1)) - ); - manifest - .append_operation(Operation::Operation( - vec![ - Record::RemoveLayer(layer3.clone()), - Record::AddLayer(layer4.clone()), - ], - Lsn::from(2), - )) - .unwrap(); - drop(manifest); - - // Open the third time and verify - let file = VirtualFile::open_with_options( - &testdir.join("MANIFEST"), - OpenOptions::new() - .read(true) - .write(true) - .create_new(false) - .truncate(false), - ) - .unwrap(); - let (_manifest, operations, corrupted) = Manifest::load(file).unwrap(); - assert!(!corrupted.0); - assert_eq!(operations.len(), 3); - assert_eq!(&operations[0], &Operation::Snapshot(snapshot, Lsn::from(0))); - assert_eq!( - &operations[1], - &Operation::Operation(vec![Record::AddLayer(layer3.clone())], Lsn::from(1)) - ); - assert_eq!( - &operations[2], - &Operation::Operation( - vec![Record::RemoveLayer(layer3), Record::AddLayer(layer4)], - Lsn::from(2) - ) - ); - } -} diff --git a/pageserver/src/tenant/metadata.rs b/pageserver/src/tenant/metadata.rs index dbf2d5ac37dd..7d895577a2dd 100644 --- a/pageserver/src/tenant/metadata.rs +++ b/pageserver/src/tenant/metadata.rs @@ -8,14 +8,14 @@ //! //! [`remote_timeline_client`]: super::remote_timeline_client -use std::fs::{File, OpenOptions}; -use std::io::{self, Write}; +use std::io::{self}; -use anyhow::{bail, ensure, Context}; +use anyhow::{ensure, Context}; use serde::{de::Error, Deserialize, Serialize, Serializer}; use thiserror::Error; use tracing::info_span; use utils::bin_ser::SerializeError; +use utils::crashsafe::path_with_suffix_extension; use utils::{ bin_ser::BeSer, id::{TenantId, TimelineId}, @@ -24,6 +24,7 @@ use utils::{ use crate::config::PageServerConf; use crate::virtual_file::VirtualFile; +use crate::TEMP_FILE_SUFFIX; /// Use special format number to enable backward compatibility. const METADATA_FORMAT_VERSION: u16 = 4; @@ -260,33 +261,13 @@ pub fn save_metadata( tenant_id: &TenantId, timeline_id: &TimelineId, data: &TimelineMetadata, - first_save: bool, ) -> anyhow::Result<()> { let _enter = info_span!("saving metadata").entered(); let path = conf.metadata_path(tenant_id, timeline_id); - // use OpenOptions to ensure file presence is consistent with first_save - let mut file = VirtualFile::open_with_options( - &path, - OpenOptions::new().write(true).create_new(first_save), - ) - .context("open_with_options")?; - - let metadata_bytes = data.to_bytes().context("Failed to get metadata bytes")?; - - if file.write(&metadata_bytes)? != metadata_bytes.len() { - bail!("Could not write all the metadata bytes in a single call"); - } - file.sync_all()?; - - // fsync the parent directory to ensure the directory entry is durable - if first_save { - let timeline_dir = File::open( - path.parent() - .expect("Metadata should always have a parent dir"), - )?; - timeline_dir.sync_all()?; - } - + let temp_path = path_with_suffix_extension(&path, TEMP_FILE_SUFFIX); + let metadata_bytes = data.to_bytes().context("serialize metadata")?; + VirtualFile::crashsafe_overwrite(&path, &temp_path, &metadata_bytes) + .context("write metadata")?; Ok(()) } diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 87617b544cf4..dd5f34b6b243 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -404,7 +404,7 @@ pub async fn set_new_tenant_config( let tenant = get_tenant(tenant_id, true).await?; let tenant_config_path = conf.tenant_config_path(&tenant_id); - Tenant::persist_tenant_config(&tenant_id, &tenant_config_path, new_tenant_conf, false) + Tenant::persist_tenant_config(&tenant_id, &tenant_config_path, new_tenant_conf) .map_err(SetNewTenantConfigError::Persist)?; tenant.set_new_tenant_config(new_tenant_conf); Ok(()) diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 7e8c0391f4b1..8a1378ecbd2e 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -2827,14 +2827,8 @@ impl Timeline { x.unwrap() )); - save_metadata( - self.conf, - &self.tenant_id, - &self.timeline_id, - &metadata, - false, - ) - .context("save_metadata")?; + save_metadata(self.conf, &self.tenant_id, &self.timeline_id, &metadata) + .context("save_metadata")?; if let Some(remote_client) = &self.remote_client { for (path, layer_metadata) in layer_paths_to_upload { diff --git a/pageserver/src/virtual_file.rs b/pageserver/src/virtual_file.rs index a86b8fa2a6fa..48d43a4e638f 100644 --- a/pageserver/src/virtual_file.rs +++ b/pageserver/src/virtual_file.rs @@ -172,6 +172,41 @@ impl OpenFiles { } } +#[derive(Debug, thiserror::Error)] +pub enum CrashsafeOverwriteError { + #[error("final path has no parent dir")] + FinalPathHasNoParentDir, + #[error("remove tempfile: {0}")] + RemovePreviousTempfile(#[source] std::io::Error), + #[error("create tempfile: {0}")] + CreateTempfile(#[source] std::io::Error), + #[error("write tempfile: {0}")] + WriteContents(#[source] std::io::Error), + #[error("sync tempfile: {0}")] + SyncTempfile(#[source] std::io::Error), + #[error("rename tempfile to final path: {0}")] + RenameTempfileToFinalPath(#[source] std::io::Error), + #[error("open final path parent dir: {0}")] + OpenFinalPathParentDir(#[source] std::io::Error), + #[error("sync final path parent dir: {0}")] + SyncFinalPathParentDir(#[source] std::io::Error), +} +impl CrashsafeOverwriteError { + /// Returns true iff the new contents are durably stored. + pub fn are_new_contents_durable(&self) -> bool { + match self { + Self::FinalPathHasNoParentDir => false, + Self::RemovePreviousTempfile(_) => false, + Self::CreateTempfile(_) => false, + Self::WriteContents(_) => false, + Self::SyncTempfile(_) => false, + Self::RenameTempfileToFinalPath(_) => false, + Self::OpenFinalPathParentDir(_) => false, + Self::SyncFinalPathParentDir(_) => true, + } + } +} + impl VirtualFile { /// Open a file in read-only mode. Like File::open. pub fn open(path: &Path) -> Result { @@ -236,6 +271,50 @@ impl VirtualFile { Ok(vfile) } + pub fn crashsafe_overwrite( + final_path: &Path, + tmp_path: &Path, + content: &[u8], + ) -> Result<(), CrashsafeOverwriteError> { + let Some(final_path_parent) = final_path.parent() else { + return Err(CrashsafeOverwriteError::FinalPathHasNoParentDir); + }; + match std::fs::remove_file(tmp_path) { + Ok(()) => {} + Err(e) if e.kind() == std::io::ErrorKind::NotFound => {} + Err(e) => return Err(CrashsafeOverwriteError::RemovePreviousTempfile(e)), + } + let mut file = Self::open_with_options( + tmp_path, + OpenOptions::new() + .write(true) + // Use `create_new` so that, if we race with ourselves or something else, + // we bail out instead of causing damage. + .create_new(true), + ) + .map_err(CrashsafeOverwriteError::CreateTempfile)?; + file.write_all(content) + .map_err(CrashsafeOverwriteError::WriteContents)?; + file.sync_all() + .map_err(CrashsafeOverwriteError::SyncTempfile)?; + drop(file); // before the rename, that's important! + // renames are atomic + std::fs::rename(tmp_path, final_path) + .map_err(CrashsafeOverwriteError::RenameTempfileToFinalPath)?; + // Only open final path parent dirfd now, so that this operation only + // ever holds one VirtualFile fd at a time. That's important because + // the current `find_victim_slot` impl might pick the same slot for both + // VirtualFile., and it eventually does a blocking write lock instead of + // try_lock. + let final_parent_dirfd = + Self::open_with_options(final_path_parent, OpenOptions::new().read(true)) + .map_err(CrashsafeOverwriteError::OpenFinalPathParentDir)?; + final_parent_dirfd + .sync_all() + .map_err(CrashsafeOverwriteError::SyncFinalPathParentDir)?; + Ok(()) + } + /// Call File::sync_all() on the underlying File. pub fn sync_all(&self) -> Result<(), Error> { self.with_file("fsync", |file| file.sync_all())?