From 2362e5e89c7d55d693d00800127bad4cb64d7f4f Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 3 Oct 2023 15:47:52 -0400 Subject: [PATCH 01/62] Add raw file backend Minor tweaks Add some comments Working on migration Move open/create out of trait, since we always call them explicitly Hide set_dirty_and_block_context outside of tests Make another trait function test-only Add migration (untested) Do unit tests Remove SQLite comments Add unit tests for raw file backend Add test for partial migration Lazy initialization of active slot Put metadata block right after raw file data, for future compatibility Sync file during initial creation, just in case Add test for autosync --- downstairs/src/extent.rs | 314 ++++-- downstairs/src/extent_inner_raw.rs | 1329 +++++++++++++++++++++++++ downstairs/src/extent_inner_sqlite.rs | 89 +- downstairs/src/lib.rs | 13 + downstairs/src/region.rs | 584 ++++++++--- downstairs/src/repair.rs | 82 +- protocol/src/lib.rs | 4 +- 7 files changed, 2097 insertions(+), 318 deletions(-) create mode 100644 downstairs/src/extent_inner_raw.rs diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index a03379a7c..310001009 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -43,7 +43,7 @@ pub(crate) trait ExtentInner: Send + Debug { ) -> Result<(), CrucibleError>; fn read( - &self, + &mut self, job_id: JobId, requests: &[&crucible_protocol::ReadRequest], responses: &mut Vec, @@ -60,14 +60,15 @@ pub(crate) trait ExtentInner: Send + Debug { #[cfg(test)] fn get_block_contexts( - &self, + &mut self, block: u64, count: u64, ) -> Result>, CrucibleError>; /// Sets the dirty flag and updates a block context /// - /// This should only be called from test functions + /// This should only be called from test functions, where we want to + /// manually modify block contexts and test associated behavior #[cfg(test)] fn set_dirty_and_block_context( &mut self, @@ -120,8 +121,18 @@ pub struct ExtentMeta { /// Extent version for SQLite-backed metadata /// /// See [`extent_inner_sqlite::SqliteInner`] for the implementation. +/// +/// This is no longer used when creating new extents, but we support opening +/// existing SQLite-based extents because snapshot images are on read-only +/// volumes, so we can't migrate them. +#[allow(dead_code)] // used in unit test pub const EXTENT_META_SQLITE: u32 = 1; +/// Extent version for raw-file-backed metadata +/// +/// See [`extent_inner_raw::RawInner`] for the implementation. +pub const EXTENT_META_RAW: u32 = 2; + impl ExtentMeta { pub fn new(ext_version: u32) -> ExtentMeta { ExtentMeta { @@ -241,9 +252,16 @@ pub fn completed_dir>(dir: P, number: u32) -> PathBuf { .with_extension("completed") } +pub fn migrate_dir>(dir: P, number: u32) -> PathBuf { + extent_dir(dir, number) + .join(extent_file_name(number, ExtentType::Data)) + .with_extension("migrate") +} + /** - * Remove directories associated with repair except for the replace - * directory. Replace is handled specifically during extent open. + * Remove temporary directories associated with repair and migration, except for + * the final .replace or .migrate directory. Replace and migration are handled + * specifically during extent open. */ pub fn remove_copy_cleanup_dir>(dir: P, eid: u32) -> Result<()> { for f in [copy_dir, completed_dir] { @@ -282,39 +300,92 @@ impl Extent { // If there was an unfinished copy, then clean it up here (??) remove_copy_cleanup_dir(dir, number)?; - // If the replace directory exists for this extent, then it means - // a repair was interrupted before it could finish. We will continue - // the repair before we open the extent. - let replace_dir = replace_dir(dir, number); - if !read_only && Path::new(&replace_dir).exists() { - info!( - log, - "Extent {} found replacement dir, finishing replacement", - number - ); - move_replacement_extent(dir, number as usize, log)?; + // There are two possible migrations that could be in progress: + // + // - Extent replacement, which is done from a `.replace` directory and + // calls `move_replacement_extent`; this is part of live repair. + // - Extent migration (from SQLite to raw files), which is done from a + // `.migrate` directory and `move_migrate_extent` + // + // Both of these are supposed to complete fully, but it's possible that + // a repair / migration was interrupted before it could finish. If the + // above directories are present, then we are **ready** for repair / + // migration, so we will continue it before opening the extent. + // + // It is not possible for a repair and migration to both be staged; + // migration happens later in this function, so the repair will be + // completed (right here). + if !read_only { + let has_replace = replace_dir(dir, number).exists(); + let has_migrate = migrate_dir(dir, number).exists(); + if has_replace && has_migrate { + bail!( + "cannot have both .replace and .migrate dirs simultaneously" + ); + } else if has_replace { + info!( + log, + "Extent {} found replacement dir, finishing replacement", + number + ); + move_replacement_extent(dir, number as usize, log)?; + } else if has_migrate { + info!( + log, + "Extent {} found migrate dir, finishing migration", number + ); + move_migrate_extent(dir, number as usize, log)?; + } + } + + // Speaking of migration, it's happening now! We will migrate + // every read-write extent with a SQLite file present. + let mut has_sqlite = path.with_extension("db").exists(); + if has_sqlite && !read_only { + let mut inner = extent_inner_sqlite::SqliteInner::open( + &path, def, number, read_only, log, + )?; + let data = inner.export()?; + + // We'll put the new file into copy_dir first, then rename it to + // migrate_dir once it's ready. The logic here matches the + // docstring in `Region::repair_extent` + let copy_dir = copy_dir(dir, number); + let extent_file_name = extent_file_name(number, ExtentType::Data); + let new_file = copy_dir.join(extent_file_name); + mkdir_for_file(&new_file)?; + + info!(log, "Extent {number} writing migration to {new_file:?}"); + std::fs::write(&new_file, data)?; + sync_path(&new_file, log)?; + + // Rename from .copy to .migrate, then sync the parent directory + info!(log, "Extent {number} renaming migration dir"); + std::fs::rename(copy_dir, migrate_dir(dir, number))?; + sync_path(extent_dir(dir, number), log)?; + + info!(log, "Extent {number} copying the migration over"); + move_migrate_extent(dir, number as usize, log)?; + has_sqlite = false; } // Pick the format for the downstairs files // // Right now, this only supports SQLite-flavored Downstairs - let inner = { - let mut sqlite_path = path.clone(); - sqlite_path.set_extension("db"); - if sqlite_path.exists() { + let inner: Box = { + if has_sqlite { let inner = extent_inner_sqlite::SqliteInner::open( &path, def, number, read_only, log, )?; Box::new(inner) } else { - bail!( - "db file {sqlite_path:?} for extent#{number} is not present" - ); + let inner = extent_inner_raw::RawInner::open( + &path, def, number, read_only, log, + )?; + Box::new(inner) } }; - // XXX: schema updates? - let extent = Extent { number, read_only, @@ -368,7 +439,8 @@ impl Extent { } remove_copy_cleanup_dir(dir, number)?; - let inner = extent_inner_sqlite::SqliteInner::create(dir, def, number)?; + // All new extents are created using the raw backend + let inner = extent_inner_raw::RawInner::create(dir, def, number)?; /* * Complete the construction of our new extent @@ -381,6 +453,31 @@ impl Extent { }) } + /// Identical to `create`, but using the SQLite backend + /// + /// This is only allowed in unit tests + #[cfg(test)] + pub fn create_sqlite( + dir: &Path, + def: &RegionDefinition, + number: u32, + ) -> Result { + { + let path = extent_path(dir, number); + if Path::new(&path).exists() { + bail!("Extent file already exists {:?}", path); + } + } + remove_copy_cleanup_dir(dir, number)?; + let inner = extent_inner_sqlite::SqliteInner::create(dir, def, number)?; + Ok(Extent { + number, + read_only: false, + iov_max: Extent::get_iov_max()?, + inner: Mutex::new(Box::new(inner)), + }) + } + pub fn number(&self) -> u32 { self.number } @@ -399,7 +496,7 @@ impl Extent { (job_id.0, self.number, requests.len() as u64) }); - let inner = self.inner.lock().await; + let mut inner = self.inner.lock().await; inner.read(job_id, requests, responses, self.iov_max)?; @@ -483,6 +580,72 @@ impl Extent { } } +/// Apply a staged migration directory to the extent file +/// +/// Migration is only supported from the SQLite extent backend to the raw extent +/// backend. It consists of copying the new (raw) extent file, then deleting +/// the SQLite files. +pub(crate) fn move_migrate_extent>( + region_dir: P, + eid: usize, + log: &Logger, +) -> Result<(), CrucibleError> { + let destination_dir = extent_dir(®ion_dir, eid as u32); + let extent_file_name = extent_file_name(eid as u32, ExtentType::Data); + let migrate_dir = migrate_dir(®ion_dir, eid as u32); + let completed_dir = completed_dir(®ion_dir, eid as u32); + + assert!(Path::new(&migrate_dir).exists()); + assert!(!Path::new(&completed_dir).exists()); + + info!( + log, + "Copy files from {migrate_dir:?} in {destination_dir:?}" + ); + + // Setup the original and replacement file names. + let new_file = migrate_dir.join(&extent_file_name); + let original_file = destination_dir.join(&extent_file_name); + + // Copy the new file on top of the original file. + if let Err(e) = std::fs::copy(new_file.clone(), original_file.clone()) { + crucible_bail!( + IoError, + "copy of {new_file:?} to {original_file:?} got: {e:?}", + ); + } + sync_path(&original_file, log)?; + + // Remove all of the old files. It is always valid for an old file to _not_ + // be present, for two reasons: + // - the .db-shm and .db-wal files are always optional + // - the .db file may have already been removed by a previous call to + // `move_migrate_extent`, if a later step failed + for ext in ["db", "db-shm", "db-wal"] { + let original_file = original_file.with_extension(ext); + if original_file.exists() { + info!( + log, + "Remove old file {original_file:?} as there is no replacement", + ); + std::fs::remove_file(&original_file)?; + } + } + sync_path(&destination_dir, log)?; + + // After we have all files: rename the migrate dir to indicate that + // migration is complete. + info!(log, "Move directory {migrate_dir:?} to {completed_dir:?}"); + std::fs::rename(migrate_dir, &completed_dir)?; + + sync_path(&destination_dir, log)?; + + std::fs::remove_dir_all(&completed_dir)?; + + sync_path(&destination_dir, log)?; + Ok(()) +} + /** * Copy the contents of the replacement directory on to the extent * files in the extent directory. @@ -525,25 +688,14 @@ pub(crate) fn move_replacement_extent>( } sync_path(&original_file, log)?; + // We distinguish between SQLite-backend and raw-file extents based on the + // presence of the `000.db` file. We should never do live migration across + // different extent formats; in fact, we should never live-migrate + // SQLite-backed extents at all, but must still handle the case of + // unfinished migrations. new_file.set_extension("db"); original_file.set_extension("db"); - if let Err(e) = std::fs::copy(new_file.clone(), original_file.clone()) { - crucible_bail!( - IoError, - "copy {:?} to {:?} got: {:?}", - new_file, - original_file, - e - ); - } - sync_path(&original_file, log)?; - - // The .db-shm and .db-wal files may or may not exist. If they don't - // exist on the source side, then be sure to remove them locally to - // avoid database corruption from a mismatch between old and new. - new_file.set_extension("db-shm"); - original_file.set_extension("db-shm"); - if new_file.exists() { + if original_file.exists() { if let Err(e) = std::fs::copy(new_file.clone(), original_file.clone()) { crucible_bail!( IoError, @@ -554,35 +706,57 @@ pub(crate) fn move_replacement_extent>( ); } sync_path(&original_file, log)?; - } else if original_file.exists() { - info!( - log, - "Remove old file {:?} as there is no replacement", - original_file.clone() - ); - std::fs::remove_file(&original_file)?; - } - new_file.set_extension("db-wal"); - original_file.set_extension("db-wal"); - if new_file.exists() { - if let Err(e) = std::fs::copy(new_file.clone(), original_file.clone()) { - crucible_bail!( - IoError, - "copy {:?} to {:?} got: {:?}", - new_file, - original_file, - e + // The .db-shm and .db-wal files may or may not exist. If they don't + // exist on the source side, then be sure to remove them locally to + // avoid database corruption from a mismatch between old and new. + new_file.set_extension("db-shm"); + original_file.set_extension("db-shm"); + if new_file.exists() { + if let Err(e) = + std::fs::copy(new_file.clone(), original_file.clone()) + { + crucible_bail!( + IoError, + "copy {:?} to {:?} got: {:?}", + new_file, + original_file, + e + ); + } + sync_path(&original_file, log)?; + } else if original_file.exists() { + info!( + log, + "Remove old file {:?} as there is no replacement", + original_file.clone() ); + std::fs::remove_file(&original_file)?; + } + + new_file.set_extension("db-wal"); + original_file.set_extension("db-wal"); + if new_file.exists() { + if let Err(e) = + std::fs::copy(new_file.clone(), original_file.clone()) + { + crucible_bail!( + IoError, + "copy {:?} to {:?} got: {:?}", + new_file, + original_file, + e + ); + } + sync_path(&original_file, log)?; + } else if original_file.exists() { + info!( + log, + "Remove old file {:?} as there is no replacement", + original_file.clone() + ); + std::fs::remove_file(&original_file)?; } - sync_path(&original_file, log)?; - } else if original_file.exists() { - info!( - log, - "Remove old file {:?} as there is no replacement", - original_file.clone() - ); - std::fs::remove_file(&original_file)?; } sync_path(&destination_dir, log)?; diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs new file mode 100644 index 000000000..334ac3a0a --- /dev/null +++ b/downstairs/src/extent_inner_raw.rs @@ -0,0 +1,1329 @@ +// Copyright 2023 Oxide Computer Company +use crate::{ + cdt, crucible_bail, + extent::{ + check_input, extent_path, DownstairsBlockContext, ExtentInner, + EXTENT_META_RAW, + }, + integrity_hash, mkdir_for_file, + region::{BatchedPwritev, JobOrReconciliationId}, + Block, BlockContext, CrucibleError, JobId, ReadResponse, RegionDefinition, +}; + +use anyhow::{bail, Result}; +use serde::{Deserialize, Serialize}; +use slog::{error, Logger}; + +use std::collections::HashSet; +use std::fs::{File, OpenOptions}; +use std::io::{IoSliceMut, Read, Seek, SeekFrom}; +use std::os::fd::AsRawFd; +use std::path::Path; + +/// Equivalent to `DownstairsBlockContext`, but without one's own block number +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OnDiskDownstairsBlockContext { + pub block_context: BlockContext, + pub on_disk_hash: u64, +} + +/// Equivalent to `ExtentMeta`, but ordered for efficient on-disk serialization +/// +/// In particular, the `dirty` byte is first, so it's easy to read at a known +/// offset within the file. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct OnDiskMeta { + pub dirty: bool, + pub gen_number: u64, + pub flush_number: u64, + pub ext_version: u32, +} + +/// Size of backup data +/// +/// This must be large enough to contain an `Option` +/// serialized using `bincode`. +pub const BLOCK_CONTEXT_SLOT_SIZE_BYTES: u64 = 64; + +/// Size of metadata region +/// +/// This must be large enough to contain an `OnDiskMeta` serialized using +/// `bincode`. +pub const BLOCK_META_SIZE_BYTES: u64 = 64; + +/// `RawInner` is a wrapper around a [`std::fs::File`] representing an extent +/// +/// The file is structured as follows: +/// - Block data, structured as `block_size` × `extent_size` +/// - Block contexts (for encryption). Each block index (in the range +/// `0..extent_size`) has two context slots; we use a ping-pong strategy when +/// writing to ensure that one slot is always valid. Each slot is +/// - [`BLOCK_META_SIZE_BYTES`], which contains an [`OnDiskMeta`] serialized +/// using `bincode`. The first byte of this range is `dirty`, serialized as a +/// `u8` (where `1` is dirty and `0` is clean). +/// [`BLOCK_CONTEXT_SLOT_SIZE_BYTES`] in size, so this region is +/// `BLOCK_CONTEXT_SLOT_SIZE_BYTES * extent_size * 2` bytes in total. The +/// slots contain an `Option`, serialized using +/// `bincode`. +#[derive(Debug)] +pub struct RawInner { + file: File, + + /// Our extent number + extent_number: u32, + + /// Extent size, in blocks + extent_size: Block, + + /// Is the `ping` or `pong` context slot active? + active_context: Vec, + + /// Local cache for the `dirty` value + /// + /// This allows us to only write the flag when the value changes + dirty: bool, + + /// Monotonically increasing sync index + /// + /// This is unrelated to `flush_number` or `gen_number`; it's purely + /// internal to this data structure and is transient. We use this value to + /// determine whether a context slot has been persisted to disk or not. + sync_index: u64, + + /// The value of `sync_index` when the current value of a context slot was + /// synched to disk. When the value of a context slot changes, the value in + /// this array is set to `self.sync_index + 1` indicates that the slot has + /// not yet been synched. + context_slot_synched_at: Vec<[u64; 2]>, +} + +#[derive(Copy, Clone, Debug)] +enum ActiveContext { + /// There is no active context for the given block + Empty, + + /// The active context for the given block is unknown + /// + /// We set the context to this state before a write, so that if the write + /// fails, we rehash the file contents to get back to a known state. + Unknown, + + /// The active context is stored in the given slot + Slot(bool), +} + +impl ExtentInner for RawInner { + fn flush_number(&self) -> Result { + self.get_metadata().map(|v| v.flush_number) + } + + fn gen_number(&self) -> Result { + self.get_metadata().map(|v| v.gen_number) + } + + fn dirty(&self) -> Result { + Ok(self.dirty) + } + + fn write( + &mut self, + job_id: JobId, + writes: &[&crucible_protocol::Write], + only_write_unwritten: bool, + iov_max: usize, + ) -> Result<(), CrucibleError> { + /* + * In order to be crash consistent, perform the following steps in + * order: + * + * 1) set the dirty bit + * 2) for each write: + * a) write out encryption context and hashes first + * b) write out extent data second + * + * If encryption context is written after the extent data, a crash or + * interruption before extent data is written would potentially leave + * data on the disk that cannot be decrypted. + * + * If hash is written after extent data, same thing - a crash or + * interruption would leave data on disk that would fail the + * integrity hash check. + * + * Note that writing extent data here does not assume that it is + * durably on disk - the only guarantee of that is returning + * ok from fsync. The data is only potentially on disk and + * this depends on operating system implementation. + * + * To minimize the performance hit of sending many transactions to the + * filesystem, as much as possible is written at the same time. This + * means multiple loops are required. The steps now look like: + * + * 1) set the dirty bit + * 2) gather and write all encryption contexts + hashes + * 3) write all extent data + * + * If "only_write_unwritten" is true, then we only issue a write for + * a block if that block has not been written to yet. Note + * that we can have a write that is "sparse" if the range of + * blocks it contains has a mix of written an unwritten + * blocks. + * + * We define a block being written to or not has if that block has + * `Some(...)` with a matching checksum serialized into a context slot + * or not. So it is required that a written block has a checksum. + */ + + // If `only_write_written`, we need to skip writing to blocks that + // already contain data. We'll first query the metadata to see which + // blocks have hashes + let mut writes_to_skip = HashSet::new(); + if only_write_unwritten { + cdt::extent__write__get__hashes__start!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + let mut write_run_start = 0; + while write_run_start < writes.len() { + let first_write = writes[write_run_start]; + + // Starting from the first write in the potential run, we scan + // forward until we find a write with a block that isn't + // contiguous with the request before it. Since we're counting + // pairs, and the number of pairs is one less than the number of + // writes, we need to add 1 to get our actual run length. + let n_contiguous_writes = writes[write_run_start..] + .windows(2) + .take_while(|wr_pair| { + wr_pair[0].offset.value + 1 == wr_pair[1].offset.value + }) + .count() + + 1; + + // Query hashes for the write range. + let block_contexts = self.get_block_contexts( + first_write.offset.value, + n_contiguous_writes as u64, + )?; + + for (i, block_contexts) in block_contexts.iter().enumerate() { + if block_contexts.is_some() { + let _ = writes_to_skip + .insert(i as u64 + first_write.offset.value); + } + } + + write_run_start += n_contiguous_writes; + } + cdt::extent__write__get__hashes__done!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + if writes_to_skip.len() == writes.len() { + // Nothing to do + return Ok(()); + } + } + + self.set_dirty()?; + + // Write all the metadata to the raw file, at the end + // + // TODO right now we're including the integrity_hash() time in the + // measured time. Is it small enough to be ignored? + cdt::extent__write__raw__context__insert__start!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + let mut pending_slots = vec![]; + for write in writes { + if writes_to_skip.contains(&write.offset.value) { + pending_slots.push(None); + continue; + } + + // TODO it would be nice if we could profile what % of time we're + // spending on hashes locally vs writing to disk + let on_disk_hash = integrity_hash(&[&write.data[..]]); + + let next_slot = + self.set_block_context(&DownstairsBlockContext { + block_context: write.block_context, + block: write.offset.value, + on_disk_hash, + })?; + pending_slots.push(Some(next_slot)); + + // Until the write lands, we can't trust the context slot and must + // rehash it if requested. This assignment is overwritten after the + // write finishes. + self.active_context[write.offset.value as usize] = + ActiveContext::Unknown; + } + + cdt::extent__write__raw__context__insert__done!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + // PERFORMANCE TODO: + // + // Something worth considering for small writes is that, based on + // my memory of conversations we had with propolis folks about what + // OSes expect out of an NVMe driver, I believe our contract with the + // upstairs doesn't require us to have the writes inside the file + // until after a flush() returns. If that is indeed true, we could + // buffer a certain amount of writes, only actually writing that + // buffer when either a flush is issued or the buffer exceeds some + // set size (based on our memory constraints). This would have + // benefits on any workload that frequently writes to the same block + // between flushes, would have benefits for small contiguous writes + // issued over multiple write commands by letting us batch them into + // a larger write, and (speculation) may benefit non-contiguous writes + // by cutting down the number of metadata writes. But, it introduces + // complexity. The time spent implementing that would probably better be + // spent switching to aio or something like that. + cdt::extent__write__file__start!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + // Now, batch writes into iovecs and use pwritev to write them all out. + let mut batched_pwritev = BatchedPwritev::new( + self.file.as_raw_fd(), + writes.len(), + self.extent_size.block_size_in_bytes().into(), + iov_max, + ); + + for write in writes { + let block = write.offset.value; + + // TODO, can this be `only_write_unwritten && + // write_to_skip.contains()`? + if writes_to_skip.contains(&block) { + debug_assert!(only_write_unwritten); + batched_pwritev.perform_writes()?; + continue; + } + + batched_pwritev.add_write(write)?; + } + + // Write any remaining data + batched_pwritev.perform_writes()?; + + // Now that writes have gone through, update our active context slots + for (write, new_slot) in writes.iter().zip(pending_slots) { + if let Some(slot) = new_slot { + self.active_context[write.offset.value as usize] = + ActiveContext::Slot(slot); + } + } + + cdt::extent__write__file__done!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + Ok(()) + } + + fn read( + &mut self, + job_id: JobId, + requests: &[&crucible_protocol::ReadRequest], + responses: &mut Vec, + iov_max: usize, + ) -> Result<(), CrucibleError> { + // This code batches up operations for contiguous regions of + // ReadRequests, so we can perform larger read syscalls queries. This + // significantly improves read throughput. + + // Keep track of the index of the first request in any contiguous run + // of requests. Of course, a "contiguous run" might just be a single + // request. + let mut req_run_start = 0; + let block_size = self.extent_size.block_size_in_bytes(); + while req_run_start < requests.len() { + let first_req = requests[req_run_start]; + + // Starting from the first request in the potential run, we scan + // forward until we find a request with a block that isn't + // contiguous with the request before it. Since we're counting + // pairs, and the number of pairs is one less than the number of + // requests, we need to add 1 to get our actual run length. + let mut n_contiguous_requests = 1; + + for request_window in requests[req_run_start..].windows(2) { + if (request_window[0].offset.value + 1 + == request_window[1].offset.value) + && ((n_contiguous_requests + 1) < iov_max) + { + n_contiguous_requests += 1; + } else { + break; + } + } + + // Create our responses and push them into the output. While we're + // at it, check for overflows. + let resp_run_start = responses.len(); + let mut iovecs = Vec::with_capacity(n_contiguous_requests); + for req in requests[req_run_start..][..n_contiguous_requests].iter() + { + let resp = ReadResponse::from_request(req, block_size as usize); + check_input(self.extent_size, req.offset, &resp.data)?; + responses.push(resp); + } + + // Create what amounts to an iovec for each response data buffer. + for resp in + &mut responses[resp_run_start..][..n_contiguous_requests] + { + iovecs.push(IoSliceMut::new(&mut resp.data[..])); + } + + // Finally we get to read the actual data. That's why we're here + cdt::extent__read__file__start!(|| { + (job_id.0, self.extent_number, n_contiguous_requests as u64) + }); + + nix::sys::uio::preadv( + self.file.as_raw_fd(), + &mut iovecs, + first_req.offset.value as i64 * block_size as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + cdt::extent__read__file__done!(|| { + (job_id.0, self.extent_number, n_contiguous_requests as u64) + }); + + // Query the block metadata + cdt::extent__read__get__contexts__start!(|| { + (job_id.0, self.extent_number, n_contiguous_requests as u64) + }); + let block_contexts = self.get_block_contexts( + first_req.offset.value, + n_contiguous_requests as u64, + )?; + cdt::extent__read__get__contexts__done!(|| { + (job_id.0, self.extent_number, n_contiguous_requests as u64) + }); + + // Now it's time to put block contexts into the responses. + // We use into_iter here to move values out of enc_ctxts/hashes, + // avoiding a clone(). For code consistency, we use iters for the + // response and data chunks too. These iters will be the same length + // (equal to n_contiguous_requests) so zipping is fine + let resp_iter = + responses[resp_run_start..][..n_contiguous_requests].iter_mut(); + let ctx_iter = block_contexts.into_iter(); + + for (resp, r_ctx) in resp_iter.zip(ctx_iter) { + resp.block_contexts = + r_ctx.into_iter().map(|x| x.block_context).collect(); + } + + req_run_start += n_contiguous_requests; + } + + cdt::extent__read__done!(|| { + (job_id.0, self.extent_number, requests.len() as u64) + }); + + Ok(()) + } + + fn flush( + &mut self, + new_flush: u64, + new_gen: u64, + job_id: JobOrReconciliationId, + ) -> Result<(), CrucibleError> { + if !self.dirty()? { + /* + * If we have made no writes to this extent since the last flush, + * we do not need to update the extent on disk + */ + return Ok(()); + } + + // We put all of our metadata updates into a single write to make this + // operation atomic. + self.set_flush_number(new_flush, new_gen)?; + + // Now, we fsync to ensure data is flushed to disk. It's okay to crash + // before this point, because setting the flush number is atomic. + cdt::extent__flush__file__start!(|| { + (job_id.get(), self.extent_number, 0) + }); + if let Err(e) = self.file.sync_all() { + /* + * XXX Retry? Mark extent as broken? + */ + crucible_bail!( + IoError, + "extent {}: fsync 1 failure: {:?}", + self.extent_number, + e + ); + } + self.sync_index += 1; + cdt::extent__flush__file__done!(|| { + (job_id.get(), self.extent_number, 0) + }); + + // Finally, reset the file's seek offset to 0 + self.file.seek(SeekFrom::Start(0))?; + + cdt::extent__flush__done!(|| { (job_id.get(), self.extent_number, 0) }); + + Ok(()) + } + + #[cfg(test)] + fn set_dirty_and_block_context( + &mut self, + block_context: &DownstairsBlockContext, + ) -> Result<(), CrucibleError> { + self.set_dirty()?; + let new_slot = self.set_block_context(block_context)?; + self.active_context[block_context.block as usize] = + ActiveContext::Slot(new_slot); + Ok(()) + } + + #[cfg(test)] + fn get_block_contexts( + &mut self, + block: u64, + count: u64, + ) -> Result>, CrucibleError> { + let out = RawInner::get_block_contexts(self, block, count)?; + Ok(out.into_iter().map(|v| v.into_iter().collect()).collect()) + } +} + +impl RawInner { + pub fn create( + dir: &Path, + def: &RegionDefinition, + extent_number: u32, + ) -> Result { + let path = extent_path(dir, extent_number); + let bcount = def.extent_size().value; + let size = def.block_size().checked_mul(bcount).unwrap() + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2 + + BLOCK_META_SIZE_BYTES; + + mkdir_for_file(&path)?; + let mut file = OpenOptions::new() + .read(true) + .write(true) + .create(true) + .open(&path)?; + + // All 0s are fine for everything except extent version in the metadata + file.set_len(size)?; + file.seek(SeekFrom::Start(0))?; + let mut out = Self { + file, + dirty: false, + extent_size: def.extent_size(), + extent_number, + active_context: vec![ + ActiveContext::Empty; + def.extent_size().value as usize + ], + context_slot_synched_at: vec![ + [0, 0]; + def.extent_size().value as usize + ], + sync_index: 0, + }; + // Setting the flush number also writes the extent version, since + // they're serialized together in the same block. + out.set_flush_number(0, 0)?; + + // Sync the file to disk, to avoid any questions + if let Err(e) = out.file.sync_all() { + return Err(CrucibleError::IoError(format!( + "extent {}: fsync 1 failure during initial sync: {e:?}", + out.extent_number, + )) + .into()); + } + Ok(out) + } + + /// Constructs a new `Inner` object from files that already exist on disk + pub fn open( + path: &Path, + def: &RegionDefinition, + extent_number: u32, + read_only: bool, + log: &Logger, + ) -> Result { + let bcount = def.extent_size().value; + let size = def.block_size().checked_mul(bcount).unwrap() + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2 + + BLOCK_META_SIZE_BYTES; + + /* + * Open the extent file and verify the size is as we expect. + */ + let mut file = + match OpenOptions::new().read(true).write(!read_only).open(path) { + Err(e) => { + error!( + log, + "Open of {path:?} for extent#{extent_number} \ + returned: {e}", + ); + bail!( + "Open of {path:?} for extent#{extent_number} \ + returned: {e}", + ); + } + Ok(f) => { + let cur_size = f.metadata().unwrap().len(); + if size != cur_size { + bail!( + "File size {size:?} does not match \ + expected {cur_size:?}", + ); + } + f + } + }; + + // Just in case, let's be very sure that the file on disk is what it + // should be + if let Err(e) = file.sync_all() { + return Err(CrucibleError::IoError(format!( + "extent {extent_number}: + fsync 1 failure during initial rehash: {e:?}", + )) + .into()); + } + + file.seek(SeekFrom::Start(Self::meta_offset_from_extent_size( + def.extent_size(), + )))?; + let mut dirty = [0u8]; + file.read_exact(&mut dirty)?; + let dirty = match dirty[0] { + 0 => false, + 1 => true, + i => bail!("invalid dirty value: {i}"), + }; + + Ok(Self { + file, + // Lazy initialization of which context slot is active + active_context: vec![ + ActiveContext::Unknown; + def.extent_size().value as usize + ], + dirty, + extent_number, + extent_size: def.extent_size(), + context_slot_synched_at: vec![ + [0, 0]; + def.extent_size().value as usize + ], + sync_index: 0, + }) + } + + fn set_dirty(&mut self) -> Result<(), CrucibleError> { + if !self.dirty { + let offset = self.meta_offset(); + nix::sys::uio::pwrite(self.file.as_raw_fd(), &[1u8], offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + self.dirty = true; + } + Ok(()) + } + + /// Looks up the current context slot for the given block + /// + /// If the slot is currently unknown, rehashes the block and picks one of + /// the two slots, storing it locally. + fn get_block_context_slot(&mut self, block: usize) -> Result> { + match self.active_context[block] { + ActiveContext::Empty => Ok(None), + ActiveContext::Slot(b) => Ok(Some(b)), + ActiveContext::Unknown => { + let block_size = self.extent_size.block_size_in_bytes(); + let mut buf = vec![0; block_size as usize]; + self.file + .seek(SeekFrom::Start(block_size as u64 * block as u64))?; + self.file.read_exact(&mut buf)?; + let hash = integrity_hash(&[&buf]); + + self.file.seek(SeekFrom::Start( + self.context_slot_offset(block as u64, false), + ))?; + let mut buf = vec![0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + let mut found = None; + for slot in 0..2 { + self.file.read_exact(&mut buf)?; + let context: Option = + bincode::deserialize(&buf)?; + if let Some(context) = context { + if context.on_disk_hash == hash { + self.active_context[block] = + ActiveContext::Slot(slot != 0); + found = Some(slot != 0); + } + } + } + if found.is_none() { + self.active_context[block] = ActiveContext::Empty; + } + Ok(found) + } + } + } + + /// Writes the inactive block context slot + /// + /// Returns the new slot which should be marked as active after the write + fn set_block_context( + &mut self, + block_context: &DownstairsBlockContext, + ) -> Result { + let block = block_context.block as usize; + // Select the inactive slot + let slot = !self.get_block_context_slot(block)?.unwrap_or(true); + + // If the context slot that we're about to write into hasn't been + // synched to disk yet, we must sync it first. This prevents subtle + // ordering issues! + let last_sync = &mut self.context_slot_synched_at[block][slot as usize]; + if *last_sync > self.sync_index { + assert_eq!(*last_sync, self.sync_index + 1); + if let Err(e) = self.file.sync_all() { + return Err(CrucibleError::IoError(format!( + "extent {}: fsync 1 failure: {:?}", + self.extent_number, e + )) + .into()); + } + self.sync_index += 1; + } + // The given slot is about to be newly unsynched, because we're going to + // write to it below. + *last_sync = self.sync_index + 1; + + let offset = self.context_slot_offset(block_context.block, slot); + + // Serialize into a local buffer, then write into the inactive slot + let mut buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + let d = OnDiskDownstairsBlockContext { + block_context: block_context.block_context, + on_disk_hash: block_context.on_disk_hash, + }; + bincode::serialize_into(buf.as_mut_slice(), &Some(d))?; + nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + // Return the just-written slot; it's the caller's responsibility to + // select it as active once data is written. + Ok(slot) + } + + fn get_metadata(&self) -> Result { + let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + let offset = self.meta_offset(); + nix::sys::uio::pread(self.file.as_raw_fd(), &mut buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + let out: OnDiskMeta = bincode::deserialize(&buf) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + Ok(out) + } + + /// Returns the byte offset of the given context slot + /// + /// Contexts slots are located after block data in the extent file. There + /// are two context slots per block. We use a ping-pong strategy to ensure + /// that one of them is always valid (i.e. matching the data in the file). + fn context_slot_offset(&self, block: u64, slot: bool) -> u64 { + self.extent_size.block_size_in_bytes() as u64 * self.extent_size.value + + BLOCK_META_SIZE_BYTES + + (block * 2 + slot as u64) * BLOCK_CONTEXT_SLOT_SIZE_BYTES + } + + /// Returns the byte offset of the metadata region + /// + /// The resulting offset points to serialized [`OnDiskMeta`] data. + fn meta_offset(&self) -> u64 { + Self::meta_offset_from_extent_size(self.extent_size) + } + + fn meta_offset_from_extent_size(extent_size: Block) -> u64 { + extent_size.block_size_in_bytes() as u64 * extent_size.value + } + + fn get_block_context( + &mut self, + block: u64, + ) -> Result> { + let Some(slot) = self.get_block_context_slot(block as usize)? else { + return Ok(None); + }; + let offset = self.context_slot_offset(block, slot); + let mut buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + nix::sys::uio::pread(self.file.as_raw_fd(), &mut buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + let out: Option = + bincode::deserialize(&buf)?; + let out = out.map(|c| DownstairsBlockContext { + block, + block_context: c.block_context, + on_disk_hash: c.on_disk_hash, + }); + + Ok(out) + } + + /// Update the flush number, generation number, and clear the dirty bit + fn set_flush_number(&mut self, new_flush: u64, new_gen: u64) -> Result<()> { + let d = OnDiskMeta { + dirty: false, + flush_number: new_flush, + gen_number: new_gen, + ext_version: EXTENT_META_RAW, + }; + // Byte 0 is the dirty byte + let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + + bincode::serialize_into(buf.as_mut_slice(), &d)?; + let offset = self.meta_offset(); + nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + self.dirty = false; + + Ok(()) + } + + /// Returns the valid block contexts (or `None`) for the given block range + fn get_block_contexts( + &mut self, + block: u64, + count: u64, + ) -> Result>> { + let mut out = vec![]; + for i in block..block + count { + let ctx = self.get_block_context(i)?; + out.push(ctx); + } + Ok(out) + } +} + +#[cfg(test)] +mod test { + use super::*; + use bytes::{Bytes, BytesMut}; + use crucible_protocol::EncryptionContext; + use crucible_protocol::ReadRequest; + use rand::Rng; + use tempfile::tempdir; + + const IOV_MAX_TEST: usize = 1000; + + fn new_region_definition() -> RegionDefinition { + let opt = crate::region::test::new_region_options(); + RegionDefinition::from_options(&opt).unwrap() + } + + #[tokio::test] + async fn encryption_context() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Encryption context for blocks 0 and 1 should start blank + + assert!(inner.get_block_contexts(0, 1)?[0].is_none()); + assert!(inner.get_block_contexts(1, 1)?[0].is_none()); + + // Set and verify block 0's context + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: Some(EncryptionContext { + nonce: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], + tag: [ + 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, + 19, + ], + }), + hash: 123, + }, + block: 0, + on_disk_hash: 456, + })?; + + let ctxs = inner.get_block_contexts(0, 1)?; + let ctx = ctxs[0].as_ref().unwrap(); + assert_eq!( + ctx.block_context.encryption_context.unwrap().nonce, + [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] + ); + assert_eq!( + ctx.block_context.encryption_context.unwrap().tag, + [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19] + ); + assert_eq!(ctx.block_context.hash, 123); + assert_eq!(ctx.on_disk_hash, 456); + + // Block 1 should still be blank + assert!(inner.get_block_contexts(1, 1)?[0].is_none()); + + // Set and verify a new context for block 0 + let blob1 = rand::thread_rng().gen::<[u8; 12]>(); + let blob2 = rand::thread_rng().gen::<[u8; 16]>(); + + // Set and verify block 0's context + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: Some(EncryptionContext { + nonce: blob1, + tag: blob2, + }), + hash: 1024, + }, + block: 0, + on_disk_hash: 65536, + })?; + + let ctxs = inner.get_block_contexts(0, 1)?; + let ctx = ctxs[0].as_ref().unwrap(); + + // Second context was appended + assert_eq!(ctx.block_context.encryption_context.unwrap().nonce, blob1); + assert_eq!(ctx.block_context.encryption_context.unwrap().tag, blob2); + assert_eq!(ctx.block_context.hash, 1024); + assert_eq!(ctx.on_disk_hash, 65536); + + Ok(()) + } + + #[tokio::test] + async fn multiple_context() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Encryption context for blocks 0 and 1 should start blank + + assert!(inner.get_block_contexts(0, 1)?[0].is_none()); + assert!(inner.get_block_contexts(1, 1)?[0].is_none()); + + // Set block 0's and 1's context and dirty flag + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: Some(EncryptionContext { + nonce: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], + tag: [ + 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, + 19, + ], + }), + hash: 123, + }, + block: 0, + on_disk_hash: 456, + })?; + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: Some(EncryptionContext { + nonce: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], + tag: [8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], + }), + hash: 9999, + }, + block: 1, + on_disk_hash: 1234567890, + })?; + + // Verify block 0's context + let ctxs = inner.get_block_contexts(0, 1)?; + let ctx = ctxs[0].as_ref().unwrap(); + assert_eq!( + ctx.block_context.encryption_context.unwrap().nonce, + [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] + ); + assert_eq!( + ctx.block_context.encryption_context.unwrap().tag, + [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19] + ); + assert_eq!(ctx.block_context.hash, 123); + assert_eq!(ctx.on_disk_hash, 456); + + // Verify block 1's context + let ctxs = inner.get_block_contexts(1, 1)?; + let ctx = ctxs[0].as_ref().unwrap(); + + assert_eq!( + ctx.block_context.encryption_context.unwrap().nonce, + [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] + ); + assert_eq!( + ctx.block_context.encryption_context.unwrap().tag, + [8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13] + ); + assert_eq!(ctx.block_context.hash, 9999); + assert_eq!(ctx.on_disk_hash, 1234567890); + + // Return both block 0's and block 1's context, and verify + + let ctxs = inner.get_block_contexts(0, 2)?; + let ctx = ctxs[0].as_ref().unwrap(); + assert_eq!( + ctx.block_context.encryption_context.unwrap().nonce, + [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] + ); + assert_eq!( + ctx.block_context.encryption_context.unwrap().tag, + [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19] + ); + assert_eq!(ctx.block_context.hash, 123); + assert_eq!(ctx.on_disk_hash, 456); + + let ctx = ctxs[1].as_ref().unwrap(); + assert_eq!( + ctx.block_context.encryption_context.unwrap().nonce, + [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] + ); + assert_eq!( + ctx.block_context.encryption_context.unwrap().tag, + [8, 9, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13] + ); + assert_eq!(ctx.block_context.hash, 9999); + assert_eq!(ctx.on_disk_hash, 1234567890); + + // Append a whole bunch of block context rows + for i in 0..10 { + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: Some(EncryptionContext { + nonce: rand::thread_rng().gen::<[u8; 12]>(), + tag: rand::thread_rng().gen::<[u8; 16]>(), + }), + hash: rand::thread_rng().gen::(), + }, + block: 0, + on_disk_hash: i, + })?; + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: Some(EncryptionContext { + nonce: rand::thread_rng().gen::<[u8; 12]>(), + tag: rand::thread_rng().gen::<[u8; 16]>(), + }), + hash: rand::thread_rng().gen::(), + }, + block: 1, + on_disk_hash: i, + })?; + } + + let ctxs = inner.get_block_contexts(0, 2)?; + + assert!(ctxs[0].is_some()); + assert_eq!(ctxs[0].as_ref().unwrap().on_disk_hash, 9); + + assert!(ctxs[1].is_some()); + assert_eq!(ctxs[1].as_ref().unwrap().on_disk_hash, 9); + + Ok(()) + } + + #[test] + fn test_write_unwritten_without_flush() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write a block, but don't flush. + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data, + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; + + // The context should be in place, though we haven't flushed yet + + // Therefore, we expect that write_unwritten to the first block won't + // do anything. + { + let data = Bytes::from(vec![0x66; 512]); + let hash = integrity_hash(&[&data[..]]); + let block_context = BlockContext { + encryption_context: None, + hash, + }; + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data: data.clone(), + block_context, + }; + inner.write(JobId(20), &[&write], true, IOV_MAX_TEST)?; + + let mut resp = Vec::new(); + let read = ReadRequest { + eid: 0, + offset: Block::new_512(0), + }; + inner.read(JobId(21), &[&read], &mut resp, IOV_MAX_TEST)?; + + // We should not get back our data, because block 0 was written. + assert_ne!( + resp, + vec![ReadResponse { + eid: 0, + offset: Block::new_512(0), + data: BytesMut::from(data.as_ref()), + block_contexts: vec![block_context] + }] + ); + } + + // But, writing to the second block still should! + { + let data = Bytes::from(vec![0x66; 512]); + let hash = integrity_hash(&[&data[..]]); + let block_context = BlockContext { + encryption_context: None, + hash, + }; + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(1), + data: data.clone(), + block_context, + }; + inner.write(JobId(30), &[&write], true, IOV_MAX_TEST)?; + + let mut resp = Vec::new(); + let read = ReadRequest { + eid: 0, + offset: Block::new_512(1), + }; + inner.read(JobId(31), &[&read], &mut resp, IOV_MAX_TEST)?; + + // We should get back our data! Block 1 was never written. + assert_eq!( + resp, + vec![ReadResponse { + eid: 0, + offset: Block::new_512(1), + data: BytesMut::from(data.as_ref()), + block_contexts: vec![block_context] + }] + ); + } + + Ok(()) + } + + #[test] + fn test_auto_sync() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write a block, but don't flush. + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data, + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + // The context should be written to slot 0 + inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 0); + + // The context should be written to slot 1 + inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 0); + + // The context should be written to slot 0, forcing a sync + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 1); + + Ok(()) + } + + #[test] + fn test_auto_sync_flush() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write a block, but don't flush. + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data, + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + // The context should be written to slot 0 + inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 0); + + // Flush, which should bump the sync number (marking slot 0 as synched) + inner.flush(12, 12, JobId(11).into())?; + + // The context should be written to slot 1 + inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 1); + + // The context should be written to slot 0 + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 1); + + // The context should be written to slot 1, forcing a sync + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 2); + + Ok(()) + } + + #[test] + fn test_auto_sync_flush_2() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write a block, but don't flush. + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data, + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + // The context should be written to slot 0 + inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 0); + + // The context should be written to slot 1 + inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 0); + + // Flush, which should bump the sync number (marking slot 0 as synched) + inner.flush(12, 12, JobId(11).into())?; + + // The context should be written to slot 0 + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 1); + + // The context should be written to slot 1 + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 1); + + // The context should be written to slot 0, forcing a sync + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.sync_index, 2); + + Ok(()) + } + + /// If a write successfully put a context into a context slot, but it never + /// actually got the data onto the disk, that block should revert back to + /// being "unwritten". After all, the data never was truly written. + /// + /// This test is very similar to test_region_open_removes_partial_writes. + #[test] + fn test_reopen_marks_blocks_unwritten_if_data_never_hit_disk() -> Result<()> + { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Partial write, the data never hits disk, but there's a context + // in the DB and the dirty flag is set. + inner.set_dirty_and_block_context(&DownstairsBlockContext { + block_context: BlockContext { + encryption_context: None, + hash: 1024, + }, + block: 0, + on_disk_hash: 65536, + })?; + drop(inner); + + // Reopen, which should note that the hash doesn't match on-disk values + // and decide that block 0 is unwritten. + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Writing to block 0 should succeed with only_write_unwritten + { + let data = Bytes::from(vec![0x66; 512]); + let hash = integrity_hash(&[&data[..]]); + let block_context = BlockContext { + encryption_context: None, + hash, + }; + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data: data.clone(), + block_context, + }; + inner.write(JobId(30), &[&write], true, IOV_MAX_TEST)?; + + let mut resp = Vec::new(); + let read = ReadRequest { + eid: 0, + offset: Block::new_512(0), + }; + inner.read(JobId(31), &[&read], &mut resp, IOV_MAX_TEST)?; + + // We should get back our data! Block 1 was never written. + assert_eq!( + resp, + vec![ReadResponse { + eid: 0, + offset: Block::new_512(0), + data: BytesMut::from(data.as_ref()), + block_contexts: vec![block_context] + }] + ); + } + + Ok(()) + } +} diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 24a9144a7..255b1257d 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -1,11 +1,8 @@ // Copyright 2023 Oxide Computer Company use crate::{ cdt, crucible_bail, - extent::{ - check_input, extent_path, DownstairsBlockContext, ExtentInner, - ExtentMeta, EXTENT_META_SQLITE, - }, - integrity_hash, mkdir_for_file, + extent::{check_input, DownstairsBlockContext, ExtentInner}, + integrity_hash, region::{BatchedPwritev, JobOrReconciliationId}, Block, BlockContext, CrucibleError, JobId, ReadResponse, RegionDefinition, }; @@ -160,7 +157,7 @@ impl ExtentInner for SqliteInner { } fn read( - &self, + &mut self, job_id: JobId, requests: &[&crucible_protocol::ReadRequest], responses: &mut Vec, @@ -394,7 +391,7 @@ impl ExtentInner for SqliteInner { let on_disk_hash = integrity_hash(&[&write.data[..]]); self.set_block_context(&DownstairsBlockContext { - block_context: write.block_context.clone(), + block_context: write.block_context, block: write.offset.value, on_disk_hash, })?; @@ -489,7 +486,7 @@ impl ExtentInner for SqliteInner { /// parent Vec contains all contexts for a single block. #[cfg(test)] fn get_block_contexts( - &self, + &mut self, block: u64, count: u64, ) -> Result>, CrucibleError> { @@ -508,6 +505,68 @@ impl ExtentInner for SqliteInner { } impl SqliteInner { + /// Converts to a raw file for use with `RawInner` + pub fn export(&mut self) -> Result, CrucibleError> { + // Clean up stale hashes. After this is done, each block should have + // either 0 or 1 contexts. + self.fully_rehash_and_clean_all_stale_contexts(true)?; + + let block_size = self.extent_size.block_size_in_bytes() as usize; + let ctxs = self.get_block_contexts(0, self.extent_size.value)?; + + // Read file contents, which are the beginning of the raw file + self.file.seek(SeekFrom::Start(0))?; + let mut buf = vec![0u8; self.extent_size.value as usize * block_size]; + self.file.read_exact(&mut buf)?; + + use crate::{ + extent::EXTENT_META_RAW, + extent_inner_raw::{ + OnDiskDownstairsBlockContext, OnDiskMeta, + BLOCK_CONTEXT_SLOT_SIZE_BYTES, BLOCK_META_SIZE_BYTES, + }, + }; + + // Record the metadata region after the raw block data + let dirty = self.dirty()?; + let flush_number = self.flush_number()?; + let gen_number = self.gen_number()?; + let meta = OnDiskMeta { + dirty, + flush_number, + gen_number, + ext_version: EXTENT_META_RAW, + }; + let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(meta_buf.as_mut_slice(), &meta) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + buf.extend(meta_buf); + + // Put the context data after the metadata + for c in ctxs { + let ctx = match c.len() { + 0 => None, + 1 => Some(OnDiskDownstairsBlockContext { + block_context: c[0].block_context, + on_disk_hash: c[0].on_disk_hash, + }), + i => panic!("invalid context count: {i}"), + }; + // Put the context into the first slot, if present + let mut ctx_buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + bincode::serialize_into(ctx_buf.as_mut_slice(), &ctx) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + buf.extend(ctx_buf); + + // The second slot is empty + buf.extend([0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]); + } + + // Reset the file read position, just in case + self.file.seek(SeekFrom::Start(0))?; + Ok(buf) + } + fn get_block_contexts( &self, block: u64, @@ -558,11 +617,19 @@ impl SqliteInner { Ok(results) } + // We should never create a new SQLite-backed extent in production code, + // because we should be using raw extents everywhere. However, we'll create + // them during tests to check that our automatic migration system works. + #[cfg(test)] pub fn create( dir: &Path, def: &RegionDefinition, extent_number: u32, ) -> Result { + use crate::{ + extent::{extent_path, ExtentMeta, EXTENT_META_SQLITE}, + mkdir_for_file, + }; let mut path = extent_path(dir, extent_number); let bcount = def.extent_size().value; let size = def.block_size().checked_mul(bcount).unwrap(); @@ -1496,7 +1563,7 @@ mod test { eid: 0, offset: Block::new_512(0), data: data.clone(), - block_context: block_context.clone(), + block_context, }; inner.write(JobId(20), &[&write], true, IOV_MAX_TEST)?; @@ -1531,7 +1598,7 @@ mod test { eid: 0, offset: Block::new_512(1), data: data.clone(), - block_context: block_context.clone(), + block_context, }; inner.write(JobId(30), &[&write], true, IOV_MAX_TEST)?; @@ -1598,7 +1665,7 @@ mod test { eid: 0, offset: Block::new_512(0), data: data.clone(), - block_context: block_context.clone(), + block_context, }; inner.write(JobId(30), &[&write], true, IOV_MAX_TEST)?; diff --git a/downstairs/src/lib.rs b/downstairs/src/lib.rs index 257d190c9..7e511f0d6 100644 --- a/downstairs/src/lib.rs +++ b/downstairs/src/lib.rs @@ -37,6 +37,7 @@ pub mod region; pub mod repair; mod stats; +mod extent_inner_raw; mod extent_inner_sqlite; use region::Region; @@ -445,6 +446,18 @@ pub mod cdt { n_blocks: u64, ) { } + fn extent__write__raw__context__insert__start( + job_id: u64, + extent_id: u32, + extent_size: u64, + ) { + } + fn extent__write__raw__context__insert__done( + _job_id: u64, + _extent_id: u32, + extent_size: u64, + ) { + } fn extent__read__start(job_id: u64, extent_id: u32, n_blocks: u64) {} fn extent__read__done(job_id: u64, extent_id: u32, n_blocks: u64) {} fn extent__read__get__contexts__start( diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index ed0460fad..31fe736d8 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -28,25 +28,12 @@ use crate::extent::{ /** * Validate a list of sorted repair files. - * There are either two or four files we expect to find, any more or less - * and we have a bad list. No duplicates. + * + * There is only one repair file: the raw file itself (which also contains + * structured context and metadata at the end). */ pub fn validate_repair_files(eid: usize, files: &[String]) -> bool { - let eid = eid as u32; - - let some = vec![ - extent_file_name(eid, ExtentType::Data), - extent_file_name(eid, ExtentType::Db), - ]; - - let mut all = some.clone(); - all.extend(vec![ - extent_file_name(eid, ExtentType::DbShm), - extent_file_name(eid, ExtentType::DbWal), - ]); - - // Either we have some or all. - files == some || files == all + files == [extent_file_name(eid as u32, ExtentType::Data)] } /// Wrapper type for either a job or reconciliation ID @@ -99,6 +86,12 @@ pub struct Region { read_only: bool, log: Logger, + + /// Use the SQLite extent backend when creating new extents + /// + /// This is only allowed in tests; all new extents must be raw files + #[cfg(test)] + prefer_sqlite_backend: bool, } impl Region { @@ -194,6 +187,16 @@ impl Region { dir: P, options: RegionOptions, log: Logger, + ) -> Result { + let mut region = Self::create_inner(dir, options, log)?; + region.open_extents(true).await?; + Ok(region) + } + + pub fn create_inner>( + dir: P, + options: RegionOptions, + log: Logger, ) -> Result { options.validate()?; @@ -214,18 +217,29 @@ impl Region { write_json(&cp, &def, false)?; info!(log, "Created new region file {:?}", cp); - /* - * Open every extent that presently exists. - */ - let mut region = Region { + Ok(Region { dir: dir.as_ref().to_path_buf(), def, extents: Vec::new(), dirty_extents: HashSet::new(), read_only: false, log, - }; + #[cfg(test)] + prefer_sqlite_backend: false, + }) + } + /** + * Create a new SQLite-backed region based on the given RegionOptions + */ + #[cfg(test)] + pub async fn create_sqlite>( + dir: P, + options: RegionOptions, + log: Logger, + ) -> Result { + let mut region = Self::create_inner(dir, options, log)?; + region.prefer_sqlite_backend = true; region.open_extents(true).await?; Ok(region) @@ -299,6 +313,9 @@ impl Region { dirty_extents: HashSet::new(), read_only, log: log.clone(), + + #[cfg(test)] + prefer_sqlite_backend: false, }; region.open_extents(false).await?; @@ -338,6 +355,13 @@ impl Region { for eid in eid_range { let extent = if create { + #[cfg(test)] + if self.prefer_sqlite_backend { + Extent::create_sqlite(&self.dir, &self.def, eid)? + } else { + Extent::create(&self.dir, &self.def, eid)? + } + #[cfg(not(test))] Extent::create(&self.dir, &self.def, eid)? } else { let extent = Extent::open( @@ -559,9 +583,7 @@ impl Region { ); // The repair file list should always contain the extent data - // file itself, and the .db file (metadata) for that extent. - // Missing these means the repair will not succeed. - // Optionally, there could be both .db-shm and .db-wal. + // file itself, and nothing else. if !validate_repair_files(eid, &repair_files) { crucible_bail!( RepairFilesInvalid, @@ -588,58 +610,6 @@ impl Region { }; save_stream_to_file(extent_copy, repair_stream.into_inner()).await?; - // The .db file is also required to exist for any valid extent. - let extent_db = Self::create_copy_file( - copy_dir.clone(), - eid, - Some(ExtentType::Db), - )?; - let repair_stream = match repair_server - .get_extent_file(eid as u32, FileType::Db) - .await - { - Ok(rs) => rs, - Err(e) => { - crucible_bail!( - RepairRequestError, - "Failed to get extent {} db file: {:?}", - eid, - e, - ); - } - }; - save_stream_to_file(extent_db, repair_stream.into_inner()).await?; - - // These next two are optional. - for opt_file in &[ExtentType::DbShm, ExtentType::DbWal] { - let filename = extent_file_name(eid as u32, opt_file.clone()); - - if repair_files.contains(&filename) { - let extent_shm = Self::create_copy_file( - copy_dir.clone(), - eid, - Some(opt_file.clone()), - )?; - let repair_stream = match repair_server - .get_extent_file(eid as u32, opt_file.to_file_type()) - .await - { - Ok(rs) => rs, - Err(e) => { - crucible_bail!( - RepairRequestError, - "Failed to get extent {} {} file: {:?}", - eid, - opt_file, - e, - ); - } - }; - save_stream_to_file(extent_shm, repair_stream.into_inner()) - .await?; - } - } - // After we have all files: move the repair dir. info!( self.log, @@ -1223,8 +1193,8 @@ pub(crate) mod test { use crate::dump::dump_region; use crate::extent::{ - completed_dir, copy_dir, extent_path, remove_copy_cleanup_dir, - DownstairsBlockContext, + completed_dir, copy_dir, extent_path, migrate_dir, + remove_copy_cleanup_dir, DownstairsBlockContext, }; use super::*; @@ -1529,6 +1499,59 @@ pub(crate) mod test { // Make copy directory for this extent let cp = Region::create_copy_dir(&dir, 1)?; + // We are simulating the copy of files from the "source" repair + // extent by copying the files from extent zero into the copy + // directory. + let dest_name = extent_file_name(1, ExtentType::Data); + let source_path = extent_path(&dir, 0); + let mut dest_path = cp.clone(); + dest_path.push(dest_name); + std::fs::copy(source_path.clone(), dest_path.clone())?; + + let rd = replace_dir(&dir, 1); + rename(cp.clone(), rd.clone())?; + + // Now we have a replace directory, we verify that special + // action is taken when we (re)open the extent. + + // Reopen extent 1 + region.reopen_extent(1).await?; + + let _ext_one = region.get_opened_extent(1).await; + + // Make sure all repair directories are gone + assert!(!Path::new(&cp).exists()); + assert!(!Path::new(&rd).exists()); + + // The reopen should have replayed the repair, renamed, then + // deleted this directory. + let cd = completed_dir(&dir, 1); + assert!(!Path::new(&cd).exists()); + + Ok(()) + } + + #[tokio::test] + async fn reopen_extent_cleanup_replay_sqlite() -> Result<()> { + // Verify on extent open that a replacement directory will + // have it's contents replace an extents existing data and + // metadata files. + // Create the region, make three extents + let dir = tempdir()?; + let mut region = + Region::create_sqlite(&dir, new_region_options(), csl()).await?; + region.extend(3).await?; + + // Close extent 1 + region.close_extent(1).await.unwrap(); + assert!(matches!( + *region.extents[1].lock().await, + ExtentState::Closed + )); + + // Make copy directory for this extent + let cp = Region::create_copy_dir(&dir, 1)?; + // We are simulating the copy of files from the "source" repair // extent by copying the files from extent zero into the copy // directory. @@ -1596,6 +1619,63 @@ pub(crate) mod test { // Make copy directory for this extent let cp = Region::create_copy_dir(&dir, 1)?; + // We are simulating the copy of files from the "source" repair + // extent by copying the files from extent zero into the copy + // directory. + let dest_name = extent_file_name(1, ExtentType::Data); + let source_path = extent_path(&dir, 0); + let mut dest_path = cp.clone(); + dest_path.push(dest_name); + println!("cp {:?} to {:?}", source_path, dest_path); + std::fs::copy(source_path.clone(), dest_path.clone())?; + + let rd = replace_dir(&dir, 1); + rename(cp.clone(), rd.clone())?; + + // Now we have a replace directory populated and our files to + // delete are ready. We verify that special action is taken + // when we (re)open the extent. + + // Reopen extent 1 + region.reopen_extent(1).await?; + + let _ext_one = region.get_opened_extent(1).await; + + // Make sure all repair directories are gone + assert!(!Path::new(&cp).exists()); + assert!(!Path::new(&rd).exists()); + + // The reopen should have replayed the repair, renamed, then + // deleted this directory. + let cd = completed_dir(&dir, 1); + assert!(!Path::new(&cd).exists()); + + Ok(()) + } + + #[tokio::test] + async fn reopen_extent_cleanup_replay_short_sqlite() -> Result<()> { + // test move_replacement_extent(), create a copy dir, populate it + // and let the reopen do the work. This time we make sure our + // copy dir only has extent data and .db files, and not .db-shm + // nor .db-wal. Verify these files are delete from the original + // extent after the reopen has cleaned them up. + // Create the region, make three extents + let dir = tempdir()?; + let mut region = + Region::create_sqlite(&dir, new_region_options(), csl()).await?; + region.extend(3).await?; + + // Close extent 1 + region.close_extent(1).await.unwrap(); + assert!(matches!( + *region.extents[1].lock().await, + ExtentState::Closed + )); + + // Make copy directory for this extent + let cp = Region::create_copy_dir(&dir, 1)?; + // We are simulating the copy of files from the "source" repair // extent by copying the files from extent zero into the copy // directory. @@ -1656,7 +1736,7 @@ pub(crate) mod test { } #[tokio::test] - async fn reopen_extent_no_replay_readonly() -> Result<()> { + async fn reopen_extent_no_replay_readonly_sqlite() -> Result<()> { // Verify on a read-only region a replacement directory will // be ignored. This is required by the dump command, as it would // be tragic if the command to inspect a region changed that @@ -1665,7 +1745,7 @@ pub(crate) mod test { // Create the region, make three extents let dir = tempdir()?; let mut region = - Region::create(&dir, new_region_options(), csl()).await?; + Region::create_sqlite(&dir, new_region_options(), csl()).await?; region.extend(3).await?; // Make copy directory for this extent @@ -1704,6 +1784,132 @@ pub(crate) mod test { Ok(()) } + #[tokio::test] + async fn reopen_extent_no_replay_readonly() -> Result<()> { + // Verify on a read-only region a replacement directory will + // be ignored. This is required by the dump command, as it would + // be tragic if the command to inspect a region changed that + // region's contents in the act of inspecting. + + // Create the region, make three extents + let dir = tempdir()?; + let mut region = + Region::create(&dir, new_region_options(), csl()).await?; + region.extend(3).await?; + + // Make copy directory for this extent + let _ext_one = region.get_opened_extent(1).await; + let cp = Region::create_copy_dir(&dir, 1)?; + + // We are simulating the copy of files from the "source" repair + // extent by copying the files from extent zero into the copy + // directory. + let dest_name = extent_file_name(1, ExtentType::Data); + let source_path = extent_path(&dir, 0); + let mut dest_path = cp.clone(); + dest_path.push(dest_name); + std::fs::copy(source_path.clone(), dest_path.clone())?; + + let rd = replace_dir(&dir, 1); + rename(cp, rd.clone())?; + + drop(region); + + // Open up the region read_only now. + let region = + Region::open(&dir, new_region_options(), false, true, &csl()) + .await?; + + // Verify extent 1 has opened again. + let _ext_one = region.get_opened_extent(1).await; + + // Make sure repair directory is still present + assert!(Path::new(&rd).exists()); + + Ok(()) + } + + #[tokio::test] + async fn reopen_extent_partial_migration() -> Result<()> { + let log = csl(); + let dir = tempdir()?; + let mut region = + Region::create_sqlite(&dir, new_region_options(), log.clone()) + .await?; + region.extend(3).await?; + let ddef = region.def(); + + // Make a copy dir for extent 1, then manually export to a raw file + let copy_dir = Region::create_copy_dir(&dir, 1)?; + let extent_file = extent_file_name(1, ExtentType::Data); + let mut inner = extent_inner_sqlite::SqliteInner::open( + &extent_dir(&dir, 1).join(&extent_file), + &ddef, + 1, + false, + &log, + )?; + std::fs::write(copy_dir.join(&extent_file), inner.export()?)?; + let migrate_dir = migrate_dir(&dir, 1); + std::fs::rename(copy_dir, migrate_dir)?; + + // Now, we're going to mess with the file on disk a little, to make sure + // that the migration happens from the old file (which we just exported) + // and not from the modified database. + region + .region_write( + &[ + crucible_protocol::Write { + eid: 1, + offset: Block::new_512(0), + data: Bytes::from(vec![1u8; 512]), + block_context: BlockContext { + encryption_context: None, + hash: 8717892996238908351, // hash for all 1s + }, + }, + crucible_protocol::Write { + eid: 2, + offset: Block::new_512(0), + data: Bytes::from(vec![2u8; 512]), + block_context: BlockContext { + encryption_context: None, + hash: 2192425179333611943, // hash for all 2s + }, + }, + ], + JobId(0), + false, + ) + .await?; + + // Now, we reopen the region. Because there is a migration folder on + // disk for extent 1, we expect that migration to take priority over the + // current state of the database, so the write we just did should not be + // present in extent 1 (but should be present for extent 2) + let region = + Region::open(&dir, new_region_options(), true, false, &log).await?; + let out = region + .region_read( + &[ + ReadRequest { + eid: 1, + offset: Block::new_512(0), + }, + ReadRequest { + eid: 2, + offset: Block::new_512(0), + }, + ], + JobId(0), + ) + .await?; + assert_eq!(out[0].data.as_ref(), [0; 512]); + assert_eq!(out[1].data.as_ref(), [2; 512]); + + Ok(()) + } + #[test] fn validate_repair_files_empty() { // No repair files is a failure @@ -1713,21 +1919,8 @@ pub(crate) mod test { #[test] fn validate_repair_files_good() { // This is an expected list of files for an extent - let good_files: Vec = vec![ - "001".to_string(), - "001.db".to_string(), - "001.db-shm".to_string(), - "001.db-wal".to_string(), - ]; - - assert!(validate_repair_files(1, &good_files)); - } + let good_files: Vec = vec!["001".to_string()]; - #[test] - fn validate_repair_files_also_good() { - // This is also an expected list of files for an extent - let good_files: Vec = - vec!["001".to_string(), "001.db".to_string()]; assert!(validate_repair_files(1, &good_files)); } @@ -1739,67 +1932,14 @@ pub(crate) mod test { assert!(!validate_repair_files(2, &good_files)); } - #[test] - fn validate_repair_files_duplicate_pair() { - // duplicate file names for extent 2 - let good_files: Vec = vec![ - "002".to_string(), - "002".to_string(), - "002.db".to_string(), - "002.db".to_string(), - ]; - assert!(!validate_repair_files(2, &good_files)); - } - - #[test] - fn validate_repair_files_quad_duplicate() { - // This is an expected list of files for an extent - let good_files: Vec = vec![ - "001".to_string(), - "001.db".to_string(), - "001.db-shm".to_string(), - "001.db-shm".to_string(), - ]; - assert!(!validate_repair_files(1, &good_files)); - } - #[test] fn validate_repair_files_offbyon() { // Incorrect file names for extent 2 - let good_files: Vec = vec![ - "001".to_string(), - "001.db".to_string(), - "001.db-shm".to_string(), - "001.db-wal".to_string(), - ]; + let good_files: Vec = vec!["001".to_string()]; assert!(!validate_repair_files(2, &good_files)); } - #[test] - fn validate_repair_files_too_good() { - // Duplicate file in list - let good_files: Vec = vec![ - "001".to_string(), - "001".to_string(), - "001.db".to_string(), - "001.db-shm".to_string(), - "001.db-wal".to_string(), - ]; - assert!(!validate_repair_files(1, &good_files)); - } - - #[test] - fn validate_repair_files_not_good_enough() { - // We require 2 or 4 files, not 3 - let good_files: Vec = vec![ - "001".to_string(), - "001.db".to_string(), - "001.db-wal".to_string(), - ]; - assert!(!validate_repair_files(1, &good_files)); - } - #[tokio::test] async fn reopen_all_extents() -> Result<()> { // Create the region, make three extents @@ -2022,11 +2162,13 @@ pub(crate) mod test { let mut read_from_files: Vec = Vec::with_capacity(total_size); + let extent_data_size = + (ddef.extent_size().value * ddef.block_size()) as usize; for i in 0..ddef.extent_count() { let path = extent_path(&dir, i); - let mut data = std::fs::read(path).expect("Unable to read file"); + let data = std::fs::read(path).expect("Unable to read file"); - read_from_files.append(&mut data); + read_from_files.extend(&data[..extent_data_size]); } assert_eq!(buffer.len(), read_from_files.len()); @@ -2059,6 +2201,122 @@ pub(crate) mod test { Ok(()) } + #[tokio::test] + async fn test_big_write_migrate() -> Result<()> { + let log = csl(); + let dir = tempdir()?; + let mut region = + Region::create_sqlite(&dir, new_region_options(), log.clone()) + .await?; + region.extend(3).await?; + + let ddef = region.def(); + let total_size: usize = ddef.total_size() as usize; + let num_blocks: usize = + ddef.extent_size().value as usize * ddef.extent_count() as usize; + + // use region_write to fill region + + let mut rng = rand::thread_rng(); + let mut buffer: Vec = vec![0; total_size]; + rng.fill_bytes(&mut buffer); + + let mut writes: Vec = + Vec::with_capacity(num_blocks); + + for i in 0..num_blocks { + let eid: u64 = i as u64 / ddef.extent_size().value; + let offset: Block = + Block::new_512((i as u64) % ddef.extent_size().value); + + let data = BytesMut::from(&buffer[(i * 512)..((i + 1) * 512)]); + let data = data.freeze(); + let hash = integrity_hash(&[&data[..]]); + + writes.push(crucible_protocol::Write { + eid, + offset, + data, + block_context: BlockContext { + encryption_context: None, + hash, + }, + }); + } + + region.region_write(&writes, JobId(0), false).await?; + for i in 0..3 { + region.region_flush_extent(i, 10, 15, JobId(21)).await?; + } + let meta = region.get_opened_extent(0).await.get_meta_info().await; + assert_eq!(meta.gen_number, 10); + assert_eq!(meta.flush_number, 15); + drop(region); + + // Open the region as read-only, which doesn't trigger a migration + let region = + Region::open(&dir, new_region_options(), true, true, &log).await?; + let meta = region.get_opened_extent(0).await.get_meta_info().await; + assert_eq!(meta.gen_number, 10); + assert_eq!(meta.flush_number, 15); + + // Assert that the .db files still exist + for i in 0..3 { + assert!(extent_dir(&dir, i) + .join(extent_file_name(i, ExtentType::Db)) + .exists()); + } + + // read all using region_read + let mut requests: Vec = + Vec::with_capacity(num_blocks); + + for i in 0..num_blocks { + let eid: u64 = i as u64 / ddef.extent_size().value; + let offset: Block = + Block::new_512((i as u64) % ddef.extent_size().value); + + requests.push(crucible_protocol::ReadRequest { eid, offset }); + } + + let read_from_region: Vec = region + .region_read(&requests, JobId(0)) + .await? + .into_iter() + .flat_map(|i| i.data.to_vec()) + .collect(); + + assert_eq!(buffer.len(), read_from_region.len()); + assert_eq!(buffer, read_from_region); + drop(region); + + // Open the region as read-write, which **does** trigger a migration + let region = + Region::open(&dir, new_region_options(), true, false, &log).await?; + let meta = region.get_opened_extent(0).await.get_meta_info().await; + assert_eq!(meta.gen_number, 10); + assert_eq!(meta.flush_number, 15); + + // Assert that the .db files have been deleted during the migration + for i in 0..3 { + assert!(!extent_dir(&dir, i) + .join(extent_file_name(i, ExtentType::Db)) + .exists()); + } + let read_from_region: Vec = region + .region_read(&requests, JobId(0)) + .await? + .into_iter() + .flat_map(|i| i.data.to_vec()) + .collect(); + + assert_eq!(buffer.len(), read_from_region.len()); + assert_eq!(buffer, read_from_region); + drop(region); + + Ok(()) + } + #[tokio::test] async fn test_region_open_removes_partial_writes() -> Result<()> { // Opening a dirty extent should fully rehash the extent to remove any @@ -2102,7 +2360,7 @@ pub(crate) mod test { // Verify no block context rows exist { let ext = region.get_opened_extent(0).await; - let inner = ext.lock().await; + let mut inner = ext.lock().await; assert!(inner.get_block_contexts(0, 1)?[0].is_empty()); } @@ -2449,11 +2707,13 @@ pub(crate) mod test { // read data into File, compare what was written to buffer let mut read_from_files: Vec = Vec::with_capacity(total_size); + let extent_data_size = + (ddef.extent_size().value * ddef.block_size()) as usize; for i in 0..ddef.extent_count() { let path = extent_path(&dir, i); - let mut data = std::fs::read(path).expect("Unable to read file"); + let data = std::fs::read(path).expect("Unable to read file"); - read_from_files.append(&mut data); + read_from_files.extend(&data[..extent_data_size]); } assert_eq!(buffer, read_from_files); @@ -2567,11 +2827,13 @@ pub(crate) mod test { // read data into File, compare what was written to buffer let mut read_from_files: Vec = Vec::with_capacity(total_size); + let extent_data_size = + (ddef.extent_size().value * ddef.block_size()) as usize; for i in 0..ddef.extent_count() { let path = extent_path(&dir, i); - let mut data = std::fs::read(path).expect("Unable to read file"); + let data = std::fs::read(path).expect("Unable to read file"); - read_from_files.append(&mut data); + read_from_files.extend(&data[..extent_data_size]); } assert_eq!(buffer, read_from_files); @@ -2686,11 +2948,13 @@ pub(crate) mod test { // read data into File, compare what was written to buffer let mut read_from_files: Vec = Vec::with_capacity(total_size); + let extent_data_size = + (ddef.extent_size().value * ddef.block_size()) as usize; for i in 0..ddef.extent_count() { let path = extent_path(&dir, i); - let mut data = std::fs::read(path).expect("Unable to read file"); + let data = std::fs::read(path).expect("Unable to read file"); - read_from_files.append(&mut data); + read_from_files.extend(&data[..extent_data_size]); } assert_eq!(buffer, read_from_files); @@ -3304,7 +3568,7 @@ pub(crate) mod test { let last_writes = writes.last().unwrap(); let ext = region.get_opened_extent(0).await; - let inner = ext.lock().await; + let mut inner = ext.lock().await; for (i, range) in ranges.iter().enumerate() { // Get the contexts for the range @@ -3386,7 +3650,7 @@ pub(crate) mod test { let last_writes = writes.last().unwrap(); let ext = region.get_opened_extent(0).await; - let inner = ext.lock().await; + let mut inner = ext.lock().await; // Get the contexts for the range let ctxts = inner.get_block_contexts(0, EXTENT_SIZE)?; diff --git a/downstairs/src/repair.rs b/downstairs/src/repair.rs index 4b47c8953..3130fff33 100644 --- a/downstairs/src/repair.rs +++ b/downstairs/src/repair.rs @@ -226,12 +226,7 @@ fn extent_file_list( eid: u32, ) -> Result, HttpError> { let mut files = Vec::new(); - let possible_files = vec![ - (extent_file_name(eid, ExtentType::Data), true), - (extent_file_name(eid, ExtentType::Db), true), - (extent_file_name(eid, ExtentType::DbShm), false), - (extent_file_name(eid, ExtentType::DbWal), false), - ]; + let possible_files = vec![(extent_file_name(eid, ExtentType::Data), true)]; for (file, required) in possible_files.into_iter() { let mut fullname = extent_dir.clone(); @@ -282,7 +277,7 @@ mod test { let ed = extent_dir(&dir, 1); let mut ex_files = extent_file_list(ed, 1).unwrap(); ex_files.sort(); - let expected = vec!["001", "001.db", "001.db-shm", "001.db-wal"]; + let expected = vec!["001"]; println!("files: {:?}", ex_files); assert_eq!(ex_files, expected); @@ -290,42 +285,11 @@ mod test { } #[tokio::test] - async fn extent_expected_files_short() -> Result<()> { + async fn extent_expected_files_with_close() -> Result<()> { // Verify that the list of files returned for an extent matches // what we expect. In this case we expect the extent data file and - // the .db file, but not the .db-shm or .db-wal database files. - let dir = tempdir()?; - let mut region = - Region::create(&dir, new_region_options(), csl()).await?; - region.extend(3).await?; - - // Determine the directory and name for expected extent files. - let extent_dir = extent_dir(&dir, 1); - - // Delete db-wal and db-shm - let mut rm_file = extent_dir.clone(); - rm_file.push(extent_file_name(1, ExtentType::Data)); - rm_file.set_extension("db-wal"); - std::fs::remove_file(&rm_file).unwrap(); - rm_file.set_extension("db-shm"); - std::fs::remove_file(rm_file).unwrap(); - - let mut ex_files = extent_file_list(extent_dir, 1).unwrap(); - ex_files.sort(); - let expected = vec!["001", "001.db"]; - println!("files: {:?}", ex_files); - assert_eq!(ex_files, expected); - - Ok(()) - } - - #[tokio::test] - async fn extent_expected_files_short_with_close() -> Result<()> { - // Verify that the list of files returned for an extent matches - // what we expect. In this case we expect the extent data file and - // the .db file, but not the .db-shm or .db-wal database files. - // We close the extent here first, and on illumos that behaves - // a little different than elsewhere. + // nothing else. We close the extent here first, and on illumos that + // behaves a little different than elsewhere. let dir = tempdir()?; let mut region = Region::create(&dir, new_region_options(), csl()).await?; @@ -336,18 +300,9 @@ mod test { // Determine the directory and name for expected extent files. let extent_dir = extent_dir(&dir, 1); - // Delete db-wal and db-shm. On illumos the close of the extent - // may remove these for us, so we ignore errors on the removal. - let mut rm_file = extent_dir.clone(); - rm_file.push(extent_file_name(1, ExtentType::Data)); - rm_file.set_extension("db-wal"); - let _ = std::fs::remove_file(&rm_file); - rm_file.set_extension("db-shm"); - let _ = std::fs::remove_file(rm_file); - let mut ex_files = extent_file_list(extent_dir, 1).unwrap(); ex_files.sort(); - let expected = vec!["001", "001.db"]; + let expected = vec!["001"]; println!("files: {:?}", ex_files); assert_eq!(ex_files, expected); @@ -356,29 +311,6 @@ mod test { #[tokio::test] async fn extent_expected_files_fail() -> Result<()> { - // Verify that we get an error if the expected extent.db file - // is missing. - let dir = tempdir()?; - let mut region = - Region::create(&dir, new_region_options(), csl()).await?; - region.extend(3).await?; - - // Determine the directory and name for expected extent files. - let extent_dir = extent_dir(&dir, 2); - - // Delete db - let mut rm_file = extent_dir.clone(); - rm_file.push(extent_file_name(2, ExtentType::Data)); - rm_file.set_extension("db"); - std::fs::remove_file(&rm_file).unwrap(); - - assert!(extent_file_list(extent_dir, 2).is_err()); - - Ok(()) - } - - #[tokio::test] - async fn extent_expected_files_fail_two() -> Result<()> { // Verify that we get an error if the expected extent file // is missing. let dir = tempdir()?; @@ -389,7 +321,7 @@ mod test { // Determine the directory and name for expected extent files. let extent_dir = extent_dir(&dir, 1); - // Delete db + // Delete the extent file let mut rm_file = extent_dir.clone(); rm_file.push(extent_file_name(1, ExtentType::Data)); std::fs::remove_file(&rm_file).unwrap(); diff --git a/protocol/src/lib.rs b/protocol/src/lib.rs index 7d0223ac9..da910679d 100644 --- a/protocol/src/lib.rs +++ b/protocol/src/lib.rs @@ -157,7 +157,7 @@ impl ReadResponse { } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)] +#[derive(Copy, Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct BlockContext { /// If this is a non-encrypted write, then the integrity hasher has the /// data as an input: @@ -183,7 +183,7 @@ pub struct BlockContext { } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)] +#[derive(Copy, Clone, Debug, PartialEq, Serialize, Deserialize)] pub struct EncryptionContext { pub nonce: [u8; 12], pub tag: [u8; 16], From b4a6f6876ff295c7f76c26867a8fca94ee8afac2 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 6 Oct 2023 14:42:39 -0400 Subject: [PATCH 02/62] Update comments --- downstairs/src/extent.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index 310001009..386d872cd 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -252,6 +252,7 @@ pub fn completed_dir>(dir: P, number: u32) -> PathBuf { .with_extension("completed") } +/// Produce a `PathBuf` for the SQLite-to-raw migration directory pub fn migrate_dir>(dir: P, number: u32) -> PathBuf { extent_dir(dir, number) .join(extent_file_name(number, ExtentType::Data)) @@ -369,9 +370,9 @@ impl Extent { has_sqlite = false; } - // Pick the format for the downstairs files - // - // Right now, this only supports SQLite-flavored Downstairs + // Pick the format for the downstairs files. In most cases, we will be + // using the raw extent format, but for read-only snapshots, we're stuck + // with the SQLite backend. let inner: Box = { if has_sqlite { let inner = extent_inner_sqlite::SqliteInner::open( From a3bc2da64fbaaba41afc8a24a2326ec34ca837c6 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 6 Oct 2023 15:04:49 -0400 Subject: [PATCH 03/62] Bump version --- protocol/src/lib.rs | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/protocol/src/lib.rs b/protocol/src/lib.rs index da910679d..9c559b810 100644 --- a/protocol/src/lib.rs +++ b/protocol/src/lib.rs @@ -241,6 +241,12 @@ pub struct SnapshotDetails { #[repr(u32)] #[derive(IntoPrimitive)] pub enum MessageVersion { + /// Switched to raw file extents + /// + /// The message format remains the same, but live repair between SQLite and + /// raw file extents is not possible. + V5 = 5, + /// Added ErrorReport V4 = 4, @@ -255,7 +261,7 @@ pub enum MessageVersion { } impl MessageVersion { pub const fn current() -> Self { - Self::V4 + Self::V5 } } @@ -264,7 +270,7 @@ impl MessageVersion { * This, along with the MessageVersion enum above should be updated whenever * changes are made to the Message enum below. */ -pub const CRUCIBLE_MESSAGE_VERSION: u32 = 4; +pub const CRUCIBLE_MESSAGE_VERSION: u32 = 5; /* * If you add or change the Message enum, you must also increment the From a7461af13213ef0719e637b68e856dcd74d5899c Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 9 Oct 2023 07:42:11 -0700 Subject: [PATCH 04/62] Implement fast migration --- downstairs/src/extent.rs | 239 +++++++++++--------------- downstairs/src/extent_inner_sqlite.rs | 19 +- downstairs/src/region.rs | 46 ++--- 3 files changed, 137 insertions(+), 167 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index 386d872cd..c1a14709e 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -2,10 +2,10 @@ use std::convert::TryInto; use std::fmt; use std::fmt::Debug; -use std::fs::File; +use std::fs::{File, OpenOptions}; use tokio::sync::Mutex; -use anyhow::{anyhow, bail, Result}; +use anyhow::{anyhow, bail, Context, Result}; use nix::unistd::{sysconf, SysconfVar}; use serde::{Deserialize, Serialize}; @@ -252,17 +252,9 @@ pub fn completed_dir>(dir: P, number: u32) -> PathBuf { .with_extension("completed") } -/// Produce a `PathBuf` for the SQLite-to-raw migration directory -pub fn migrate_dir>(dir: P, number: u32) -> PathBuf { - extent_dir(dir, number) - .join(extent_file_name(number, ExtentType::Data)) - .with_extension("migrate") -} - /** - * Remove temporary directories associated with repair and migration, except for - * the final .replace or .migrate directory. Replace and migration are handled - * specifically during extent open. + * Remove directories associated with repair except for the replace + * directory. Replace is handled specifically during extent open. */ pub fn remove_copy_cleanup_dir>(dir: P, eid: u32) -> Result<()> { for f in [copy_dir, completed_dir] { @@ -301,73 +293,112 @@ impl Extent { // If there was an unfinished copy, then clean it up here (??) remove_copy_cleanup_dir(dir, number)?; - // There are two possible migrations that could be in progress: - // - // - Extent replacement, which is done from a `.replace` directory and - // calls `move_replacement_extent`; this is part of live repair. - // - Extent migration (from SQLite to raw files), which is done from a - // `.migrate` directory and `move_migrate_extent` + // If the replace directory exists for this extent, then it means + // a repair was interrupted before it could finish. We will continue + // the repair before we open the extent. // - // Both of these are supposed to complete fully, but it's possible that - // a repair / migration was interrupted before it could finish. If the - // above directories are present, then we are **ready** for repair / - // migration, so we will continue it before opening the extent. - // - // It is not possible for a repair and migration to both be staged; - // migration happens later in this function, so the repair will be - // completed (right here). - if !read_only { - let has_replace = replace_dir(dir, number).exists(); - let has_migrate = migrate_dir(dir, number).exists(); - if has_replace && has_migrate { - bail!( - "cannot have both .replace and .migrate dirs simultaneously" - ); - } else if has_replace { - info!( - log, - "Extent {} found replacement dir, finishing replacement", - number - ); - move_replacement_extent(dir, number as usize, log)?; - } else if has_migrate { - info!( - log, - "Extent {} found migrate dir, finishing migration", number - ); - move_migrate_extent(dir, number as usize, log)?; - } + // Note that `move_replacement_extent` must support migrating old + // (SQLite-based) extents and new (raw file) extents, because it's + // possible that an old extent replacement crashed right before we + // updated Crucible. + let replace_dir = replace_dir(dir, number); + if !read_only && Path::new(&replace_dir).exists() { + info!( + log, + "Extent {} found replacement dir, finishing replacement", + number + ); + move_replacement_extent(dir, number as usize, log)?; } - // Speaking of migration, it's happening now! We will migrate - // every read-write extent with a SQLite file present. + // We will migrate every read-write extent with a SQLite file present. + // + // We use the presence of the .db file as a marker to whether the extent + // data file has been migrated. + // + // Remember, the goal is to turn 2-4 files (extent data, .db, .db-wal, + // db-shm) into a single file containing + // + // - The extent data (same as before, at the beginning of the file) + // - Metadata and encryption context slots (after extent data in the + // - same file) + // + // Here's the procedure: + // + // - If the .db file is present + // - Truncate it to the length of the extent data. This is a no-op + // normally, but lets us recover from a partial migration (e.g. + // if we wrote the raw context but crashed before deleting the + // .db file) + // - Open the extent using our existing SQLite extent code + // - Using standard extent APIs, find the metadata and encryption + // context for each block. Encode this in the new raw file format + // (as a Vec) + // - Close the (SQLite) extent + // - Open the extent data file in append mode, and append the new + // raw metadata + block contexts to the end of the file. + // - Close the extent data file and fsync it to disk + // - Delete the .db file + // - fsync the containing directory (to make the deletion durable; + // not sure if this is necessary) + // - At this point, the .db file is not present and the extent data + // file represents a raw extent that has been persisted to disk + // - If the .db-wal file is present, remove it + // - If the .db-shm file is present, remove it + // + // It's safe to fail at any point in this procedure; the next time + // `Extent::open` is called, we will restart the migration (if we + // haven't gotten to deleting the `.db` file). let mut has_sqlite = path.with_extension("db").exists(); if has_sqlite && !read_only { - let mut inner = extent_inner_sqlite::SqliteInner::open( - &path, def, number, read_only, log, - )?; - let data = inner.export()?; - - // We'll put the new file into copy_dir first, then rename it to - // migrate_dir once it's ready. The logic here matches the - // docstring in `Region::repair_extent` - let copy_dir = copy_dir(dir, number); - let extent_file_name = extent_file_name(number, ExtentType::Data); - let new_file = copy_dir.join(extent_file_name); - mkdir_for_file(&new_file)?; - - info!(log, "Extent {number} writing migration to {new_file:?}"); - std::fs::write(&new_file, data)?; - sync_path(&new_file, log)?; - - // Rename from .copy to .migrate, then sync the parent directory - info!(log, "Extent {number} renaming migration dir"); - std::fs::rename(copy_dir, migrate_dir(dir, number))?; + info!(log, "Migrating extent {number}"); + // Truncate the file to the length of the extent data + { + let f = + OpenOptions::new().read(true).write(true).open(&path)?; + f.set_len(def.extent_size().value * def.block_size())?; + } + // Compute metadata and context slots + let meta_and_context = { + let mut inner = extent_inner_sqlite::SqliteInner::open( + &path, def, number, read_only, log, + )?; + inner.export_meta_and_context()? + }; + // Append the new raw data, then sync the file to disk + { + let mut f = OpenOptions::new() + .read(true) + .write(true) + .append(true) + .open(&path)?; + f.write_all(&meta_and_context)?; + f.sync_all() + .with_context(|| format!("{path:?}: fsync failure"))?; + } + + // Remove the .db file, because our extent is now a raw extent and + // has been persisted to disk. + std::fs::remove_file(path.with_extension("db"))?; + + // Make that removal persistent by synching the parent directory sync_path(extent_dir(dir, number), log)?; - info!(log, "Extent {number} copying the migration over"); - move_migrate_extent(dir, number as usize, log)?; + // We have now migrated from SQLite to raw file extents! has_sqlite = false; + info!(log, "Done migrating extent {number}"); + } + + // Clean up spare SQLite files if this is a raw file extent. These + // deletions don't need to be durable, because we're not using their + // presence / absence for anything meaningful. + if !has_sqlite && !read_only { + for ext in ["db-shm", "db-wal"] { + let f = path.with_extension(ext); + if f.exists() { + std::fs::remove_file(f)?; + } + } } // Pick the format for the downstairs files. In most cases, we will be @@ -581,72 +612,6 @@ impl Extent { } } -/// Apply a staged migration directory to the extent file -/// -/// Migration is only supported from the SQLite extent backend to the raw extent -/// backend. It consists of copying the new (raw) extent file, then deleting -/// the SQLite files. -pub(crate) fn move_migrate_extent>( - region_dir: P, - eid: usize, - log: &Logger, -) -> Result<(), CrucibleError> { - let destination_dir = extent_dir(®ion_dir, eid as u32); - let extent_file_name = extent_file_name(eid as u32, ExtentType::Data); - let migrate_dir = migrate_dir(®ion_dir, eid as u32); - let completed_dir = completed_dir(®ion_dir, eid as u32); - - assert!(Path::new(&migrate_dir).exists()); - assert!(!Path::new(&completed_dir).exists()); - - info!( - log, - "Copy files from {migrate_dir:?} in {destination_dir:?}" - ); - - // Setup the original and replacement file names. - let new_file = migrate_dir.join(&extent_file_name); - let original_file = destination_dir.join(&extent_file_name); - - // Copy the new file on top of the original file. - if let Err(e) = std::fs::copy(new_file.clone(), original_file.clone()) { - crucible_bail!( - IoError, - "copy of {new_file:?} to {original_file:?} got: {e:?}", - ); - } - sync_path(&original_file, log)?; - - // Remove all of the old files. It is always valid for an old file to _not_ - // be present, for two reasons: - // - the .db-shm and .db-wal files are always optional - // - the .db file may have already been removed by a previous call to - // `move_migrate_extent`, if a later step failed - for ext in ["db", "db-shm", "db-wal"] { - let original_file = original_file.with_extension(ext); - if original_file.exists() { - info!( - log, - "Remove old file {original_file:?} as there is no replacement", - ); - std::fs::remove_file(&original_file)?; - } - } - sync_path(&destination_dir, log)?; - - // After we have all files: rename the migrate dir to indicate that - // migration is complete. - info!(log, "Move directory {migrate_dir:?} to {completed_dir:?}"); - std::fs::rename(migrate_dir, &completed_dir)?; - - sync_path(&destination_dir, log)?; - - std::fs::remove_dir_all(&completed_dir)?; - - sync_path(&destination_dir, log)?; - Ok(()) -} - /** * Copy the contents of the replacement directory on to the extent * files in the extent directory. diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 255b1257d..c64b26b7a 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -506,19 +506,18 @@ impl ExtentInner for SqliteInner { impl SqliteInner { /// Converts to a raw file for use with `RawInner` - pub fn export(&mut self) -> Result, CrucibleError> { + /// + /// Returns the metadata and context slots, which should be positioned + /// directly after the raw block data in memory. + pub fn export_meta_and_context( + &mut self, + ) -> Result, CrucibleError> { // Clean up stale hashes. After this is done, each block should have // either 0 or 1 contexts. self.fully_rehash_and_clean_all_stale_contexts(true)?; - let block_size = self.extent_size.block_size_in_bytes() as usize; let ctxs = self.get_block_contexts(0, self.extent_size.value)?; - // Read file contents, which are the beginning of the raw file - self.file.seek(SeekFrom::Start(0))?; - let mut buf = vec![0u8; self.extent_size.value as usize * block_size]; - self.file.read_exact(&mut buf)?; - use crate::{ extent::EXTENT_META_RAW, extent_inner_raw::{ @@ -527,7 +526,7 @@ impl SqliteInner { }, }; - // Record the metadata region after the raw block data + // Record the metadata region, which will be right after raw block data let dirty = self.dirty()?; let flush_number = self.flush_number()?; let gen_number = self.gen_number()?; @@ -535,11 +534,13 @@ impl SqliteInner { dirty, flush_number, gen_number, - ext_version: EXTENT_META_RAW, + ext_version: EXTENT_META_RAW, // new extent version for raw files }; let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; bincode::serialize_into(meta_buf.as_mut_slice(), &meta) .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + let mut buf = vec![]; buf.extend(meta_buf); // Put the context data after the metadata diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 31fe736d8..dba146109 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1193,8 +1193,8 @@ pub(crate) mod test { use crate::dump::dump_region; use crate::extent::{ - completed_dir, copy_dir, extent_path, migrate_dir, - remove_copy_cleanup_dir, DownstairsBlockContext, + completed_dir, copy_dir, extent_path, remove_copy_cleanup_dir, + DownstairsBlockContext, }; use super::*; @@ -1839,20 +1839,6 @@ pub(crate) mod test { region.extend(3).await?; let ddef = region.def(); - // Make a copy dir for extent 1, then manually export to a raw file - let copy_dir = Region::create_copy_dir(&dir, 1)?; - let extent_file = extent_file_name(1, ExtentType::Data); - let mut inner = extent_inner_sqlite::SqliteInner::open( - &extent_dir(&dir, 1).join(&extent_file), - &ddef, - 1, - false, - &log, - )?; - std::fs::write(copy_dir.join(&extent_file), inner.export()?)?; - let migrate_dir = migrate_dir(&dir, 1); - std::fs::rename(copy_dir, migrate_dir)?; - // Now, we're going to mess with the file on disk a little, to make sure // that the migration happens from the old file (which we just exported) // and not from the modified database. @@ -1882,11 +1868,29 @@ pub(crate) mod test { false, ) .await?; + drop(region); + + // Calculate the migration from extent 1 + let extent_file = extent_path(&dir, 1); + let exported = { + let mut inner = extent_inner_sqlite::SqliteInner::open( + &extent_file, + &ddef, + 1, + false, + &log, + )?; + inner.export_meta_and_context()? + }; + + { + let mut f = OpenOptions::new() + .write(true) + .append(true) + .open(&extent_file)?; + f.write_all(&exported)?; + } - // Now, we reopen the region. Because there is a migration folder on - // disk for extent 1, we expect that migration to take priority over the - // current state of the database, so the write we just did should not be - // present in extent 1 (but should be present for extent 2) let region = Region::open(&dir, new_region_options(), true, false, &log).await?; let out = region @@ -1904,7 +1908,7 @@ pub(crate) mod test { JobId(0), ) .await?; - assert_eq!(out[0].data.as_ref(), [0; 512]); + assert_eq!(out[0].data.as_ref(), [1; 512]); assert_eq!(out[1].data.as_ref(), [2; 512]); Ok(()) From 3ae42f4f21168e6265477806d9f1e15878b96ee4 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 9 Oct 2023 08:33:38 -0700 Subject: [PATCH 05/62] Further optimization --- downstairs/src/extent_inner_sqlite.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index c64b26b7a..cd56bfb6a 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -512,9 +512,17 @@ impl SqliteInner { pub fn export_meta_and_context( &mut self, ) -> Result, CrucibleError> { - // Clean up stale hashes. After this is done, each block should have - // either 0 or 1 contexts. - self.fully_rehash_and_clean_all_stale_contexts(true)?; + // Check whether we need to rehash. This is theoretically represented + // by the `dirty` bit, but we're being _somewhat_ paranoid and manually + // forcing a rehash if any blocks have multiple contexts stored. + let ctxs = self.get_block_contexts(0, self.extent_size.value)?; + let needs_rehash = ctxs.iter().any(|c| c.len() > 1); + + if needs_rehash { + // Clean up stale hashes. After this is done, each block should + // have either 0 or 1 contexts. + self.fully_rehash_and_clean_all_stale_contexts(true)?; + } let ctxs = self.get_block_contexts(0, self.extent_size.value)?; From 8a71b115616b3945ff188f006fe234d8f66adcd6 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 9 Oct 2023 14:12:20 -0700 Subject: [PATCH 06/62] Fix ordering --- downstairs/src/extent_inner_raw.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 334ac3a0a..5dc901add 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -55,12 +55,12 @@ pub const BLOCK_META_SIZE_BYTES: u64 = 64; /// /// The file is structured as follows: /// - Block data, structured as `block_size` × `extent_size` -/// - Block contexts (for encryption). Each block index (in the range -/// `0..extent_size`) has two context slots; we use a ping-pong strategy when -/// writing to ensure that one slot is always valid. Each slot is /// - [`BLOCK_META_SIZE_BYTES`], which contains an [`OnDiskMeta`] serialized /// using `bincode`. The first byte of this range is `dirty`, serialized as a /// `u8` (where `1` is dirty and `0` is clean). +/// - Block contexts (for encryption). Each block index (in the range +/// `0..extent_size`) has two context slots; we use a ping-pong strategy when +/// writing to ensure that one slot is always valid. Each slot is /// [`BLOCK_CONTEXT_SLOT_SIZE_BYTES`] in size, so this region is /// `BLOCK_CONTEXT_SLOT_SIZE_BYTES * extent_size * 2` bytes in total. The /// slots contain an `Option`, serialized using @@ -510,8 +510,8 @@ impl RawInner { let path = extent_path(dir, extent_number); let bcount = def.extent_size().value; let size = def.block_size().checked_mul(bcount).unwrap() - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2 - + BLOCK_META_SIZE_BYTES; + + BLOCK_META_SIZE_BYTES + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2; mkdir_for_file(&path)?; let mut file = OpenOptions::new() From 1e02809dfdb0ff457c3f940be2b036c96bdb5612 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 9 Oct 2023 14:58:51 -0700 Subject: [PATCH 07/62] Preallocate during migration --- downstairs/src/extent_inner_sqlite.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index cd56bfb6a..572d18cd5 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -548,7 +548,10 @@ impl SqliteInner { bincode::serialize_into(meta_buf.as_mut_slice(), &meta) .map_err(|e| CrucibleError::IoError(e.to_string()))?; - let mut buf = vec![]; + let mut buf = Vec::with_capacity( + BLOCK_META_SIZE_BYTES as usize + + ctxs.len() * 2 * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize, + ); buf.extend(meta_buf); // Put the context data after the metadata From e247dfb0b6300e9f64819f0dc80e3a3b3b07cfda Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 9 Oct 2023 15:37:09 -0700 Subject: [PATCH 08/62] Better definition of EXTENT_META_SQLITE --- downstairs/src/extent.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index c1a14709e..08acf184c 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -125,7 +125,7 @@ pub struct ExtentMeta { /// This is no longer used when creating new extents, but we support opening /// existing SQLite-based extents because snapshot images are on read-only /// volumes, so we can't migrate them. -#[allow(dead_code)] // used in unit test +#[cfg(test)] // This constant is only actually used in unit tests pub const EXTENT_META_SQLITE: u32 = 1; /// Extent version for raw-file-backed metadata From 0f5ddca73d8d71e24c23b8874e576a0ce4068627 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 12 Oct 2023 09:23:41 -0700 Subject: [PATCH 09/62] Updates per review --- downstairs/src/extent_inner_sqlite.rs | 2 +- downstairs/src/region.rs | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 572d18cd5..a09e526a0 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -534,7 +534,7 @@ impl SqliteInner { }, }; - // Record the metadata region, which will be right after raw block data + // Record the metadata section, which will be right after raw block data let dirty = self.dirty()?; let flush_number = self.flush_number()?; let gen_number = self.gen_number()?; diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index dba146109..e6d02084a 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1600,9 +1600,8 @@ pub(crate) mod test { async fn reopen_extent_cleanup_replay_short() -> Result<()> { // test move_replacement_extent(), create a copy dir, populate it // and let the reopen do the work. This time we make sure our - // copy dir only has extent data and .db files, and not .db-shm - // nor .db-wal. Verify these files are delete from the original - // extent after the reopen has cleaned them up. + // copy dir only has the extent data file. + // Create the region, make three extents let dir = tempdir()?; let mut region = @@ -1944,6 +1943,15 @@ pub(crate) mod test { assert!(!validate_repair_files(2, &good_files)); } + #[test] + fn validate_repair_files_old() { + // Old extent files + let good_files: Vec = + vec!["001".to_string(), "001.db".to_string()]; + + assert!(!validate_repair_files(1, &good_files)); + } + #[tokio::test] async fn reopen_all_extents() -> Result<()> { // Create the region, make three extents From 837ec441509143dfe3dd0f1311b516008da5be7c Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 12 Oct 2023 09:24:57 -0700 Subject: [PATCH 10/62] Rephrase --- downstairs/src/extent.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index 08acf184c..f65144a1e 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -655,7 +655,7 @@ pub(crate) fn move_replacement_extent>( sync_path(&original_file, log)?; // We distinguish between SQLite-backend and raw-file extents based on the - // presence of the `000.db` file. We should never do live migration across + // presence of the `.db` file. We should never do live migration across // different extent formats; in fact, we should never live-migrate // SQLite-backed extents at all, but must still handle the case of // unfinished migrations. From 01dc06bcbb653095698364ce3f65239179d33ece Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 12 Oct 2023 09:26:35 -0700 Subject: [PATCH 11/62] Remove outdated comment --- downstairs/src/extent.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index f65144a1e..3f0dc8665 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -275,7 +275,6 @@ impl Extent { /** * Open an existing extent file at the location requested. - * Read in the metadata from the first block of the file. */ pub fn open( dir: &Path, From 5400ae82772660691572ba209ec1388a152d2c1d Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 12 Oct 2023 09:27:59 -0700 Subject: [PATCH 12/62] Clarify read-only snapshot situation --- downstairs/src/extent.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index 3f0dc8665..a8d80781a 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -401,10 +401,12 @@ impl Extent { } // Pick the format for the downstairs files. In most cases, we will be - // using the raw extent format, but for read-only snapshots, we're stuck - // with the SQLite backend. + // using the raw extent format, but for older read-only snapshots that + // were constructed using the SQLite backend, we have to keep them + // as-is. let inner: Box = { if has_sqlite { + assert!(read_only); let inner = extent_inner_sqlite::SqliteInner::open( &path, def, number, read_only, log, )?; From b52ca74010b430029b94de1cebdeaaebc91d6b2a Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 12 Oct 2023 09:31:56 -0700 Subject: [PATCH 13/62] extent-read-done probe is elsewhere --- downstairs/src/extent_inner_raw.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 5dc901add..0d605fcfb 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -424,10 +424,6 @@ impl ExtentInner for RawInner { req_run_start += n_contiguous_requests; } - cdt::extent__read__done!(|| { - (job_id.0, self.extent_number, requests.len() as u64) - }); - Ok(()) } From cb9a772ed1362f462100e943a9137d2b94cf1147 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 15:33:31 -0400 Subject: [PATCH 14/62] Make active_context always valid (instead of optional) --- downstairs/src/extent_inner_raw.rs | 290 ++++++++++++++++++----------- 1 file changed, 178 insertions(+), 112 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 0d605fcfb..e96350520 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -16,7 +16,7 @@ use slog::{error, Logger}; use std::collections::HashSet; use std::fs::{File, OpenOptions}; -use std::io::{IoSliceMut, Read, Seek, SeekFrom}; +use std::io::{BufReader, IoSliceMut, Read, Seek, SeekFrom}; use std::os::fd::AsRawFd; use std::path::Path; @@ -75,8 +75,8 @@ pub struct RawInner { /// Extent size, in blocks extent_size: Block, - /// Is the `ping` or `pong` context slot active? - active_context: Vec, + /// Is the `A` or `B` context slot active, on a per-block basis? + active_context: Vec, /// Local cache for the `dirty` value /// @@ -98,18 +98,19 @@ pub struct RawInner { } #[derive(Copy, Clone, Debug)] -enum ActiveContext { - /// There is no active context for the given block - Empty, - - /// The active context for the given block is unknown - /// - /// We set the context to this state before a write, so that if the write - /// fails, we rehash the file contents to get back to a known state. - Unknown, +enum ContextSlot { + A, + B, +} - /// The active context is stored in the given slot - Slot(bool), +impl std::ops::Not for ContextSlot { + type Output = Self; + fn not(self) -> Self { + match self { + ContextSlot::A => ContextSlot::B, + ContextSlot::B => ContextSlot::A, + } + } } impl ExtentInner for RawInner { @@ -251,12 +252,6 @@ impl ExtentInner for RawInner { on_disk_hash, })?; pending_slots.push(Some(next_slot)); - - // Until the write lands, we can't trust the context slot and must - // rehash it if requested. This assignment is overwritten after the - // write finishes. - self.active_context[write.offset.value as usize] = - ActiveContext::Unknown; } cdt::extent__write__raw__context__insert__done!(|| { @@ -284,36 +279,24 @@ impl ExtentInner for RawInner { (job_id.0, self.extent_number, writes.len() as u64) }); - // Now, batch writes into iovecs and use pwritev to write them all out. - let mut batched_pwritev = BatchedPwritev::new( - self.file.as_raw_fd(), - writes.len(), - self.extent_size.block_size_in_bytes().into(), - iov_max, - ); - - for write in writes { - let block = write.offset.value; + let r = self.write_inner(writes, &writes_to_skip, iov_max); - // TODO, can this be `only_write_unwritten && - // write_to_skip.contains()`? - if writes_to_skip.contains(&block) { - debug_assert!(only_write_unwritten); - batched_pwritev.perform_writes()?; - continue; + if r.is_err() { + for write in writes.iter() { + let block = write.offset.value; + if !writes_to_skip.contains(&block) { + // Try to recompute the context slot from the file. If this + // fails, then we _really_ can't recover, so bail out + // unceremoniously. + self.recompute_slot_from_file(block).unwrap(); + } } - - batched_pwritev.add_write(write)?; - } - - // Write any remaining data - batched_pwritev.perform_writes()?; - - // Now that writes have gone through, update our active context slots - for (write, new_slot) in writes.iter().zip(pending_slots) { - if let Some(slot) = new_slot { - self.active_context[write.offset.value as usize] = - ActiveContext::Slot(slot); + } else { + // Now that writes have gone through, update active context slots + for (write, new_slot) in writes.iter().zip(pending_slots) { + if let Some(slot) = new_slot { + self.active_context[write.offset.value as usize] = slot; + } } } @@ -481,8 +464,7 @@ impl ExtentInner for RawInner { ) -> Result<(), CrucibleError> { self.set_dirty()?; let new_slot = self.set_block_context(block_context)?; - self.active_context[block_context.block as usize] = - ActiveContext::Slot(new_slot); + self.active_context[block_context.block as usize] = new_slot; Ok(()) } @@ -525,7 +507,7 @@ impl RawInner { extent_size: def.extent_size(), extent_number, active_context: vec![ - ActiveContext::Empty; + ContextSlot::A; // both slots are empty, so this is fine def.extent_size().value as usize ], context_slot_synched_at: vec![ @@ -557,7 +539,8 @@ impl RawInner { read_only: bool, log: &Logger, ) -> Result { - let bcount = def.extent_size().value; + let extent_size = def.extent_size(); + let bcount = extent_size.value; let size = def.block_size().checked_mul(bcount).unwrap() + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2 + BLOCK_META_SIZE_BYTES; @@ -565,7 +548,7 @@ impl RawInner { /* * Open the extent file and verify the size is as we expect. */ - let mut file = + let file = match OpenOptions::new().read(true).write(!read_only).open(path) { Err(e) => { error!( @@ -592,32 +575,81 @@ impl RawInner { // Just in case, let's be very sure that the file on disk is what it // should be - if let Err(e) = file.sync_all() { - return Err(CrucibleError::IoError(format!( - "extent {extent_number}: + if !read_only { + if let Err(e) = file.sync_all() { + return Err(CrucibleError::IoError(format!( + "extent {extent_number}: fsync 1 failure during initial rehash: {e:?}", - )) - .into()); + )) + .into()); + } } - file.seek(SeekFrom::Start(Self::meta_offset_from_extent_size( - def.extent_size(), - )))?; - let mut dirty = [0u8]; - file.read_exact(&mut dirty)?; - let dirty = match dirty[0] { - 0 => false, - 1 => true, - i => bail!("invalid dirty value: {i}"), + // Now, we'll compute which context slots are active in the file! We + // start by reading + hashing the file, then compare those hashes + // against values in the context slots. This is equivalent to + // `recompute_slot_from_file`, but reads the file in bulk for + // efficiency. + + // Buffer the file so we don't spend all day waiting on syscalls + let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); + + let block_hashes = { + let mut block_hashes = + Vec::with_capacity(extent_size.value as usize); + + // Stream the contents of the file and rehash them. + let mut buf = vec![0; extent_size.block_size_in_bytes() as usize]; + for _block in 0..extent_size.value as usize { + file_buffered.read_exact(&mut buf)?; + block_hashes.push(integrity_hash(&[&buf])); + } + block_hashes + }; + + let dirty = { + let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + file_buffered.read_exact(&mut meta_buf)?; + match meta_buf[0] { + 0 => false, + 1 => true, + i => bail!("invalid dirty value: {i}"), + } + }; + + // Now, read context data from the file and assign slots + let active_context = { + let mut buf = vec![0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + let mut active_context = vec![]; + for (block, hash) in block_hashes.into_iter().enumerate() { + let mut matching_slot = None; + let mut empty_slot = None; + for slot in [ContextSlot::A, ContextSlot::B] { + file_buffered.read_exact(&mut buf)?; + let context: Option = + bincode::deserialize(&buf)?; + if let Some(context) = context { + if context.on_disk_hash == hash { + matching_slot = Some(slot); + } + } else if empty_slot.is_none() { + empty_slot = Some(slot); + } + } + let value = matching_slot.or(empty_slot).ok_or( + CrucibleError::IoError(format!( + "no slot found for {block}" + )), + )?; + active_context.push(value); + } + active_context }; Ok(Self { file, // Lazy initialization of which context slot is active - active_context: vec![ - ActiveContext::Unknown; - def.extent_size().value as usize - ], + active_context, dirty, extent_number, extent_size: def.extent_size(), @@ -639,45 +671,55 @@ impl RawInner { Ok(()) } - /// Looks up the current context slot for the given block + /// Updates `self.active_context[block]` based on data read from the file /// - /// If the slot is currently unknown, rehashes the block and picks one of - /// the two slots, storing it locally. - fn get_block_context_slot(&mut self, block: usize) -> Result> { - match self.active_context[block] { - ActiveContext::Empty => Ok(None), - ActiveContext::Slot(b) => Ok(Some(b)), - ActiveContext::Unknown => { - let block_size = self.extent_size.block_size_in_bytes(); - let mut buf = vec![0; block_size as usize]; - self.file - .seek(SeekFrom::Start(block_size as u64 * block as u64))?; - self.file.read_exact(&mut buf)?; - let hash = integrity_hash(&[&buf]); - - self.file.seek(SeekFrom::Start( - self.context_slot_offset(block as u64, false), - ))?; - let mut buf = vec![0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - let mut found = None; - for slot in 0..2 { - self.file.read_exact(&mut buf)?; - let context: Option = - bincode::deserialize(&buf)?; - if let Some(context) = context { - if context.on_disk_hash == hash { - self.active_context[block] = - ActiveContext::Slot(slot != 0); - found = Some(slot != 0); - } - } - } - if found.is_none() { - self.active_context[block] = ActiveContext::Empty; + /// This returns an error if neither context slot matches the block data, + /// which should never happen (even during error conditions or after a + /// crash). + /// + /// We expect to call this function rarely, so it does not attempt to + /// minimize the number of syscalls it executes. + fn recompute_slot_from_file( + &mut self, + block: u64, + ) -> Result<(), CrucibleError> { + // Read the block data itself: + let block_size = self.extent_size.block_size_in_bytes(); + let mut buf = vec![0; block_size as usize]; + self.file.seek(SeekFrom::Start(block_size as u64 * block))?; + self.file.read_exact(&mut buf)?; + let hash = integrity_hash(&[&buf]); + + // Then, read the slot data and decide if either slot + // (1) is present and + // (2) has a matching hash + let mut buf = [0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + let mut matching_slot = None; + let mut empty_slot = None; + for slot in [ContextSlot::A, ContextSlot::B] { + // TODO: is `pread` faster than seek + read_exact? + self.file + .seek(SeekFrom::Start(self.context_slot_offset(block, slot)))?; + self.file.read_exact(&mut buf)?; + let context: Option = + bincode::deserialize(&buf).map_err(|e| { + CrucibleError::IoError(format!( + "context deserialization failed: {e:?}" + )) + })?; + if let Some(context) = context { + if context.on_disk_hash == hash { + matching_slot = Some(slot); } - Ok(found) + } else if empty_slot.is_none() { + empty_slot = Some(slot); } } + let value = matching_slot.or(empty_slot).ok_or( + CrucibleError::IoError(format!("no slot found for {block}")), + )?; + self.active_context[block as usize] = value; + Ok(()) } /// Writes the inactive block context slot @@ -686,10 +728,10 @@ impl RawInner { fn set_block_context( &mut self, block_context: &DownstairsBlockContext, - ) -> Result { + ) -> Result { let block = block_context.block as usize; // Select the inactive slot - let slot = !self.get_block_context_slot(block)?.unwrap_or(true); + let slot = !self.active_context[block]; // If the context slot that we're about to write into hasn't been // synched to disk yet, we must sync it first. This prevents subtle @@ -742,7 +784,7 @@ impl RawInner { /// Contexts slots are located after block data in the extent file. There /// are two context slots per block. We use a ping-pong strategy to ensure /// that one of them is always valid (i.e. matching the data in the file). - fn context_slot_offset(&self, block: u64, slot: bool) -> u64 { + fn context_slot_offset(&self, block: u64, slot: ContextSlot) -> u64 { self.extent_size.block_size_in_bytes() as u64 * self.extent_size.value + BLOCK_META_SIZE_BYTES + (block * 2 + slot as u64) * BLOCK_CONTEXT_SLOT_SIZE_BYTES @@ -763,9 +805,7 @@ impl RawInner { &mut self, block: u64, ) -> Result> { - let Some(slot) = self.get_block_context_slot(block as usize)? else { - return Ok(None); - }; + let slot = self.active_context[block as usize]; let offset = self.context_slot_offset(block, slot); let mut buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; nix::sys::uio::pread(self.file.as_raw_fd(), &mut buf, offset as i64) @@ -814,6 +854,32 @@ impl RawInner { } Ok(out) } + + fn write_inner( + &self, + writes: &[&crucible_protocol::Write], + writes_to_skip: &HashSet, + iov_max: usize, + ) -> Result<(), CrucibleError> { + // Now, batch writes into iovecs and use pwritev to write them all out. + let mut batched_pwritev = BatchedPwritev::new( + self.file.as_raw_fd(), + writes.len(), + self.extent_size.block_size_in_bytes().into(), + iov_max, + ); + + for write in writes { + if !writes_to_skip.contains(&write.offset.value) { + batched_pwritev.add_write(write)?; + } + } + + // Write any remaining data + batched_pwritev.perform_writes()?; + + Ok(()) + } } #[cfg(test)] From 3ffbc0a72172f4af888e02dc39ec6aef3b00fd2a Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 15:41:43 -0400 Subject: [PATCH 15/62] Shuffle array order --- downstairs/src/extent_inner_raw.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index e96350520..43a23d96a 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -781,13 +781,15 @@ impl RawInner { /// Returns the byte offset of the given context slot /// - /// Contexts slots are located after block data in the extent file. There - /// are two context slots per block. We use a ping-pong strategy to ensure - /// that one of them is always valid (i.e. matching the data in the file). + /// Contexts slots are located after block and meta data in the extent file. + /// There are two context slots arrays, each of which contains one context + /// slot per block. We use a ping-pong strategy to ensure that one of them + /// is always valid (i.e. matching the data in the file). fn context_slot_offset(&self, block: u64, slot: ContextSlot) -> u64 { self.extent_size.block_size_in_bytes() as u64 * self.extent_size.value + BLOCK_META_SIZE_BYTES - + (block * 2 + slot as u64) * BLOCK_CONTEXT_SLOT_SIZE_BYTES + + (self.extent_size.value * slot as u64 + block) + * BLOCK_CONTEXT_SLOT_SIZE_BYTES } /// Returns the byte offset of the metadata region From e14c5cd9dbb53e268d5977c0067772d8e18a3c6b Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 16:08:26 -0400 Subject: [PATCH 16/62] Almost fixed reading file from disk --- downstairs/src/extent_inner_raw.rs | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 43a23d96a..4fca3c708 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -619,15 +619,28 @@ impl RawInner { // Now, read context data from the file and assign slots let active_context = { - let mut buf = vec![0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; let mut active_context = vec![]; + let context_array_size = + BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_hashes.len(); + let mut a_data = vec![0u8; context_array_size]; + let mut b_data = vec![0u8; context_array_size]; + file_buffered.read_exact(&mut a_data)?; + file_buffered.read_exact(&mut b_data)?; + let mut a_iter = + a_data.chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); + let mut b_iter = + b_data.chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); for (block, hash) in block_hashes.into_iter().enumerate() { let mut matching_slot = None; let mut empty_slot = None; for slot in [ContextSlot::A, ContextSlot::B] { - file_buffered.read_exact(&mut buf)?; + let buf = match slot { + ContextSlot::A => a_iter.next(), + ContextSlot::B => b_iter.next(), + } + .unwrap(); let context: Option = - bincode::deserialize(&buf)?; + bincode::deserialize(buf)?; if let Some(context) = context { if context.on_disk_hash == hash { matching_slot = Some(slot); @@ -638,7 +651,7 @@ impl RawInner { } let value = matching_slot.or(empty_slot).ok_or( CrucibleError::IoError(format!( - "no slot found for {block}" + "open: no slot found for {block}" )), )?; active_context.push(value); From 10ce166cdd2d67dda39c9a6e52a41ec338b20058 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 16:19:59 -0400 Subject: [PATCH 17/62] Fix migration --- downstairs/src/extent_inner_sqlite.rs | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index a09e526a0..71cecd965 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -544,17 +544,11 @@ impl SqliteInner { gen_number, ext_version: EXTENT_META_RAW, // new extent version for raw files }; - let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - bincode::serialize_into(meta_buf.as_mut_slice(), &meta) + let mut buf = vec![0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(buf.as_mut_slice(), &meta) .map_err(|e| CrucibleError::IoError(e.to_string()))?; - let mut buf = Vec::with_capacity( - BLOCK_META_SIZE_BYTES as usize - + ctxs.len() * 2 * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize, - ); - buf.extend(meta_buf); - - // Put the context data after the metadata + // Put the context data after the metadata, all in slot A for c in ctxs { let ctx = match c.len() { 0 => None, @@ -569,10 +563,11 @@ impl SqliteInner { bincode::serialize_into(ctx_buf.as_mut_slice(), &ctx) .map_err(|e| CrucibleError::IoError(e.to_string()))?; buf.extend(ctx_buf); - - // The second slot is empty - buf.extend([0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]); } + // Slot B is entirely empty + buf.extend(std::iter::repeat(0).take( + (BLOCK_CONTEXT_SLOT_SIZE_BYTES * self.extent_size.value) as usize, + )); // Reset the file read position, just in case self.file.seek(SeekFrom::Start(0))?; From a71b80797408c13d9e61fa2da27b337cbc78b78c Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 16:30:35 -0400 Subject: [PATCH 18/62] Preparing to coalesce context writes --- downstairs/src/extent_inner_raw.rs | 74 ++++++++++++++++++++++-------- 1 file changed, 55 insertions(+), 19 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 4fca3c708..8f25e826a 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -234,25 +234,24 @@ impl ExtentInner for RawInner { (job_id.0, self.extent_number, writes.len() as u64) }); - let mut pending_slots = vec![]; - for write in writes { - if writes_to_skip.contains(&write.offset.value) { - pending_slots.push(None); - continue; - } - - // TODO it would be nice if we could profile what % of time we're - // spending on hashes locally vs writing to disk - let on_disk_hash = integrity_hash(&[&write.data[..]]); - - let next_slot = - self.set_block_context(&DownstairsBlockContext { + // Compute block contexts, then write them to disk + let block_ctx: Vec<_> = writes + .iter() + .filter(|write| !writes_to_skip.contains(&write.offset.value)) + .map(|write| { + // TODO it would be nice if we could profile what % of time we're + // spending on hashes locally vs writing to disk + let on_disk_hash = integrity_hash(&[&write.data[..]]); + + DownstairsBlockContext { block_context: write.block_context, block: write.offset.value, on_disk_hash, - })?; - pending_slots.push(Some(next_slot)); - } + } + }) + .collect(); + + self.set_block_contexts(&block_ctx)?; cdt::extent__write__raw__context__insert__done!(|| { (job_id.0, self.extent_number, writes.len() as u64) @@ -293,9 +292,12 @@ impl ExtentInner for RawInner { } } else { // Now that writes have gone through, update active context slots - for (write, new_slot) in writes.iter().zip(pending_slots) { - if let Some(slot) = new_slot { - self.active_context[write.offset.value as usize] = slot; + for write in writes.iter() { + let block = write.offset.value; + if !writes_to_skip.contains(&block) { + // We always write to the inactive slot, so just swap it + self.active_context[block as usize] = + !self.active_context[block as usize]; } } } @@ -782,6 +784,40 @@ impl RawInner { Ok(slot) } + fn set_block_contexts( + &mut self, + block_contexts: &[DownstairsBlockContext], + ) -> Result<()> { + let mut start = 0; + for i in 0..block_contexts.len() { + if i + 1 == block_contexts.len() + || block_contexts[i].block + 1 != block_contexts[i + 1].block + { + self.set_block_contexts_contiguous(&block_contexts[start..=i])?; + start = i + 1; + } + } + Ok(()) + } + + /// Efficiently sets block contexts in bulk + /// + /// # Panics + /// `block_contexts` must represent a contiguous set of blocks + fn set_block_contexts_contiguous( + &mut self, + block_contexts: &[DownstairsBlockContext], + ) -> Result<()> { + for (a, b) in block_contexts.iter().zip(block_contexts.iter().skip(1)) { + assert_eq!(a.block + 1, b.block, "blocks must be contiguous"); + } + + for b in block_contexts { + self.set_block_context(b)?; + } + Ok(()) + } + fn get_metadata(&self) -> Result { let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; let offset = self.meta_offset(); From 4e5c823d9a84001c961ce824e58c277fb666e79f Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 16:53:35 -0400 Subject: [PATCH 19/62] Write metadata in (semi) bulk --- Cargo.lock | 1 + downstairs/Cargo.toml | 1 + downstairs/src/extent_inner_raw.rs | 120 ++++++++++++++++------------- 3 files changed, 67 insertions(+), 55 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 693a74d8a..54fe42502 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -861,6 +861,7 @@ dependencies = [ "http", "hyper", "hyper-staticfile", + "itertools 0.11.0", "libc", "mime_guess", "nix", diff --git a/downstairs/Cargo.toml b/downstairs/Cargo.toml index ed9cdaa0f..e938a38a1 100644 --- a/downstairs/Cargo.toml +++ b/downstairs/Cargo.toml @@ -22,6 +22,7 @@ hex.workspace = true http.workspace = true hyper-staticfile.workspace = true hyper.workspace = true +itertools.workspace = true libc.workspace = true mime_guess.workspace = true nix.workspace = true diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 8f25e826a..d5a911c21 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -11,6 +11,7 @@ use crate::{ }; use anyhow::{bail, Result}; +use itertools::Itertools; use serde::{Deserialize, Serialize}; use slog::{error, Logger}; @@ -97,7 +98,7 @@ pub struct RawInner { context_slot_synched_at: Vec<[u64; 2]>, } -#[derive(Copy, Clone, Debug)] +#[derive(Copy, Clone, Debug, Eq, PartialEq)] enum ContextSlot { A, B, @@ -465,8 +466,9 @@ impl ExtentInner for RawInner { block_context: &DownstairsBlockContext, ) -> Result<(), CrucibleError> { self.set_dirty()?; - let new_slot = self.set_block_context(block_context)?; - self.active_context[block_context.block as usize] = new_slot; + self.set_block_contexts(&[block_context.clone()])?; + self.active_context[block_context.block as usize] = + !self.active_context[block_context.block as usize]; Ok(()) } @@ -737,57 +739,42 @@ impl RawInner { Ok(()) } - /// Writes the inactive block context slot - /// - /// Returns the new slot which should be marked as active after the write - fn set_block_context( + fn set_block_contexts( &mut self, - block_context: &DownstairsBlockContext, - ) -> Result { - let block = block_context.block as usize; - // Select the inactive slot - let slot = !self.active_context[block]; - - // If the context slot that we're about to write into hasn't been - // synched to disk yet, we must sync it first. This prevents subtle - // ordering issues! - let last_sync = &mut self.context_slot_synched_at[block][slot as usize]; - if *last_sync > self.sync_index { - assert_eq!(*last_sync, self.sync_index + 1); - if let Err(e) = self.file.sync_all() { - return Err(CrucibleError::IoError(format!( - "extent {}: fsync 1 failure: {:?}", - self.extent_number, e + block_contexts: &[DownstairsBlockContext], + ) -> Result<()> { + // If any of these block contexts will be overwriting an unsyched + // context slot, then we insert a sync here. + let needs_sync = block_contexts.iter().any(|block_context| { + let block = block_context.block as usize; + // We'll be writing to the inactive slot + let slot = !self.active_context[block]; + let last_sync = self.context_slot_synched_at[block][slot as usize]; + // We should never be > 1 sync ahead! + assert!(last_sync <= self.sync_index + 1); + last_sync > self.sync_index + }); + if needs_sync { + self.file.sync_all().map_err(|e| { + CrucibleError::IoError(format!( + "extent {}: fsync 1 failure: {e:?}", + self.extent_number, )) - .into()); - } + })?; self.sync_index += 1; } - // The given slot is about to be newly unsynched, because we're going to - // write to it below. - *last_sync = self.sync_index + 1; - - let offset = self.context_slot_offset(block_context.block, slot); - - // Serialize into a local buffer, then write into the inactive slot - let mut buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - let d = OnDiskDownstairsBlockContext { - block_context: block_context.block_context, - on_disk_hash: block_context.on_disk_hash, - }; - bincode::serialize_into(buf.as_mut_slice(), &Some(d))?; - nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - - // Return the just-written slot; it's the caller's responsibility to - // select it as active once data is written. - Ok(slot) - } + // Mark the to-be-written slots as unsynched on disk + // + // It's harmless if we bail out before writing the actual context slot + // here, because all it will do is force a sync next time this is called + // (that sync is right above here!) + for block_context in block_contexts { + let block = block_context.block as usize; + let slot = !self.active_context[block]; + self.context_slot_synched_at[block][slot as usize] = + self.sync_index + 1; + } - fn set_block_contexts( - &mut self, - block_contexts: &[DownstairsBlockContext], - ) -> Result<()> { let mut start = 0; for i in 0..block_contexts.len() { if i + 1 == block_contexts.len() @@ -812,9 +799,31 @@ impl RawInner { assert_eq!(a.block + 1, b.block, "blocks must be contiguous"); } - for b in block_contexts { - self.set_block_context(b)?; + let mut buf = vec![]; + for (slot, group) in block_contexts + .iter() + .group_by(|block_context| + // We'll be writing to the inactive slot + !self.active_context[block_context.block as usize]) + .into_iter() + { + let mut group = group.peekable(); + let block_start = group.peek().unwrap().block; + buf.clear(); + for block_context in group { + let n = buf.len(); + buf.extend([0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]); + let d = OnDiskDownstairsBlockContext { + block_context: block_context.block_context, + on_disk_hash: block_context.on_disk_hash, + }; + bincode::serialize_into(&mut buf[n..], &Some(d))?; + } + let offset = self.context_slot_offset(block_start, slot); + nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; } + Ok(()) } @@ -1304,18 +1313,19 @@ mod test { hash, }, }; - // The context should be written to slot 0 + // The context should be written to slot B inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.sync_index, 0); - // Flush, which should bump the sync number (marking slot 0 as synched) + // Flush, which should bump the sync number (marking slot B as synched) inner.flush(12, 12, JobId(11).into())?; + assert_eq!(inner.sync_index, 1); - // The context should be written to slot 1 + // The context should be written to slot A inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.sync_index, 1); - // The context should be written to slot 0 + // The context should be written to slot B, forcing a sync inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.sync_index, 1); From 4412b840662fd693a5b399e821e00223e46166c2 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 17 Oct 2023 17:21:42 -0400 Subject: [PATCH 20/62] Add DTrace probe for write count --- downstairs/src/extent_inner_raw.rs | 17 ++++++++++++++--- downstairs/src/lib.rs | 5 +++++ 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index d5a911c21..f8b8a31a1 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -776,30 +776,40 @@ impl RawInner { } let mut start = 0; + let mut write_count = 0; for i in 0..block_contexts.len() { if i + 1 == block_contexts.len() || block_contexts[i].block + 1 != block_contexts[i + 1].block { - self.set_block_contexts_contiguous(&block_contexts[start..=i])?; + write_count += self.set_block_contexts_contiguous( + &block_contexts[start..=i], + )?; start = i + 1; } } + cdt::extent__set__block__contexts__write__count!(|| ( + self.extent_number, + write_count as u64, + )); Ok(()) } /// Efficiently sets block contexts in bulk /// + /// Returns the number of writes, for profiling + /// /// # Panics /// `block_contexts` must represent a contiguous set of blocks fn set_block_contexts_contiguous( &mut self, block_contexts: &[DownstairsBlockContext], - ) -> Result<()> { + ) -> Result { for (a, b) in block_contexts.iter().zip(block_contexts.iter().skip(1)) { assert_eq!(a.block + 1, b.block, "blocks must be contiguous"); } let mut buf = vec![]; + let mut writes = 0; for (slot, group) in block_contexts .iter() .group_by(|block_context| @@ -822,9 +832,10 @@ impl RawInner { let offset = self.context_slot_offset(block_start, slot); nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) .map_err(|e| CrucibleError::IoError(e.to_string()))?; + writes += 1; } - Ok(()) + Ok(writes) } fn get_metadata(&self) -> Result { diff --git a/downstairs/src/lib.rs b/downstairs/src/lib.rs index 7e511f0d6..cd833a077 100644 --- a/downstairs/src/lib.rs +++ b/downstairs/src/lib.rs @@ -477,6 +477,11 @@ pub mod cdt { fn extent__context__truncate__start(n_deletions: u64) {} fn extent__context__truncate__done() {} + fn extent__set__block__contexts__write__count( + extent_id: u32, + n_blocks: u64, + ) { + } fn submit__el__close__done(_: u64) {} fn submit__el__flush__close__done(_: u64) {} fn submit__el__repair__done(_: u64) {} From 600be6e6405bf41ad6c8bbc083e23ad07fb24e57 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Wed, 18 Oct 2023 09:20:06 -0400 Subject: [PATCH 21/62] Add unit test for serialized sizes --- downstairs/src/extent_inner_raw.rs | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index f8b8a31a1..9c58e226e 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -42,7 +42,7 @@ pub struct OnDiskMeta { /// Size of backup data /// -/// This must be large enough to contain an `Option` +/// This must be large enough to fit an `Option` /// serialized using `bincode`. pub const BLOCK_CONTEXT_SLOT_SIZE_BYTES: u64 = 64; @@ -1460,4 +1460,29 @@ mod test { Ok(()) } + + #[test] + fn test_serialized_sizes() { + let c = OnDiskDownstairsBlockContext { + block_context: BlockContext { + hash: u64::MAX, + encryption_context: Some(EncryptionContext { + nonce: [0xFF; 12], + tag: [0xFF; 16], + }), + }, + on_disk_hash: u64::MAX, + }; + let mut ctx_buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + bincode::serialize_into(ctx_buf.as_mut_slice(), &Some(c)).unwrap(); + + let m = OnDiskMeta { + dirty: true, + gen_number: u64::MAX, + flush_number: u64::MAX, + ext_version: u32::MAX, + }; + let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(meta_buf.as_mut_slice(), &Some(m)).unwrap(); + } } From 867fd567334b395fb8922ce7fcebf47215056b56 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Wed, 18 Oct 2023 09:55:26 -0400 Subject: [PATCH 22/62] Shrink metadata sizes --- downstairs/src/extent_inner_raw.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 9c58e226e..47723319f 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -44,13 +44,13 @@ pub struct OnDiskMeta { /// /// This must be large enough to fit an `Option` /// serialized using `bincode`. -pub const BLOCK_CONTEXT_SLOT_SIZE_BYTES: u64 = 64; +pub const BLOCK_CONTEXT_SLOT_SIZE_BYTES: u64 = 48; /// Size of metadata region /// /// This must be large enough to contain an `OnDiskMeta` serialized using /// `bincode`. -pub const BLOCK_META_SIZE_BYTES: u64 = 64; +pub const BLOCK_META_SIZE_BYTES: u64 = 32; /// `RawInner` is a wrapper around a [`std::fs::File`] representing an extent /// From 02ab7160351ef1aa3af36c9a183a283504a98ae8 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Wed, 18 Oct 2023 11:19:16 -0400 Subject: [PATCH 23/62] Coalesce block context reads --- downstairs/src/extent_inner_raw.rs | 86 +++++++++++++++++------------- 1 file changed, 50 insertions(+), 36 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 47723319f..8b58014ce 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -872,26 +872,6 @@ impl RawInner { extent_size.block_size_in_bytes() as u64 * extent_size.value } - fn get_block_context( - &mut self, - block: u64, - ) -> Result> { - let slot = self.active_context[block as usize]; - let offset = self.context_slot_offset(block, slot); - let mut buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - nix::sys::uio::pread(self.file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - let out: Option = - bincode::deserialize(&buf)?; - let out = out.map(|c| DownstairsBlockContext { - block, - block_context: c.block_context, - on_disk_hash: c.on_disk_hash, - }); - - Ok(out) - } - /// Update the flush number, generation number, and clear the dirty bit fn set_flush_number(&mut self, new_flush: u64, new_gen: u64) -> Result<()> { let d = OnDiskMeta { @@ -919,10 +899,35 @@ impl RawInner { count: u64, ) -> Result>> { let mut out = vec![]; - for i in block..block + count { - let ctx = self.get_block_context(i)?; - out.push(ctx); + let mut buf = vec![]; + for (slot, mut group) in (block..block + count) + .group_by(|block| self.active_context[*block as usize]) + .into_iter() + { + let group_start = group.next().unwrap(); + let group_count = group.count() + 1; + buf.resize(group_count * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize, 0); + let offset = self.context_slot_offset(group_start, slot); + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut buf, + offset as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + for (i, chunk) in buf + .chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) + .enumerate() + { + let ctx: Option = + bincode::deserialize(chunk)?; + out.push(ctx.map(|c| DownstairsBlockContext { + block: group_start + i as u64, + block_context: c.block_context, + on_disk_hash: c.on_disk_hash, + })); + } } + Ok(out) } @@ -951,6 +956,19 @@ impl RawInner { Ok(()) } + + /// Helper function to get a single block context + /// + /// This is inefficient and should only be used in unit tests + #[cfg(test)] + fn get_block_context( + &mut self, + block: u64, + ) -> Result> { + let mut out = self.get_block_contexts(block, 1)?; + assert_eq!(out.len(), 1); + Ok(out.pop().unwrap()) + } } #[cfg(test)] @@ -978,8 +996,8 @@ mod test { // Encryption context for blocks 0 and 1 should start blank - assert!(inner.get_block_contexts(0, 1)?[0].is_none()); - assert!(inner.get_block_contexts(1, 1)?[0].is_none()); + assert!(inner.get_block_context(0)?.is_none()); + assert!(inner.get_block_context(1)?.is_none()); // Set and verify block 0's context inner.set_dirty_and_block_context(&DownstairsBlockContext { @@ -997,8 +1015,7 @@ mod test { on_disk_hash: 456, })?; - let ctxs = inner.get_block_contexts(0, 1)?; - let ctx = ctxs[0].as_ref().unwrap(); + let ctx = inner.get_block_context(0)?.unwrap(); assert_eq!( ctx.block_context.encryption_context.unwrap().nonce, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] @@ -1011,7 +1028,7 @@ mod test { assert_eq!(ctx.on_disk_hash, 456); // Block 1 should still be blank - assert!(inner.get_block_contexts(1, 1)?[0].is_none()); + assert!(inner.get_block_context(1)?.is_none()); // Set and verify a new context for block 0 let blob1 = rand::thread_rng().gen::<[u8; 12]>(); @@ -1030,8 +1047,7 @@ mod test { on_disk_hash: 65536, })?; - let ctxs = inner.get_block_contexts(0, 1)?; - let ctx = ctxs[0].as_ref().unwrap(); + let ctx = inner.get_block_context(0)?.unwrap(); // Second context was appended assert_eq!(ctx.block_context.encryption_context.unwrap().nonce, blob1); @@ -1051,8 +1067,8 @@ mod test { // Encryption context for blocks 0 and 1 should start blank - assert!(inner.get_block_contexts(0, 1)?[0].is_none()); - assert!(inner.get_block_contexts(1, 1)?[0].is_none()); + assert!(inner.get_block_context(0)?.is_none()); + assert!(inner.get_block_context(1)?.is_none()); // Set block 0's and 1's context and dirty flag inner.set_dirty_and_block_context(&DownstairsBlockContext { @@ -1082,8 +1098,7 @@ mod test { })?; // Verify block 0's context - let ctxs = inner.get_block_contexts(0, 1)?; - let ctx = ctxs[0].as_ref().unwrap(); + let ctx = inner.get_block_context(0)?.unwrap(); assert_eq!( ctx.block_context.encryption_context.unwrap().nonce, [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] @@ -1096,8 +1111,7 @@ mod test { assert_eq!(ctx.on_disk_hash, 456); // Verify block 1's context - let ctxs = inner.get_block_contexts(1, 1)?; - let ctx = ctxs[0].as_ref().unwrap(); + let ctx = inner.get_block_context(1)?.unwrap(); assert_eq!( ctx.block_context.encryption_context.unwrap().nonce, From ba020800898f3dd099fd499c4b09f85cd03e45ee Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Wed, 18 Oct 2023 13:13:16 -0400 Subject: [PATCH 24/62] Remove seeking in favor of pread --- downstairs/src/extent_inner_raw.rs | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 8b58014ce..7d7043ef6 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -17,7 +17,7 @@ use slog::{error, Logger}; use std::collections::HashSet; use std::fs::{File, OpenOptions}; -use std::io::{BufReader, IoSliceMut, Read, Seek, SeekFrom}; +use std::io::{BufReader, IoSliceMut, Read}; use std::os::fd::AsRawFd; use std::path::Path; @@ -452,9 +452,6 @@ impl ExtentInner for RawInner { (job_id.get(), self.extent_number, 0) }); - // Finally, reset the file's seek offset to 0 - self.file.seek(SeekFrom::Start(0))?; - cdt::extent__flush__done!(|| { (job_id.get(), self.extent_number, 0) }); Ok(()) @@ -496,7 +493,7 @@ impl RawInner { + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2; mkdir_for_file(&path)?; - let mut file = OpenOptions::new() + let file = OpenOptions::new() .read(true) .write(true) .create(true) @@ -504,7 +501,6 @@ impl RawInner { // All 0s are fine for everything except extent version in the metadata file.set_len(size)?; - file.seek(SeekFrom::Start(0))?; let mut out = Self { file, dirty: false, @@ -703,8 +699,12 @@ impl RawInner { // Read the block data itself: let block_size = self.extent_size.block_size_in_bytes(); let mut buf = vec![0; block_size as usize]; - self.file.seek(SeekFrom::Start(block_size as u64 * block))?; - self.file.read_exact(&mut buf)?; + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut buf, + (block_size as u64 * block) as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; let hash = integrity_hash(&[&buf]); // Then, read the slot data and decide if either slot @@ -714,10 +714,12 @@ impl RawInner { let mut matching_slot = None; let mut empty_slot = None; for slot in [ContextSlot::A, ContextSlot::B] { - // TODO: is `pread` faster than seek + read_exact? - self.file - .seek(SeekFrom::Start(self.context_slot_offset(block, slot)))?; - self.file.read_exact(&mut buf)?; + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut buf, + self.context_slot_offset(block, slot) as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; let context: Option = bincode::deserialize(&buf).map_err(|e| { CrucibleError::IoError(format!( From badd2f9246d16c534ead78ab10f3dc476949bedd Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Wed, 18 Oct 2023 17:26:12 -0400 Subject: [PATCH 25/62] Implement context defragmentation --- downstairs/src/extent_inner_raw.rs | 145 +++++++++++++++++++++++++++-- 1 file changed, 139 insertions(+), 6 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 7d7043ef6..62519bd91 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -427,6 +427,10 @@ impl ExtentInner for RawInner { return Ok(()); } + cdt::extent__flush__start!(|| { + (job_id.get(), self.extent_number, 0) + }); + // We put all of our metadata updates into a single write to make this // operation atomic. self.set_flush_number(new_flush, new_gen)?; @@ -452,9 +456,131 @@ impl ExtentInner for RawInner { (job_id.get(), self.extent_number, 0) }); + // At this point, the active context slots (on a per-block basis) + // may be scattered across the two arrays: + // + // block | 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | ... | + // context | A | A | | | | | A | A | A | ... | [A array] + // | | | B | B | B | B | | | | ... | [B array] + // + // This can be inefficient, because it means that a write would have to + // be split into updating multiple regions (instead of a single + // contiguous write). As such, if the context slots disagree, we + // "defragment" them: + // + // - Figure out whether A or B is more popular + // - Copy context data from the less-popular slot to the more-popular + // + // In the example above, `A` is more popular so we would perform the + // following copy: + // + // block | 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | ... | + // context | A | A | ^ | ^ | ^ | ^ | A | A | A | ... | [A array] + // | | | B | B | B | B | | | | ... | [B array] + // + // This is safe because it occurs immediately after a flush, so we are + // dead certain that the active context matches file contents. This + // means that we can safely overwrite the old (inactive) context. + // + // We track the number of A vs B slots, as well as the range covered by + // the slots. It's that range that we'll need to read + write, so we + // want to pick whichever slot does less work. + #[derive(Copy, Clone)] + struct Counter { + count: usize, + min_block: usize, + max_block: usize, + } + let mut a_count = Counter { + count: 0, + min_block: usize::MAX, + max_block: 0, + }; + let mut b_count = a_count; + for (i, s) in self.active_context.iter().enumerate() { + let count = match s { + ContextSlot::A => &mut a_count, + ContextSlot::B => &mut b_count, + }; + count.count += 1; + count.min_block = count.min_block.min(i); + count.max_block = count.max_block.max(i); + } + let r = if a_count.count != 0 && b_count.count != 0 { + // We want to do as little copying as possible + let (copy_from, counter) = if a_count.count < b_count.count { + (ContextSlot::A, a_count) + } else { + (ContextSlot::B, b_count) + }; + assert!(counter.count > 0); + assert!(counter.max_block >= counter.min_block); + let num_slots = counter.max_block + 1 - counter.min_block; + + // Read the source context slots from the file + let mut source_buf = + vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut source_buf, + self.context_slot_offset(counter.min_block as u64, copy_from) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + // Read the destination context slots from the file + let mut dest_buf = + vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut dest_buf, + self.context_slot_offset(counter.min_block as u64, !copy_from) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + // Selectively overwrite dest with source context slots + for (i, block) in + (counter.min_block..=counter.max_block).enumerate() + { + if self.active_context[block] == copy_from { + let chunk = (i * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) + ..((i + 1) * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); + dest_buf[chunk.clone()].copy_from_slice(&source_buf[chunk]); + + // Mark this slot as unsynched, so that we don't overwrite + // it later on. + self.context_slot_synched_at[block][!copy_from as usize] = + self.sync_index + 1; + } + } + let r = nix::sys::uio::pwrite( + self.file.as_raw_fd(), + &dest_buf, + self.context_slot_offset(counter.min_block as u64, !copy_from) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string())); + + // If this write failed, then try recomputing every context slot + // within the unknown range + if r.is_err() { + for block in counter.min_block..=counter.max_block { + self.recompute_slot_from_file(block as u64).unwrap(); + } + } else { + for block in counter.min_block..=counter.max_block { + self.active_context[block] = !copy_from; + } + } + r.map(|_| ()) + } else { + Ok(()) + }; + cdt::extent__flush__done!(|| { (job_id.get(), self.extent_number, 0) }); - Ok(()) + r } #[cfg(test)] @@ -1342,22 +1468,29 @@ mod test { }; // The context should be written to slot B inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.active_context[0], ContextSlot::B); assert_eq!(inner.sync_index, 0); - // Flush, which should bump the sync number (marking slot B as synched) + // Flush! This will (1) bump the sync number (marking slot B as + // synched), then (2) defragment by copying slot B to slot A. inner.flush(12, 12, JobId(11).into())?; + assert_eq!(inner.active_context[0], ContextSlot::A); assert_eq!(inner.sync_index, 1); - // The context should be written to slot A + // The context should be written to slot B inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.active_context[0], ContextSlot::B); assert_eq!(inner.sync_index, 1); - // The context should be written to slot B, forcing a sync + // The context should be written to slot A, forcing a sync (because slot + // A was left unsynched after being defragmented) inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.active_context[0], ContextSlot::A); + assert_eq!(inner.sync_index, 2); - // The context should be written to slot 1, forcing a sync + // The context should be written to slot B, which is fine inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.active_context[0], ContextSlot::B); assert_eq!(inner.sync_index, 2); Ok(()) From 5563c2ae1a77445f632ab4a17727377e50190a30 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 19 Oct 2023 08:32:36 -0400 Subject: [PATCH 26/62] Only defragment if we're doing extra work --- downstairs/src/extent_inner_raw.rs | 307 ++++++++++++++++------------- 1 file changed, 174 insertions(+), 133 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 62519bd91..cb87e8964 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -52,6 +52,9 @@ pub const BLOCK_CONTEXT_SLOT_SIZE_BYTES: u64 = 48; /// `bincode`. pub const BLOCK_META_SIZE_BYTES: u64 = 32; +/// Number of extra syscalls per read / write that triggers defragmentation +const DEFRAGMENT_THRESHOLD: u64 = 3; + /// `RawInner` is a wrapper around a [`std::fs::File`] representing an extent /// /// The file is structured as follows: @@ -96,6 +99,12 @@ pub struct RawInner { /// this array is set to `self.sync_index + 1` indicates that the slot has /// not yet been synched. context_slot_synched_at: Vec<[u64; 2]>, + + /// Total number of extra syscalls due to context slot fragmentation + extra_syscall_count: u64, + + /// Denominator corresponding to `extra_syscall_count` + extra_syscall_denominator: u64, } #[derive(Copy, Clone, Debug, Eq, PartialEq)] @@ -456,124 +465,16 @@ impl ExtentInner for RawInner { (job_id.get(), self.extent_number, 0) }); - // At this point, the active context slots (on a per-block basis) - // may be scattered across the two arrays: - // - // block | 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | ... | - // context | A | A | | | | | A | A | A | ... | [A array] - // | | | B | B | B | B | | | | ... | [B array] - // - // This can be inefficient, because it means that a write would have to - // be split into updating multiple regions (instead of a single - // contiguous write). As such, if the context slots disagree, we - // "defragment" them: - // - // - Figure out whether A or B is more popular - // - Copy context data from the less-popular slot to the more-popular - // - // In the example above, `A` is more popular so we would perform the - // following copy: - // - // block | 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | ... | - // context | A | A | ^ | ^ | ^ | ^ | A | A | A | ... | [A array] - // | | | B | B | B | B | | | | ... | [B array] - // - // This is safe because it occurs immediately after a flush, so we are - // dead certain that the active context matches file contents. This - // means that we can safely overwrite the old (inactive) context. - // - // We track the number of A vs B slots, as well as the range covered by - // the slots. It's that range that we'll need to read + write, so we - // want to pick whichever slot does less work. - #[derive(Copy, Clone)] - struct Counter { - count: usize, - min_block: usize, - max_block: usize, - } - let mut a_count = Counter { - count: 0, - min_block: usize::MAX, - max_block: 0, - }; - let mut b_count = a_count; - for (i, s) in self.active_context.iter().enumerate() { - let count = match s { - ContextSlot::A => &mut a_count, - ContextSlot::B => &mut b_count, - }; - count.count += 1; - count.min_block = count.min_block.min(i); - count.max_block = count.max_block.max(i); - } - let r = if a_count.count != 0 && b_count.count != 0 { - // We want to do as little copying as possible - let (copy_from, counter) = if a_count.count < b_count.count { - (ContextSlot::A, a_count) - } else { - (ContextSlot::B, b_count) - }; - assert!(counter.count > 0); - assert!(counter.max_block >= counter.min_block); - let num_slots = counter.max_block + 1 - counter.min_block; - - // Read the source context slots from the file - let mut source_buf = - vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - nix::sys::uio::pread( - self.file.as_raw_fd(), - &mut source_buf, - self.context_slot_offset(counter.min_block as u64, copy_from) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - - // Read the destination context slots from the file - let mut dest_buf = - vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - nix::sys::uio::pread( - self.file.as_raw_fd(), - &mut dest_buf, - self.context_slot_offset(counter.min_block as u64, !copy_from) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - - // Selectively overwrite dest with source context slots - for (i, block) in - (counter.min_block..=counter.max_block).enumerate() - { - if self.active_context[block] == copy_from { - let chunk = (i * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) - ..((i + 1) * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); - dest_buf[chunk.clone()].copy_from_slice(&source_buf[chunk]); - - // Mark this slot as unsynched, so that we don't overwrite - // it later on. - self.context_slot_synched_at[block][!copy_from as usize] = - self.sync_index + 1; - } - } - let r = nix::sys::uio::pwrite( - self.file.as_raw_fd(), - &dest_buf, - self.context_slot_offset(counter.min_block as u64, !copy_from) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string())); - - // If this write failed, then try recomputing every context slot - // within the unknown range - if r.is_err() { - for block in counter.min_block..=counter.max_block { - self.recompute_slot_from_file(block as u64).unwrap(); - } - } else { - for block in counter.min_block..=counter.max_block { - self.active_context[block] = !copy_from; - } - } - r.map(|_| ()) + // Check for fragmentation in the context slots leading to worse + // performance, and defragment if that's the case. + let extra_syscalls_per_rw = self + .extra_syscall_count + .checked_div(self.extra_syscall_denominator) + .unwrap_or(0); + self.extra_syscall_count = 0; + self.extra_syscall_denominator = 0; + let r = if extra_syscalls_per_rw > DEFRAGMENT_THRESHOLD { + self.defragment() } else { Ok(()) }; @@ -641,6 +542,8 @@ impl RawInner { def.extent_size().value as usize ], sync_index: 0, + extra_syscall_count: 0, + extra_syscall_denominator: 0, }; // Setting the flush number also writes the extent version, since // they're serialized together in the same block. @@ -785,7 +688,7 @@ impl RawInner { active_context }; - Ok(Self { + let mut out = Self { file, // Lazy initialization of which context slot is active active_context, @@ -796,8 +699,14 @@ impl RawInner { [0, 0]; def.extent_size().value as usize ], + extra_syscall_count: 0, + extra_syscall_denominator: 0, sync_index: 0, - }) + }; + if !read_only { + out.defragment()?; + } + Ok(out) } fn set_dirty(&mut self) -> Result<(), CrucibleError> { @@ -917,7 +826,7 @@ impl RawInner { } cdt::extent__set__block__contexts__write__count!(|| ( self.extent_number, - write_count as u64, + write_count, )); Ok(()) } @@ -931,13 +840,13 @@ impl RawInner { fn set_block_contexts_contiguous( &mut self, block_contexts: &[DownstairsBlockContext], - ) -> Result { + ) -> Result { for (a, b) in block_contexts.iter().zip(block_contexts.iter().skip(1)) { assert_eq!(a.block + 1, b.block, "blocks must be contiguous"); } let mut buf = vec![]; - let mut writes = 0; + let mut writes = 0u64; for (slot, group) in block_contexts .iter() .group_by(|block_context| @@ -962,6 +871,10 @@ impl RawInner { .map_err(|e| CrucibleError::IoError(e.to_string()))?; writes += 1; } + if let Some(writes) = writes.checked_sub(1) { + self.extra_syscall_count += writes; + self.extra_syscall_denominator += 1; + } Ok(writes) } @@ -1028,6 +941,7 @@ impl RawInner { ) -> Result>> { let mut out = vec![]; let mut buf = vec![]; + let mut reads = 0u64; for (slot, mut group) in (block..block + count) .group_by(|block| self.active_context[*block as usize]) .into_iter() @@ -1054,6 +968,11 @@ impl RawInner { on_disk_hash: c.on_disk_hash, })); } + reads += 1; + } + if let Some(reads) = reads.checked_sub(1) { + self.extra_syscall_count += reads; + self.extra_syscall_denominator += 1; } Ok(out) @@ -1097,6 +1016,130 @@ impl RawInner { assert_eq!(out.len(), 1); Ok(out.pop().unwrap()) } + + /// Consolidates context slots into either the A or B array + /// + /// This must only be run directly after the file is synced to disk + fn defragment(&mut self) -> Result<(), CrucibleError> { + // At this point, the active context slots (on a per-block basis) + // may be scattered across the two arrays: + // + // block | 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | ... | + // context | A | A | | | | | A | A | A | ... | [A array] + // | | | B | B | B | B | | | | ... | [B array] + // + // This can be inefficient, because it means that a write would have to + // be split into updating multiple regions (instead of a single + // contiguous write). As such, if the context slots disagree, we + // "defragment" them: + // + // - Figure out whether A or B is more popular + // - Copy context data from the less-popular slot to the more-popular + // + // In the example above, `A` is more popular so we would perform the + // following copy: + // + // block | 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | ... | + // context | A | A | ^ | ^ | ^ | ^ | A | A | A | ... | [A array] + // | | | B | B | B | B | | | | ... | [B array] + // + // This is safe because it occurs immediately after a flush, so we are + // dead certain that the active context matches file contents. This + // means that we can safely overwrite the old (inactive) context. + // + // We track the number of A vs B slots, as well as the range covered by + // the slots. It's that range that we'll need to read + write, so we + // want to pick whichever slot does less work. + #[derive(Copy, Clone)] + struct Counter { + count: usize, + min_block: usize, + max_block: usize, + } + let mut a_count = Counter { + count: 0, + min_block: usize::MAX, + max_block: 0, + }; + let mut b_count = a_count; + for (i, s) in self.active_context.iter().enumerate() { + let count = match s { + ContextSlot::A => &mut a_count, + ContextSlot::B => &mut b_count, + }; + count.count += 1; + count.min_block = count.min_block.min(i); + count.max_block = count.max_block.max(i); + } + if a_count.count == 0 || b_count.count == 0 { + return Ok(()); + } + + let (copy_from, counter) = if a_count.count < b_count.count { + (ContextSlot::A, a_count) + } else { + (ContextSlot::B, b_count) + }; + assert!(counter.count > 0); + assert!(counter.max_block >= counter.min_block); + let num_slots = counter.max_block + 1 - counter.min_block; + + // Read the source context slots from the file + let mut source_buf = + vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut source_buf, + self.context_slot_offset(counter.min_block as u64, copy_from) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + // Read the destination context slots from the file + let mut dest_buf = + vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + nix::sys::uio::pread( + self.file.as_raw_fd(), + &mut dest_buf, + self.context_slot_offset(counter.min_block as u64, !copy_from) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + // Selectively overwrite dest with source context slots + for (i, block) in (counter.min_block..=counter.max_block).enumerate() { + if self.active_context[block] == copy_from { + let chunk = (i * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) + ..((i + 1) * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); + dest_buf[chunk.clone()].copy_from_slice(&source_buf[chunk]); + + // Mark this slot as unsynched, so that we don't overwrite + // it later on without a sync + self.context_slot_synched_at[block][!copy_from as usize] = + self.sync_index + 1; + } + } + let r = nix::sys::uio::pwrite( + self.file.as_raw_fd(), + &dest_buf, + self.context_slot_offset(counter.min_block as u64, !copy_from) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string())); + + // If this write failed, then try recomputing every context slot + // within the unknown range + if r.is_err() { + for block in counter.min_block..=counter.max_block { + self.recompute_slot_from_file(block as u64).unwrap(); + } + } else { + for block in counter.min_block..=counter.max_block { + self.active_context[block] = !copy_from; + } + } + r.map(|_| ()) + } } #[cfg(test)] @@ -1471,28 +1514,26 @@ mod test { assert_eq!(inner.active_context[0], ContextSlot::B); assert_eq!(inner.sync_index, 0); - // Flush! This will (1) bump the sync number (marking slot B as - // synched), then (2) defragment by copying slot B to slot A. + // Flush! This will bump the sync number, marking slot B as synched inner.flush(12, 12, JobId(11).into())?; + assert_eq!(inner.active_context[0], ContextSlot::B); + assert_eq!(inner.sync_index, 1); + + // The context should be written to slot A + inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::A); assert_eq!(inner.sync_index, 1); // The context should be written to slot B - inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; + inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::B); assert_eq!(inner.sync_index, 1); - // The context should be written to slot A, forcing a sync (because slot - // A was left unsynched after being defragmented) + // The context should be written to slot A, forcing a sync inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::A); assert_eq!(inner.sync_index, 2); - // The context should be written to slot B, which is fine - inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.active_context[0], ContextSlot::B); - assert_eq!(inner.sync_index, 2); - Ok(()) } From 0ce8cabcdc525251991b7bd5091373c121fde3f1 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 19 Oct 2023 15:17:54 -0400 Subject: [PATCH 27/62] Only hash the file when slot is ambiguous --- downstairs/src/extent_inner_raw.rs | 112 +++++++++++++++-------------- 1 file changed, 60 insertions(+), 52 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index cb87e8964..c5118d527 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -17,12 +17,12 @@ use slog::{error, Logger}; use std::collections::HashSet; use std::fs::{File, OpenOptions}; -use std::io::{BufReader, IoSliceMut, Read}; +use std::io::{BufReader, IoSliceMut, Read, Seek, SeekFrom}; use std::os::fd::AsRawFd; use std::path::Path; /// Equivalent to `DownstairsBlockContext`, but without one's own block number -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct OnDiskDownstairsBlockContext { pub block_context: BlockContext, pub on_disk_hash: u64, @@ -577,7 +577,7 @@ impl RawInner { /* * Open the extent file and verify the size is as we expect. */ - let file = + let mut file = match OpenOptions::new().read(true).write(!read_only).open(path) { Err(e) => { error!( @@ -614,28 +614,13 @@ impl RawInner { } } - // Now, we'll compute which context slots are active in the file! We - // start by reading + hashing the file, then compare those hashes - // against values in the context slots. This is equivalent to - // `recompute_slot_from_file`, but reads the file in bulk for - // efficiency. + // Position ourselves at the end of file data + file.seek(SeekFrom::Start(bcount * def.block_size()))?; // Buffer the file so we don't spend all day waiting on syscalls let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); - let block_hashes = { - let mut block_hashes = - Vec::with_capacity(extent_size.value as usize); - - // Stream the contents of the file and rehash them. - let mut buf = vec![0; extent_size.block_size_in_bytes() as usize]; - for _block in 0..extent_size.value as usize { - file_buffered.read_exact(&mut buf)?; - block_hashes.push(integrity_hash(&[&buf])); - } - block_hashes - }; - + // Read the metadata block let dirty = { let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; file_buffered.read_exact(&mut meta_buf)?; @@ -646,30 +631,56 @@ impl RawInner { } }; - // Now, read context data from the file and assign slots - let active_context = { - let mut active_context = vec![]; - let context_array_size = - BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_hashes.len(); - let mut a_data = vec![0u8; context_array_size]; - let mut b_data = vec![0u8; context_array_size]; - file_buffered.read_exact(&mut a_data)?; - file_buffered.read_exact(&mut b_data)?; - let mut a_iter = - a_data.chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); - let mut b_iter = - b_data.chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); - for (block, hash) in block_hashes.into_iter().enumerate() { + // Read the two context slot arrays + let mut context_arrays = vec![]; + for _slot in [ContextSlot::A, ContextSlot::B] { + let mut contexts = Vec::with_capacity(bcount as usize); + let mut buf = vec![0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + for _block in 0..bcount as usize { + file_buffered.read_exact(&mut buf)?; + let context: Option = + bincode::deserialize(&buf).map_err(|e| { + CrucibleError::IoError(format!( + "context deserialization failed: {e}" + )) + })?; + contexts.push(context); + } + context_arrays.push(contexts); + } + + file.seek(SeekFrom::Start(0))?; + let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); + let mut active_context = vec![]; + let mut buf = vec![0; extent_size.block_size_in_bytes() as usize]; + let mut last_seek_block = 0; + for (block, (context_a, context_b)) in context_arrays[0] + .iter() + .zip(context_arrays[1].iter()) + .enumerate() + { + let slot = if context_a == context_b { + // If both slots are identical, then either they're both None or + // we have defragmented recently (which copies the active slot + // to the inactive one). That makes life easy! + ContextSlot::A + } else { + // Otherwise, we have to compute hashes from the file. + if block != last_seek_block { + file_buffered.seek_relative( + (block - last_seek_block) as i64 + * extent_size.block_size_in_bytes() as i64, + )?; + } + file_buffered.read_exact(&mut buf)?; + last_seek_block = block + 1; // since we just read a block + let hash = integrity_hash(&[&buf]); + let mut matching_slot = None; let mut empty_slot = None; + for slot in [ContextSlot::A, ContextSlot::B] { - let buf = match slot { - ContextSlot::A => a_iter.next(), - ContextSlot::B => b_iter.next(), - } - .unwrap(); - let context: Option = - bincode::deserialize(buf)?; + let context = [context_a, context_b][slot as usize]; if let Some(context) = context { if context.on_disk_hash == hash { matching_slot = Some(slot); @@ -678,19 +689,16 @@ impl RawInner { empty_slot = Some(slot); } } - let value = matching_slot.or(empty_slot).ok_or( - CrucibleError::IoError(format!( - "open: no slot found for {block}" - )), - )?; - active_context.push(value); - } - active_context - }; + + matching_slot.or(empty_slot).ok_or(CrucibleError::IoError( + format!("open: no slot found for {block}"), + ))? + }; + active_context.push(slot); + } let mut out = Self { file, - // Lazy initialization of which context slot is active active_context, dirty, extent_number, From e7bae58dd07e777b4fdb0a7fca6f35a651fe74da Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 19 Oct 2023 17:14:11 -0400 Subject: [PATCH 28/62] Fix indentation --- downstairs/src/extent_inner_raw.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index c5118d527..5db52f9c9 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -857,9 +857,10 @@ impl RawInner { let mut writes = 0u64; for (slot, group) in block_contexts .iter() - .group_by(|block_context| - // We'll be writing to the inactive slot - !self.active_context[block_context.block as usize]) + .group_by(|block_context| { + // We'll be writing to the inactive slot + !self.active_context[block_context.block as usize] + }) .into_iter() { let mut group = group.peekable(); From de5f1a0c1d4ff49f6f7325b739617910378a1783 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 20 Oct 2023 08:28:21 -0400 Subject: [PATCH 29/62] Encode active context in metadata region --- downstairs/src/extent_inner_raw.rs | 213 +++++++++++++++----------- downstairs/src/extent_inner_sqlite.rs | 6 + 2 files changed, 133 insertions(+), 86 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 5db52f9c9..1eb3a833a 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -62,13 +62,18 @@ const DEFRAGMENT_THRESHOLD: u64 = 3; /// - [`BLOCK_META_SIZE_BYTES`], which contains an [`OnDiskMeta`] serialized /// using `bincode`. The first byte of this range is `dirty`, serialized as a /// `u8` (where `1` is dirty and `0` is clean). -/// - Block contexts (for encryption). Each block index (in the range -/// `0..extent_size`) has two context slots; we use a ping-pong strategy when -/// writing to ensure that one slot is always valid. Each slot is -/// [`BLOCK_CONTEXT_SLOT_SIZE_BYTES`] in size, so this region is -/// `BLOCK_CONTEXT_SLOT_SIZE_BYTES * extent_size * 2` bytes in total. The -/// slots contain an `Option`, serialized using -/// `bincode`. +/// - Active context slots, stored as a bit-packed array (where 0 is +/// [`ContextSlot::A`] and 1 is [`ContextSlot::B`]). This array contains +/// `(extent_size + 7) / 8` bytes. It is only valid when the `dirty` bit is +/// cleared. This is an optimization that speeds up opening a clean extent +/// file; otherwise, we would have to rehash every block to find the active +/// context slot. +/// - Block contexts (for encryption). There are two arrays of context slots, +/// each containing `extent_size` elements (i.e. one slot for each block). +/// Each slot is [`BLOCK_CONTEXT_SLOT_SIZE_BYTES`] in size, so this section of +/// the file is `BLOCK_CONTEXT_SLOT_SIZE_BYTES * extent_size * 2` bytes in +/// total. The slots contain an `Option`, +/// serialized using `bincode`. #[derive(Debug)] pub struct RawInner { file: File, @@ -517,6 +522,7 @@ impl RawInner { let bcount = def.extent_size().value; let size = def.block_size().checked_mul(bcount).unwrap() + BLOCK_META_SIZE_BYTES + + (bcount + 7) / 8 + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2; mkdir_for_file(&path)?; @@ -571,8 +577,9 @@ impl RawInner { let extent_size = def.extent_size(); let bcount = extent_size.value; let size = def.block_size().checked_mul(bcount).unwrap() - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2 - + BLOCK_META_SIZE_BYTES; + + BLOCK_META_SIZE_BYTES + + (bcount + 7) / 8 + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2; /* * Open the extent file and verify the size is as we expect. @@ -620,84 +627,112 @@ impl RawInner { // Buffer the file so we don't spend all day waiting on syscalls let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); - // Read the metadata block - let dirty = { - let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - file_buffered.read_exact(&mut meta_buf)?; - match meta_buf[0] { - 0 => false, - 1 => true, - i => bail!("invalid dirty value: {i}"), - } + // Read the metadata block and active slots + let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + file_buffered.read_exact(&mut meta_buf)?; + let dirty = match meta_buf[0] { + 0 => false, + 1 => true, + i => bail!("invalid dirty value: {i}"), }; - // Read the two context slot arrays - let mut context_arrays = vec![]; - for _slot in [ContextSlot::A, ContextSlot::B] { - let mut contexts = Vec::with_capacity(bcount as usize); - let mut buf = vec![0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - for _block in 0..bcount as usize { - file_buffered.read_exact(&mut buf)?; - let context: Option = - bincode::deserialize(&buf).map_err(|e| { - CrucibleError::IoError(format!( - "context deserialization failed: {e}" - )) - })?; - contexts.push(context); + // If the file is dirty, then we have to recompute which context slot is + // active for every block. This is slow, but can't be avoided; we + // closed the file without a flush so we can't be confident about the + // data that was on disk. + let active_context = if !dirty { + // Easy case first: if it's **not** dirty, then just assign active + // slots based on trailing bytes in the metadata section of the file + let mut active_context = vec![]; + let mut buf = vec![0u8; (bcount as usize + 7) / 8]; + file_buffered.read_exact(&mut buf)?; + for b in buf[BLOCK_META_SIZE_BYTES as usize..].iter() { + // Unpack bits from each byte + for i in 0..8 { + active_context.push(if b & (1 << i) == 0 { + ContextSlot::A + } else { + ContextSlot::B + }); + } } - context_arrays.push(contexts); - } - - file.seek(SeekFrom::Start(0))?; - let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); - let mut active_context = vec![]; - let mut buf = vec![0; extent_size.block_size_in_bytes() as usize]; - let mut last_seek_block = 0; - for (block, (context_a, context_b)) in context_arrays[0] - .iter() - .zip(context_arrays[1].iter()) - .enumerate() - { - let slot = if context_a == context_b { - // If both slots are identical, then either they're both None or - // we have defragmented recently (which copies the active slot - // to the inactive one). That makes life easy! - ContextSlot::A - } else { - // Otherwise, we have to compute hashes from the file. - if block != last_seek_block { - file_buffered.seek_relative( - (block - last_seek_block) as i64 - * extent_size.block_size_in_bytes() as i64, - )?; + // It's possible that block count isn't a multiple of 8; in that + // case, shrink down the active context array. + assert!(bcount as usize <= active_context.len()); + active_context.resize(bcount as usize, ContextSlot::A); + active_context + } else { + // Read the two context slot arrays + let mut context_arrays = vec![]; + for _slot in [ContextSlot::A, ContextSlot::B] { + let mut contexts = Vec::with_capacity(bcount as usize); + let mut buf = vec![0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + for _block in 0..bcount as usize { + file_buffered.read_exact(&mut buf)?; + let context: Option = + bincode::deserialize(&buf).map_err(|e| { + CrucibleError::IoError(format!( + "context deserialization failed: {e}" + )) + })?; + contexts.push(context); } - file_buffered.read_exact(&mut buf)?; - last_seek_block = block + 1; // since we just read a block - let hash = integrity_hash(&[&buf]); - - let mut matching_slot = None; - let mut empty_slot = None; - - for slot in [ContextSlot::A, ContextSlot::B] { - let context = [context_a, context_b][slot as usize]; - if let Some(context) = context { - if context.on_disk_hash == hash { - matching_slot = Some(slot); + context_arrays.push(contexts); + } + + file.seek(SeekFrom::Start(0))?; + let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); + let mut active_context = vec![]; + let mut buf = vec![0; extent_size.block_size_in_bytes() as usize]; + let mut last_seek_block = 0; + for (block, (context_a, context_b)) in context_arrays[0] + .iter() + .zip(context_arrays[1].iter()) + .enumerate() + { + let slot = if context_a == context_b { + // If both slots are identical, then either they're both None or + // we have defragmented recently (which copies the active slot + // to the inactive one). That makes life easy! + ContextSlot::A + } else { + // Otherwise, we have to compute hashes from the file. + if block != last_seek_block { + file_buffered.seek_relative( + (block - last_seek_block) as i64 + * extent_size.block_size_in_bytes() as i64, + )?; + } + file_buffered.read_exact(&mut buf)?; + last_seek_block = block + 1; // since we just read a block + let hash = integrity_hash(&[&buf]); + + let mut matching_slot = None; + let mut empty_slot = None; + + for slot in [ContextSlot::A, ContextSlot::B] { + let context = [context_a, context_b][slot as usize]; + if let Some(context) = context { + if context.on_disk_hash == hash { + matching_slot = Some(slot); + } + } else if empty_slot.is_none() { + empty_slot = Some(slot); } - } else if empty_slot.is_none() { - empty_slot = Some(slot); } - } - matching_slot.or(empty_slot).ok_or(CrucibleError::IoError( - format!("open: no slot found for {block}"), - ))? - }; - active_context.push(slot); - } + matching_slot.or(empty_slot).ok_or( + CrucibleError::IoError(format!( + "open: no slot found for {block}" + )), + )? + }; + active_context.push(slot); + } + active_context + }; - let mut out = Self { + Ok(Self { file, active_context, dirty, @@ -710,11 +745,7 @@ impl RawInner { extra_syscall_count: 0, extra_syscall_denominator: 0, sync_index: 0, - }; - if !read_only { - out.defragment()?; - } - Ok(out) + }) } fn set_dirty(&mut self) -> Result<(), CrucibleError> { @@ -907,6 +938,7 @@ impl RawInner { fn context_slot_offset(&self, block: u64, slot: ContextSlot) -> u64 { self.extent_size.block_size_in_bytes() as u64 * self.extent_size.value + BLOCK_META_SIZE_BYTES + + (self.extent_size.value + 7) / 8 + (self.extent_size.value * slot as u64 + block) * BLOCK_CONTEXT_SLOT_SIZE_BYTES } @@ -930,11 +962,20 @@ impl RawInner { gen_number: new_gen, ext_version: EXTENT_META_RAW, }; - // Byte 0 is the dirty byte - let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + let mut buf = vec![0u8; BLOCK_META_SIZE_BYTES as usize]; bincode::serialize_into(buf.as_mut_slice(), &d)?; + + // Serialize bitpacked active slot values let offset = self.meta_offset(); + for c in self.active_context.chunks(8) { + let mut v = 0; + for (i, slot) in c.iter().enumerate() { + v |= (*slot as u8) << i; + } + buf.push(v); + } + nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) .map_err(|e| CrucibleError::IoError(e.to_string()))?; self.dirty = false; diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 71cecd965..8183784e0 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -548,6 +548,12 @@ impl SqliteInner { bincode::serialize_into(buf.as_mut_slice(), &meta) .map_err(|e| CrucibleError::IoError(e.to_string()))?; + // Add bitpacked data indicating which slot is active; this is always A + buf.extend( + std::iter::repeat(0) + .take((self.extent_size.value as usize + 7) / 8), + ); + // Put the context data after the metadata, all in slot A for c in ctxs { let ctx = match c.len() { From 7bad3dd8cf629fc3bb6239e93f7de1fae53a5fb6 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 20 Oct 2023 08:47:55 -0400 Subject: [PATCH 30/62] Improvements and bug fixes --- downstairs/src/extent_inner_raw.rs | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 1eb3a833a..9c7999472 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -646,7 +646,7 @@ impl RawInner { let mut active_context = vec![]; let mut buf = vec![0u8; (bcount as usize + 7) / 8]; file_buffered.read_exact(&mut buf)?; - for b in buf[BLOCK_META_SIZE_BYTES as usize..].iter() { + for b in buf { // Unpack bits from each byte for i in 0..8 { active_context.push(if b & (1 << i) == 0 { @@ -662,6 +662,10 @@ impl RawInner { active_context.resize(bcount as usize, ContextSlot::A); active_context } else { + // Skip over the active slot array, since the extent is dirty and we + // have to reload from disk. + file_buffered.seek_relative((bcount as i64 + 7) / 8)?; + // Read the two context slot arrays let mut context_arrays = vec![]; for _slot in [ContextSlot::A, ContextSlot::B] { @@ -680,6 +684,8 @@ impl RawInner { context_arrays.push(contexts); } + // Now that we've read the context slot arrays, read file data and + // figure out which context slot is active. file.seek(SeekFrom::Start(0))?; let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); let mut active_context = vec![]; @@ -690,10 +696,15 @@ impl RawInner { .zip(context_arrays[1].iter()) .enumerate() { - let slot = if context_a == context_b { - // If both slots are identical, then either they're both None or - // we have defragmented recently (which copies the active slot - // to the inactive one). That makes life easy! + let slot = if context_a.is_none() && context_b.is_none() { + // Small optimization: if both context slots are empty, the + // block must also be empty (and we don't need to read + + // hash it, saving a little time) + // + // Note that if `context_a == context_b` but they are both + // `Some(..)`, we still want to read + hash the block to + // make sure that it matches. Otherwise, someone has managed + // to corrupt our extent file on disk, which is Bad News. ContextSlot::A } else { // Otherwise, we have to compute hashes from the file. From e1e28b1d98205f596a4c9e4833ccbb3ba21791b8 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 23 Oct 2023 13:43:51 -0400 Subject: [PATCH 31/62] Move metadata to the very end of the file --- downstairs/src/extent_inner_raw.rs | 98 ++++++++++++++++----------- downstairs/src/extent_inner_sqlite.rs | 45 +++++++----- 2 files changed, 85 insertions(+), 58 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 9c7999472..715180b6c 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -59,21 +59,30 @@ const DEFRAGMENT_THRESHOLD: u64 = 3; /// /// The file is structured as follows: /// - Block data, structured as `block_size` × `extent_size` -/// - [`BLOCK_META_SIZE_BYTES`], which contains an [`OnDiskMeta`] serialized -/// using `bincode`. The first byte of this range is `dirty`, serialized as a -/// `u8` (where `1` is dirty and `0` is clean). -/// - Active context slots, stored as a bit-packed array (where 0 is -/// [`ContextSlot::A`] and 1 is [`ContextSlot::B`]). This array contains -/// `(extent_size + 7) / 8` bytes. It is only valid when the `dirty` bit is -/// cleared. This is an optimization that speeds up opening a clean extent -/// file; otherwise, we would have to rehash every block to find the active -/// context slot. /// - Block contexts (for encryption). There are two arrays of context slots, /// each containing `extent_size` elements (i.e. one slot for each block). /// Each slot is [`BLOCK_CONTEXT_SLOT_SIZE_BYTES`] in size, so this section of /// the file is `BLOCK_CONTEXT_SLOT_SIZE_BYTES * extent_size * 2` bytes in /// total. The slots contain an `Option`, /// serialized using `bincode`. +/// - Active context slots, stored as a bit-packed array (where 0 is +/// [`ContextSlot::A`] and 1 is [`ContextSlot::B`]). This array contains +/// `(extent_size + 7) / 8` bytes. It is only valid when the `dirty` bit is +/// cleared. This is an optimization that speeds up opening a clean extent +/// file; otherwise, we would have to rehash every block to find the active +/// context slot. +/// - [`BLOCK_META_SIZE_BYTES`], which contains an [`OnDiskMeta`] serialized +/// using `bincode`. The first byte of this range is `dirty`, serialized as a +/// `u8` (where `1` is dirty and `0` is clean). +/// +/// There are a few considerations that led to this particular ordering: +/// - Active context slots and metadata must be contiguous, because we want to +/// write them atomically when clearing the `dirty` flag +/// - The metadata contains an extent version (currently [`EXTENT_META_RAW`]). +/// We will eventually have multiple raw file formats, so it's convenient to +/// always place the metadata at the end; this lets us deserialize it without +/// knowing anything else about the file, then dispatch based on extent +/// version. #[derive(Debug)] pub struct RawInner { file: File, @@ -241,7 +250,7 @@ impl ExtentInner for RawInner { self.set_dirty()?; - // Write all the metadata to the raw file, at the end + // Write all the context data to the raw file // // TODO right now we're including the integrity_hash() time in the // measured time. Is it small enough to be ignored? @@ -621,15 +630,23 @@ impl RawInner { } } - // Position ourselves at the end of file data - file.seek(SeekFrom::Start(bcount * def.block_size()))?; - - // Buffer the file so we don't spend all day waiting on syscalls - let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); - - // Read the metadata block and active slots + // Read the active context bitpacked array and metadata. The former is + // only valid if `dirty` is false in the metadata, but that's the most + // common case, so we'll optimize for it. + let mut active_context_buf = vec![0u8; (bcount as usize + 7) / 8]; let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - file_buffered.read_exact(&mut meta_buf)?; + let mut iovecs = [ + IoSliceMut::new(&mut active_context_buf), + IoSliceMut::new(&mut meta_buf), + ]; + nix::sys::uio::preadv( + file.as_raw_fd(), + &mut iovecs, + (bcount * (def.block_size() + BLOCK_CONTEXT_SLOT_SIZE_BYTES * 2)) + as i64, + ) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + let dirty = match meta_buf[0] { 0 => false, 1 => true, @@ -642,11 +659,9 @@ impl RawInner { // data that was on disk. let active_context = if !dirty { // Easy case first: if it's **not** dirty, then just assign active - // slots based on trailing bytes in the metadata section of the file + // slots based on the bitpacked active context buffer from the file. let mut active_context = vec![]; - let mut buf = vec![0u8; (bcount as usize + 7) / 8]; - file_buffered.read_exact(&mut buf)?; - for b in buf { + for b in active_context_buf { // Unpack bits from each byte for i in 0..8 { active_context.push(if b & (1 << i) == 0 { @@ -662,11 +677,9 @@ impl RawInner { active_context.resize(bcount as usize, ContextSlot::A); active_context } else { - // Skip over the active slot array, since the extent is dirty and we - // have to reload from disk. - file_buffered.seek_relative((bcount as i64 + 7) / 8)?; - // Read the two context slot arrays + file.seek(SeekFrom::Start(bcount * def.block_size()))?; + let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); let mut context_arrays = vec![]; for _slot in [ContextSlot::A, ContextSlot::B] { let mut contexts = Vec::with_capacity(bcount as usize); @@ -948,8 +961,6 @@ impl RawInner { /// is always valid (i.e. matching the data in the file). fn context_slot_offset(&self, block: u64, slot: ContextSlot) -> u64 { self.extent_size.block_size_in_bytes() as u64 * self.extent_size.value - + BLOCK_META_SIZE_BYTES - + (self.extent_size.value + 7) / 8 + (self.extent_size.value * slot as u64 + block) * BLOCK_CONTEXT_SLOT_SIZE_BYTES } @@ -962,23 +973,16 @@ impl RawInner { } fn meta_offset_from_extent_size(extent_size: Block) -> u64 { - extent_size.block_size_in_bytes() as u64 * extent_size.value + (extent_size.block_size_in_bytes() as u64 + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * 2) + * extent_size.value + + (extent_size.value + 7) / 8 } /// Update the flush number, generation number, and clear the dirty bit fn set_flush_number(&mut self, new_flush: u64, new_gen: u64) -> Result<()> { - let d = OnDiskMeta { - dirty: false, - flush_number: new_flush, - gen_number: new_gen, - ext_version: EXTENT_META_RAW, - }; - let mut buf = vec![0u8; BLOCK_META_SIZE_BYTES as usize]; - - bincode::serialize_into(buf.as_mut_slice(), &d)?; - // Serialize bitpacked active slot values - let offset = self.meta_offset(); + let mut buf = vec![]; for c in self.active_context.chunks(8) { let mut v = 0; for (i, slot) in c.iter().enumerate() { @@ -987,6 +991,20 @@ impl RawInner { buf.push(v); } + let d = OnDiskMeta { + dirty: false, + flush_number: new_flush, + gen_number: new_gen, + ext_version: EXTENT_META_RAW, + }; + let mut meta = [0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(meta.as_mut_slice(), &d)?; + buf.extend(meta); + + let offset = (self.extent_size.block_size_in_bytes() as u64 + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * 2) + * self.extent_size.value; + nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) .map_err(|e| CrucibleError::IoError(e.to_string()))?; self.dirty = false; diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 8183784e0..884e69994 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -534,24 +534,11 @@ impl SqliteInner { }, }; - // Record the metadata section, which will be right after raw block data - let dirty = self.dirty()?; - let flush_number = self.flush_number()?; - let gen_number = self.gen_number()?; - let meta = OnDiskMeta { - dirty, - flush_number, - gen_number, - ext_version: EXTENT_META_RAW, // new extent version for raw files - }; - let mut buf = vec![0u8; BLOCK_META_SIZE_BYTES as usize]; - bincode::serialize_into(buf.as_mut_slice(), &meta) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - - // Add bitpacked data indicating which slot is active; this is always A - buf.extend( - std::iter::repeat(0) - .take((self.extent_size.value as usize + 7) / 8), + let block_count = ctxs.len(); + let mut buf = Vec::with_capacity( + (BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_count * 2) + + (block_count + 7) / 8 + + BLOCK_META_SIZE_BYTES as usize, ); // Put the context data after the metadata, all in slot A @@ -570,11 +557,33 @@ impl SqliteInner { .map_err(|e| CrucibleError::IoError(e.to_string()))?; buf.extend(ctx_buf); } + // Slot B is entirely empty buf.extend(std::iter::repeat(0).take( (BLOCK_CONTEXT_SLOT_SIZE_BYTES * self.extent_size.value) as usize, )); + // Add bitpacked data indicating which slot is active; this is always A + buf.extend( + std::iter::repeat(0) + .take((self.extent_size.value as usize + 7) / 8), + ); + + // Record the metadata section, which will be right after raw block data + let dirty = self.dirty()?; + let flush_number = self.flush_number()?; + let gen_number = self.gen_number()?; + let meta = OnDiskMeta { + dirty, + flush_number, + gen_number, + ext_version: EXTENT_META_RAW, // new extent version for raw files + }; + let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(meta_buf.as_mut(), &meta) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + buf.extend(meta_buf); + // Reset the file read position, just in case self.file.seek(SeekFrom::Start(0))?; Ok(buf) From e5f1d22674845783b0aecc6e1e6d6a4618ccceac Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 23 Oct 2023 16:05:58 -0400 Subject: [PATCH 32/62] Only get block contexts once (hopefully) --- downstairs/src/extent_inner_sqlite.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 884e69994..eb81068b5 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -518,13 +518,14 @@ impl SqliteInner { let ctxs = self.get_block_contexts(0, self.extent_size.value)?; let needs_rehash = ctxs.iter().any(|c| c.len() > 1); - if needs_rehash { + let ctxs = if needs_rehash { // Clean up stale hashes. After this is done, each block should // have either 0 or 1 contexts. self.fully_rehash_and_clean_all_stale_contexts(true)?; - } - - let ctxs = self.get_block_contexts(0, self.extent_size.value)?; + self.get_block_contexts(0, self.extent_size.value)? + } else { + ctxs + }; use crate::{ extent::EXTENT_META_RAW, From caa5b500f31616ce39759135da83f0fde4b7c793 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 24 Oct 2023 10:59:21 -0400 Subject: [PATCH 33/62] Only rehash if this isn't read-only --- downstairs/src/extent_inner_sqlite.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index eb81068b5..a76609090 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -868,7 +868,9 @@ impl SqliteInner { }; // Clean out any irrelevant block contexts, which may be present // if downstairs crashed between a write() and a flush(). - out.fully_rehash_and_clean_all_stale_contexts(false)?; + if !read_only { + out.fully_rehash_and_clean_all_stale_contexts(false)?; + } Ok(out) } From 3667b3ac22bb45f38592efc1197540542eb5c7a3 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 26 Oct 2023 10:05:48 -0400 Subject: [PATCH 34/62] Move file size to a function --- downstairs/src/extent_inner_raw.rs | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 715180b6c..29d9ecac3 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -528,11 +528,7 @@ impl RawInner { extent_number: u32, ) -> Result { let path = extent_path(dir, extent_number); - let bcount = def.extent_size().value; - let size = def.block_size().checked_mul(bcount).unwrap() - + BLOCK_META_SIZE_BYTES - + (bcount + 7) / 8 - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2; + let size = Self::file_size(def); mkdir_for_file(&path)?; let file = OpenOptions::new() @@ -575,6 +571,17 @@ impl RawInner { Ok(out) } + /// Returns the total size of the raw data file + /// + /// This includes block data, context slots, active slot array, and metadata + fn file_size(def: &RegionDefinition) -> u64 { + let block_count = def.extent_size().value; + def.block_size().checked_mul(block_count).unwrap() + + BLOCK_META_SIZE_BYTES + + (block_count + 7) / 8 + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count * 2 + } + /// Constructs a new `Inner` object from files that already exist on disk pub fn open( path: &Path, @@ -585,10 +592,7 @@ impl RawInner { ) -> Result { let extent_size = def.extent_size(); let bcount = extent_size.value; - let size = def.block_size().checked_mul(bcount).unwrap() - + BLOCK_META_SIZE_BYTES - + (bcount + 7) / 8 - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * bcount * 2; + let size = Self::file_size(def); /* * Open the extent file and verify the size is as we expect. From 863a1dc0b2476695c50d6eb03bf12961036a3fb2 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 26 Oct 2023 10:28:48 -0400 Subject: [PATCH 35/62] Add defragmentation test --- downstairs/src/extent_inner_raw.rs | 102 +++++++++++++++++++++++++++++ 1 file changed, 102 insertions(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 29d9ecac3..37b6a51bf 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1734,6 +1734,108 @@ mod test { Ok(()) } + #[test] + fn test_defragment() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write every other block, so that the active context slot alternates + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + for i in 0..5 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i * 2), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + inner.write(JobId(30), &[&write], false, IOV_MAX_TEST)?; + } + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + assert_eq!(inner.extra_syscall_count, 0); + assert_eq!(inner.extra_syscall_denominator, 5); + inner.flush(10, 10, JobId(10).into())?; + assert_eq!(inner.sync_index, 1); + + // This should not have changed active context slots! + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + + // Now, do one big write, which will be forced to bounce between the + // context slots. + let mut writes = vec![]; + let data = Bytes::from(vec![0x33; 512]); + let hash = integrity_hash(&[&data[..]]); + for i in 0..10 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + writes.push(write); + } + // This write has toggled every single context slot + let writes_ref: Vec<&_> = writes.iter().collect(); + inner.write(JobId(30), &writes_ref, false, IOV_MAX_TEST)?; + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 { + ContextSlot::A + } else { + ContextSlot::B + }, + "invalid context slot at {i}", + ); + } + assert!(inner.extra_syscall_count > 0); + assert_eq!(inner.extra_syscall_denominator, 1); + + // Do a flush! Because we had a bunch of extra syscalls, this should + // trigger defragmentation; after the flush, every context slot should + // be in array A. + assert_eq!(inner.sync_index, 1); + inner.flush(11, 11, JobId(11).into())?; + assert_eq!(inner.sync_index, 2); + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + ContextSlot::A, + "invalid context slot at {i}", + ); + } + + Ok(()) + } + #[test] fn test_serialized_sizes() { let c = OnDiskDownstairsBlockContext { From b84cb7892bfd5e8ff4173358e381428fa4c5d636 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 26 Oct 2023 11:29:10 -0400 Subject: [PATCH 36/62] Move migration file knowledge to RawInner --- downstairs/src/extent.rs | 11 +++- downstairs/src/extent_inner_raw.rs | 54 ++++++++++++++++++++ downstairs/src/extent_inner_sqlite.rs | 72 ++++----------------------- downstairs/src/region.rs | 14 +++++- 4 files changed, 87 insertions(+), 64 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index a8d80781a..c15dd5fae 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -362,7 +362,16 @@ impl Extent { let mut inner = extent_inner_sqlite::SqliteInner::open( &path, def, number, read_only, log, )?; - inner.export_meta_and_context()? + let ctxs = inner.export_contexts()?; + let dirty = inner.dirty()?; + let flush_number = inner.flush_number()?; + let gen_number = inner.gen_number()?; + extent_inner_raw::RawInner::import( + ctxs, + dirty, + flush_number, + gen_number, + )? }; // Append the new raw data, then sync the file to disk { diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 37b6a51bf..26d340461 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -522,6 +522,60 @@ impl ExtentInner for RawInner { } impl RawInner { + /// Imports context and metadata + /// + /// Returns a buffer that must be appended to raw block data to form the + /// full raw extent file. + pub fn import( + ctxs: Vec>, + dirty: bool, + flush_number: u64, + gen_number: u64, + ) -> Result, CrucibleError> { + let block_count = ctxs.len(); + let mut buf = Vec::with_capacity( + (BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_count * 2) + + (block_count + 7) / 8 + + BLOCK_META_SIZE_BYTES as usize, + ); + + // Contexts are stored immediately after block data in the file, so + // they're at the beginning of the supplemental data section. + for c in ctxs { + let ctx = c.map(|c| OnDiskDownstairsBlockContext { + block_context: c.block_context, + on_disk_hash: c.on_disk_hash, + }); + let mut ctx_buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; + bincode::serialize_into(ctx_buf.as_mut_slice(), &ctx) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + buf.extend(ctx_buf); + } + + // Slot B is entirely empty + buf.extend( + std::iter::repeat(0) + .take(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_count), + ); + + // Add bitpacked data indicating which slot is active; this is always A + buf.extend(std::iter::repeat(0).take((block_count + 7) / 8)); + + // Record the metadata section, which will be right after raw block data + let meta = OnDiskMeta { + dirty, + flush_number, + gen_number, + ext_version: EXTENT_META_RAW, // new extent version for raw files + }; + let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(meta_buf.as_mut(), &meta) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + buf.extend(meta_buf); + + Ok(buf) + } + pub fn create( dir: &Path, def: &RegionDefinition, diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index a76609090..f3afb5236 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -505,20 +505,20 @@ impl ExtentInner for SqliteInner { } impl SqliteInner { - /// Converts to a raw file for use with `RawInner` + /// Exports context slots for every block in the file /// - /// Returns the metadata and context slots, which should be positioned - /// directly after the raw block data in memory. - pub fn export_meta_and_context( + /// Unlike `get_block_contexts`, this function ensures that only a single + /// context per block is present (or `None`, if the block is unwritten). + pub fn export_contexts( &mut self, - ) -> Result, CrucibleError> { + ) -> Result>> { // Check whether we need to rehash. This is theoretically represented // by the `dirty` bit, but we're being _somewhat_ paranoid and manually // forcing a rehash if any blocks have multiple contexts stored. let ctxs = self.get_block_contexts(0, self.extent_size.value)?; let needs_rehash = ctxs.iter().any(|c| c.len() > 1); - let ctxs = if needs_rehash { + let mut ctxs = if needs_rehash { // Clean up stale hashes. After this is done, each block should // have either 0 or 1 contexts. self.fully_rehash_and_clean_all_stale_contexts(true)?; @@ -527,67 +527,17 @@ impl SqliteInner { ctxs }; - use crate::{ - extent::EXTENT_META_RAW, - extent_inner_raw::{ - OnDiskDownstairsBlockContext, OnDiskMeta, - BLOCK_CONTEXT_SLOT_SIZE_BYTES, BLOCK_META_SIZE_BYTES, - }, - }; - - let block_count = ctxs.len(); - let mut buf = Vec::with_capacity( - (BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_count * 2) - + (block_count + 7) / 8 - + BLOCK_META_SIZE_BYTES as usize, - ); - - // Put the context data after the metadata, all in slot A - for c in ctxs { + let mut out = vec![]; + for c in ctxs.iter_mut() { let ctx = match c.len() { 0 => None, - 1 => Some(OnDiskDownstairsBlockContext { - block_context: c[0].block_context, - on_disk_hash: c[0].on_disk_hash, - }), + 1 => c.pop(), i => panic!("invalid context count: {i}"), }; - // Put the context into the first slot, if present - let mut ctx_buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - bincode::serialize_into(ctx_buf.as_mut_slice(), &ctx) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - buf.extend(ctx_buf); + out.push(ctx); } - // Slot B is entirely empty - buf.extend(std::iter::repeat(0).take( - (BLOCK_CONTEXT_SLOT_SIZE_BYTES * self.extent_size.value) as usize, - )); - - // Add bitpacked data indicating which slot is active; this is always A - buf.extend( - std::iter::repeat(0) - .take((self.extent_size.value as usize + 7) / 8), - ); - - // Record the metadata section, which will be right after raw block data - let dirty = self.dirty()?; - let flush_number = self.flush_number()?; - let gen_number = self.gen_number()?; - let meta = OnDiskMeta { - dirty, - flush_number, - gen_number, - ext_version: EXTENT_META_RAW, // new extent version for raw files - }; - let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - bincode::serialize_into(meta_buf.as_mut(), &meta) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - buf.extend(meta_buf); - - // Reset the file read position, just in case - self.file.seek(SeekFrom::Start(0))?; - Ok(buf) + Ok(out) } fn get_block_contexts( diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index e6d02084a..78efa6549 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1869,7 +1869,7 @@ pub(crate) mod test { .await?; drop(region); - // Calculate the migration from extent 1 + // Manually calculate the migration from extent 1 let extent_file = extent_path(&dir, 1); let exported = { let mut inner = extent_inner_sqlite::SqliteInner::open( @@ -1879,7 +1879,17 @@ pub(crate) mod test { false, &log, )?; - inner.export_meta_and_context()? + use crate::extent::ExtentInner; + let ctxs = inner.export_contexts()?; + let dirty = inner.dirty()?; + let flush_number = inner.flush_number()?; + let gen_number = inner.gen_number()?; + extent_inner_raw::RawInner::import( + ctxs, + dirty, + flush_number, + gen_number, + )? }; { From ea777cac9e0411f6a2231d48fa12a44d21eab0b7 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Thu, 26 Oct 2023 15:15:19 -0400 Subject: [PATCH 37/62] Add RawLayout class to manage layout --- downstairs/src/extent.rs | 39 +- downstairs/src/extent_inner_raw.rs | 618 ++++++++++++++++------------- downstairs/src/region.rs | 155 ++++++-- 3 files changed, 492 insertions(+), 320 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index c15dd5fae..d248d57ca 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -77,7 +77,7 @@ pub(crate) trait ExtentInner: Send + Debug { } /// BlockContext, with the addition of block index and on_disk_hash -#[derive(Clone)] +#[derive(Copy, Clone)] pub struct DownstairsBlockContext { pub block_context: BlockContext, @@ -357,30 +357,29 @@ impl Extent { OpenOptions::new().read(true).write(true).open(&path)?; f.set_len(def.extent_size().value * def.block_size())?; } - // Compute metadata and context slots - let meta_and_context = { - let mut inner = extent_inner_sqlite::SqliteInner::open( - &path, def, number, read_only, log, - )?; - let ctxs = inner.export_contexts()?; - let dirty = inner.dirty()?; - let flush_number = inner.flush_number()?; - let gen_number = inner.gen_number()?; + + // Compute supplemental data from the SQLite extent + let mut inner = extent_inner_sqlite::SqliteInner::open( + &path, def, number, read_only, log, + )?; + let ctxs = inner.export_contexts()?; + let dirty = inner.dirty()?; + let flush_number = inner.flush_number()?; + let gen_number = inner.gen_number()?; + drop(inner); + + // Reopen the file and import those changes + { + let mut f = + OpenOptions::new().read(true).write(true).open(&path)?; extent_inner_raw::RawInner::import( + &mut f, + def, ctxs, dirty, flush_number, gen_number, - )? - }; - // Append the new raw data, then sync the file to disk - { - let mut f = OpenOptions::new() - .read(true) - .write(true) - .append(true) - .open(&path)?; - f.write_all(&meta_and_context)?; + )?; f.sync_all() .with_context(|| format!("{path:?}: fsync failure"))?; } diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 26d340461..5f210e547 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -17,7 +17,7 @@ use slog::{error, Logger}; use std::collections::HashSet; use std::fs::{File, OpenOptions}; -use std::io::{BufReader, IoSliceMut, Read, Seek, SeekFrom}; +use std::io::{BufReader, IoSliceMut, Read}; use std::os::fd::AsRawFd; use std::path::Path; @@ -93,6 +93,9 @@ pub struct RawInner { /// Extent size, in blocks extent_size: Block, + /// Helper `struct` controlling layout within the file + layout: RawLayout, + /// Is the `A` or `B` context slot active, on a per-block basis? active_context: Vec, @@ -504,7 +507,7 @@ impl ExtentInner for RawInner { block_context: &DownstairsBlockContext, ) -> Result<(), CrucibleError> { self.set_dirty()?; - self.set_block_contexts(&[block_context.clone()])?; + self.set_block_contexts(&[*block_context])?; self.active_context[block_context.block as usize] = !self.active_context[block_context.block as usize]; Ok(()) @@ -527,53 +530,39 @@ impl RawInner { /// Returns a buffer that must be appended to raw block data to form the /// full raw extent file. pub fn import( + file: &mut File, + def: &RegionDefinition, ctxs: Vec>, dirty: bool, flush_number: u64, gen_number: u64, - ) -> Result, CrucibleError> { - let block_count = ctxs.len(); - let mut buf = Vec::with_capacity( - (BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_count * 2) - + (block_count + 7) / 8 - + BLOCK_META_SIZE_BYTES as usize, - ); - - // Contexts are stored immediately after block data in the file, so - // they're at the beginning of the supplemental data section. - for c in ctxs { - let ctx = c.map(|c| OnDiskDownstairsBlockContext { - block_context: c.block_context, - on_disk_hash: c.on_disk_hash, - }); - let mut ctx_buf = [0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - bincode::serialize_into(ctx_buf.as_mut_slice(), &ctx) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - buf.extend(ctx_buf); - } - - // Slot B is entirely empty - buf.extend( - std::iter::repeat(0) - .take(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize * block_count), - ); + ) -> Result<(), CrucibleError> { + let layout = RawLayout::new(def.extent_size()); + let block_count = layout.block_count() as usize; - // Add bitpacked data indicating which slot is active; this is always A - buf.extend(std::iter::repeat(0).take((block_count + 7) / 8)); + file.set_len(layout.file_size())?; + layout.write_context_slots_contiguous( + file, + 0, + ctxs.iter().map(Option::as_ref), + ContextSlot::A, + )?; + layout.write_context_slots_contiguous( + file, + 0, + std::iter::repeat(None).take(block_count), + ContextSlot::B, + )?; - // Record the metadata section, which will be right after raw block data - let meta = OnDiskMeta { + layout.write_active_context_and_metadata( + file, + vec![ContextSlot::A; block_count].as_slice(), dirty, flush_number, gen_number, - ext_version: EXTENT_META_RAW, // new extent version for raw files - }; - let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - bincode::serialize_into(meta_buf.as_mut(), &meta) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - buf.extend(meta_buf); + )?; - Ok(buf) + Ok(()) } pub fn create( @@ -582,7 +571,9 @@ impl RawInner { extent_number: u32, ) -> Result { let path = extent_path(dir, extent_number); - let size = Self::file_size(def); + let extent_size = def.extent_size(); + let layout = RawLayout::new(extent_size); + let size = layout.file_size(); mkdir_for_file(&path)?; let file = OpenOptions::new() @@ -596,7 +587,8 @@ impl RawInner { let mut out = Self { file, dirty: false, - extent_size: def.extent_size(), + extent_size, + layout, extent_number, active_context: vec![ ContextSlot::A; // both slots are empty, so this is fine @@ -625,17 +617,6 @@ impl RawInner { Ok(out) } - /// Returns the total size of the raw data file - /// - /// This includes block data, context slots, active slot array, and metadata - fn file_size(def: &RegionDefinition) -> u64 { - let block_count = def.extent_size().value; - def.block_size().checked_mul(block_count).unwrap() - + BLOCK_META_SIZE_BYTES - + (block_count + 7) / 8 - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count * 2 - } - /// Constructs a new `Inner` object from files that already exist on disk pub fn open( path: &Path, @@ -645,13 +626,13 @@ impl RawInner { log: &Logger, ) -> Result { let extent_size = def.extent_size(); - let bcount = extent_size.value; - let size = Self::file_size(def); + let layout = RawLayout::new(extent_size); + let size = layout.file_size(); /* * Open the extent file and verify the size is as we expect. */ - let mut file = + let file = match OpenOptions::new().read(true).write(!read_only).open(path) { Err(e) => { error!( @@ -688,84 +669,41 @@ impl RawInner { } } - // Read the active context bitpacked array and metadata. The former is - // only valid if `dirty` is false in the metadata, but that's the most - // common case, so we'll optimize for it. - let mut active_context_buf = vec![0u8; (bcount as usize + 7) / 8]; - let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - let mut iovecs = [ - IoSliceMut::new(&mut active_context_buf), - IoSliceMut::new(&mut meta_buf), - ]; - nix::sys::uio::preadv( - file.as_raw_fd(), - &mut iovecs, - (bcount * (def.block_size() + BLOCK_CONTEXT_SLOT_SIZE_BYTES * 2)) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - - let dirty = match meta_buf[0] { - 0 => false, - 1 => true, - i => bail!("invalid dirty value: {i}"), - }; + let layout = RawLayout::new(def.extent_size()); + let meta = layout.get_metadata(&file)?; // If the file is dirty, then we have to recompute which context slot is // active for every block. This is slow, but can't be avoided; we // closed the file without a flush so we can't be confident about the // data that was on disk. - let active_context = if !dirty { + let active_context = if !meta.dirty { // Easy case first: if it's **not** dirty, then just assign active // slots based on the bitpacked active context buffer from the file. - let mut active_context = vec![]; - for b in active_context_buf { - // Unpack bits from each byte - for i in 0..8 { - active_context.push(if b & (1 << i) == 0 { - ContextSlot::A - } else { - ContextSlot::B - }); - } - } - // It's possible that block count isn't a multiple of 8; in that - // case, shrink down the active context array. - assert!(bcount as usize <= active_context.len()); - active_context.resize(bcount as usize, ContextSlot::A); - active_context + layout.get_active_contexts(&file)? } else { - // Read the two context slot arrays - file.seek(SeekFrom::Start(bcount * def.block_size()))?; - let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); - let mut context_arrays = vec![]; - for _slot in [ContextSlot::A, ContextSlot::B] { - let mut contexts = Vec::with_capacity(bcount as usize); - let mut buf = vec![0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - for _block in 0..bcount as usize { - file_buffered.read_exact(&mut buf)?; - let context: Option = - bincode::deserialize(&buf).map_err(|e| { - CrucibleError::IoError(format!( - "context deserialization failed: {e}" - )) - })?; - contexts.push(context); - } - context_arrays.push(contexts); - } + // Otherwise, read block-size chunks and check hashes against + // both context slots, looking for a match. + let ctx_a = layout.read_context_slots_contiguous( + &file, + 0, + layout.block_count(), + ContextSlot::A, + )?; + let ctx_b = layout.read_context_slots_contiguous( + &file, + 0, + layout.block_count(), + ContextSlot::B, + )?; // Now that we've read the context slot arrays, read file data and // figure out which context slot is active. - file.seek(SeekFrom::Start(0))?; let mut file_buffered = BufReader::with_capacity(64 * 1024, &file); let mut active_context = vec![]; let mut buf = vec![0; extent_size.block_size_in_bytes() as usize]; let mut last_seek_block = 0; - for (block, (context_a, context_b)) in context_arrays[0] - .iter() - .zip(context_arrays[1].iter()) - .enumerate() + for (block, (context_a, context_b)) in + ctx_a.into_iter().zip(ctx_b).enumerate() { let slot = if context_a.is_none() && context_b.is_none() { // Small optimization: if both context slots are empty, the @@ -817,9 +755,10 @@ impl RawInner { Ok(Self { file, active_context, - dirty, + dirty: meta.dirty, extent_number, extent_size: def.extent_size(), + layout: RawLayout::new(def.extent_size()), context_slot_synched_at: vec![ [0, 0]; def.extent_size().value as usize @@ -832,9 +771,7 @@ impl RawInner { fn set_dirty(&mut self) -> Result<(), CrucibleError> { if !self.dirty { - let offset = self.meta_offset(); - nix::sys::uio::pwrite(self.file.as_raw_fd(), &[1u8], offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + self.layout.set_dirty(&self.file)?; self.dirty = true; } Ok(()) @@ -866,22 +803,16 @@ impl RawInner { // Then, read the slot data and decide if either slot // (1) is present and // (2) has a matching hash - let mut buf = [0; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; let mut matching_slot = None; let mut empty_slot = None; for slot in [ContextSlot::A, ContextSlot::B] { - nix::sys::uio::pread( - self.file.as_raw_fd(), - &mut buf, - self.context_slot_offset(block, slot) as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - let context: Option = - bincode::deserialize(&buf).map_err(|e| { - CrucibleError::IoError(format!( - "context deserialization failed: {e:?}" - )) - })?; + // Read a single context slot, which is by definition contiguous + let mut context = self + .layout + .read_context_slots_contiguous(&self.file, block, 1, slot)?; + assert_eq!(context.len(), 1); + let context = context.pop().unwrap(); + if let Some(context) = context { if context.on_disk_hash == hash { matching_slot = Some(slot); @@ -966,7 +897,6 @@ impl RawInner { assert_eq!(a.block + 1, b.block, "blocks must be contiguous"); } - let mut buf = vec![]; let mut writes = 0u64; for (slot, group) in block_contexts .iter() @@ -977,20 +907,13 @@ impl RawInner { .into_iter() { let mut group = group.peekable(); - let block_start = group.peek().unwrap().block; - buf.clear(); - for block_context in group { - let n = buf.len(); - buf.extend([0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]); - let d = OnDiskDownstairsBlockContext { - block_context: block_context.block_context, - on_disk_hash: block_context.on_disk_hash, - }; - bincode::serialize_into(&mut buf[n..], &Some(d))?; - } - let offset = self.context_slot_offset(block_start, slot); - nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + let start = group.peek().unwrap().block; + self.layout.write_context_slots_contiguous( + &self.file, + start, + group.map(Option::Some), + slot, + )?; writes += 1; } if let Some(writes) = writes.checked_sub(1) { @@ -1002,71 +925,19 @@ impl RawInner { } fn get_metadata(&self) -> Result { - let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; - let offset = self.meta_offset(); - nix::sys::uio::pread(self.file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - let out: OnDiskMeta = bincode::deserialize(&buf) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - Ok(out) - } - - /// Returns the byte offset of the given context slot - /// - /// Contexts slots are located after block and meta data in the extent file. - /// There are two context slots arrays, each of which contains one context - /// slot per block. We use a ping-pong strategy to ensure that one of them - /// is always valid (i.e. matching the data in the file). - fn context_slot_offset(&self, block: u64, slot: ContextSlot) -> u64 { - self.extent_size.block_size_in_bytes() as u64 * self.extent_size.value - + (self.extent_size.value * slot as u64 + block) - * BLOCK_CONTEXT_SLOT_SIZE_BYTES - } - - /// Returns the byte offset of the metadata region - /// - /// The resulting offset points to serialized [`OnDiskMeta`] data. - fn meta_offset(&self) -> u64 { - Self::meta_offset_from_extent_size(self.extent_size) - } - - fn meta_offset_from_extent_size(extent_size: Block) -> u64 { - (extent_size.block_size_in_bytes() as u64 - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * 2) - * extent_size.value - + (extent_size.value + 7) / 8 + self.layout.get_metadata(&self.file) } /// Update the flush number, generation number, and clear the dirty bit fn set_flush_number(&mut self, new_flush: u64, new_gen: u64) -> Result<()> { - // Serialize bitpacked active slot values - let mut buf = vec![]; - for c in self.active_context.chunks(8) { - let mut v = 0; - for (i, slot) in c.iter().enumerate() { - v |= (*slot as u8) << i; - } - buf.push(v); - } - - let d = OnDiskMeta { - dirty: false, - flush_number: new_flush, - gen_number: new_gen, - ext_version: EXTENT_META_RAW, - }; - let mut meta = [0u8; BLOCK_META_SIZE_BYTES as usize]; - bincode::serialize_into(meta.as_mut_slice(), &d)?; - buf.extend(meta); - - let offset = (self.extent_size.block_size_in_bytes() as u64 - + BLOCK_CONTEXT_SLOT_SIZE_BYTES * 2) - * self.extent_size.value; - - nix::sys::uio::pwrite(self.file.as_raw_fd(), &buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + self.layout.write_active_context_and_metadata( + &self.file, + &self.active_context, + false, // dirty + new_flush, + new_gen, + )?; self.dirty = false; - Ok(()) } @@ -1077,34 +948,20 @@ impl RawInner { count: u64, ) -> Result>> { let mut out = vec![]; - let mut buf = vec![]; let mut reads = 0u64; - for (slot, mut group) in (block..block + count) + for (slot, group) in (block..block + count) .group_by(|block| self.active_context[*block as usize]) .into_iter() { - let group_start = group.next().unwrap(); - let group_count = group.count() + 1; - buf.resize(group_count * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize, 0); - let offset = self.context_slot_offset(group_start, slot); - nix::sys::uio::pread( - self.file.as_raw_fd(), - &mut buf, - offset as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - for (i, chunk) in buf - .chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) - .enumerate() - { - let ctx: Option = - bincode::deserialize(chunk)?; - out.push(ctx.map(|c| DownstairsBlockContext { - block: group_start + i as u64, - block_context: c.block_context, - on_disk_hash: c.on_disk_hash, - })); - } + let mut group = group.peekable(); + let start = *group.peek().unwrap(); + let count = group.count(); + out.extend(self.layout.read_context_slots_contiguous( + &self.file, + start, + count as u64, + slot, + )?); reads += 1; } if let Some(reads) = reads.checked_sub(1) { @@ -1190,12 +1047,12 @@ impl RawInner { #[derive(Copy, Clone)] struct Counter { count: usize, - min_block: usize, - max_block: usize, + min_block: u64, + max_block: u64, } let mut a_count = Counter { count: 0, - min_block: usize::MAX, + min_block: u64::MAX, max_block: 0, }; let mut b_count = a_count; @@ -1205,8 +1062,8 @@ impl RawInner { ContextSlot::B => &mut b_count, }; count.count += 1; - count.min_block = count.min_block.min(i); - count.max_block = count.max_block.max(i); + count.min_block = count.min_block.min(i as u64); + count.max_block = count.max_block.max(i as u64); } if a_count.count == 0 || b_count.count == 0 { return Ok(()); @@ -1222,33 +1079,24 @@ impl RawInner { let num_slots = counter.max_block + 1 - counter.min_block; // Read the source context slots from the file - let mut source_buf = - vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - nix::sys::uio::pread( - self.file.as_raw_fd(), - &mut source_buf, - self.context_slot_offset(counter.min_block as u64, copy_from) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; - - // Read the destination context slots from the file - let mut dest_buf = - vec![0u8; num_slots * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]; - nix::sys::uio::pread( - self.file.as_raw_fd(), - &mut dest_buf, - self.context_slot_offset(counter.min_block as u64, !copy_from) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + let source_slots = self.layout.read_context_slots_contiguous( + &self.file, + counter.min_block, + num_slots, + copy_from, + )?; + let mut dest_slots = self.layout.read_context_slots_contiguous( + &self.file, + counter.min_block, + num_slots, + !copy_from, + )?; // Selectively overwrite dest with source context slots for (i, block) in (counter.min_block..=counter.max_block).enumerate() { + let block = block as usize; if self.active_context[block] == copy_from { - let chunk = (i * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) - ..((i + 1) * BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize); - dest_buf[chunk.clone()].copy_from_slice(&source_buf[chunk]); + dest_slots[i] = source_slots[i]; // Mark this slot as unsynched, so that we don't overwrite // it later on without a sync @@ -1256,29 +1104,257 @@ impl RawInner { self.sync_index + 1; } } - let r = nix::sys::uio::pwrite( - self.file.as_raw_fd(), - &dest_buf, - self.context_slot_offset(counter.min_block as u64, !copy_from) - as i64, - ) - .map_err(|e| CrucibleError::IoError(e.to_string())); + let r = self.layout.write_context_slots_contiguous( + &self.file, + counter.min_block, + dest_slots.iter().map(|v| v.as_ref()), + !copy_from, + ); // If this write failed, then try recomputing every context slot // within the unknown range if r.is_err() { for block in counter.min_block..=counter.max_block { - self.recompute_slot_from_file(block as u64).unwrap(); + self.recompute_slot_from_file(block).unwrap(); } } else { for block in counter.min_block..=counter.max_block { - self.active_context[block] = !copy_from; + self.active_context[block as usize] = !copy_from; } } r.map(|_| ()) } } +/// Data structure that implements the on-disk layout of a raw extent file +struct RawLayout { + extent_size: Block, + + /// Miscellaneous buffer for reading and writing + /// + /// This is simply to avoid churning through memory allocations. It is the + /// user's responsibility to take `buf` out of the cell when it's in use, + /// and return it when they're done with it. + buf: std::cell::Cell>, +} + +impl std::fmt::Debug for RawLayout { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("RawLayout") + .field("extent_size", &self.extent_size) + .finish() + } +} + +impl RawLayout { + fn new(extent_size: Block) -> Self { + RawLayout { + extent_size, + buf: std::cell::Cell::default(), + } + } + + fn set_dirty(&self, file: &File) -> Result<(), CrucibleError> { + let offset = self.metadata_offset(); + nix::sys::uio::pwrite(file.as_raw_fd(), &[1u8], offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + Ok(()) + } + + /// Returns the total size of the raw data file + /// + /// This includes block data, context slots, active slot array, and metadata + fn file_size(&self) -> u64 { + let block_count = self.block_count(); + self.block_size().checked_mul(block_count).unwrap() + + BLOCK_META_SIZE_BYTES + + (block_count + 7) / 8 + + BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count * 2 + } + + /// Returns the beginning of supplementary data in the file + fn supplementary_data_offset(&self) -> u64 { + self.block_count() * self.block_size() + } + + /// Returns the byte offset of the given context slot in the file + /// + /// Contexts slots are located after block data in the extent file. There + /// are two context slots arrays, each of which contains one context slot + /// per block. We use a ping-pong strategy to ensure that one of them is + /// always valid (i.e. matching the data in the file). + fn context_slot_offset(&self, block: u64, slot: ContextSlot) -> u64 { + self.supplementary_data_offset() + + (self.block_count() * slot as u64 + block) + * BLOCK_CONTEXT_SLOT_SIZE_BYTES + } + + /// Number of blocks in the extent file + fn block_count(&self) -> u64 { + self.extent_size.value + } + + /// Returns the byte offset of the `active_context` bitpacked array + fn active_context_offset(&self) -> u64 { + self.supplementary_data_offset() + + self.block_count() * 2 * BLOCK_CONTEXT_SLOT_SIZE_BYTES + } + + fn active_context_size(&self) -> u64 { + (self.block_count() + 7) / 8 + } + + fn metadata_offset(&self) -> u64 { + self.active_context_offset() + self.active_context_size() + } + + /// Number of bytes in each block + fn block_size(&self) -> u64 { + self.extent_size.block_size_in_bytes() as u64 + } + + fn get_metadata(&self, file: &File) -> Result { + let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; + let offset = self.metadata_offset(); + nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + let out: OnDiskMeta = bincode::deserialize(&buf) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + Ok(out) + } + + fn write_context_slots_contiguous<'a, I>( + &self, + file: &File, + block_start: u64, + iter: I, + slot: ContextSlot, + ) -> Result<(), CrucibleError> + where + I: Iterator>, + { + let mut buf = self.buf.take(); + buf.clear(); + + for block_context in iter { + let n = buf.len(); + buf.extend([0u8; BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize]); + let d = block_context.map(|b| OnDiskDownstairsBlockContext { + block_context: b.block_context, + on_disk_hash: b.on_disk_hash, + }); + bincode::serialize_into(&mut buf[n..], &d).map_err(|e| { + CrucibleError::IoError(format!( + "could not serialize context: {e}" + )) + })?; + } + let offset = self.context_slot_offset(block_start, slot); + nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + self.buf.set(buf); + Ok(()) + } + + fn read_context_slots_contiguous( + &self, + file: &File, + block_start: u64, + block_count: u64, + slot: ContextSlot, + ) -> Result>> { + let mut buf = self.buf.take(); + buf.resize((BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count) as usize, 0u8); + + let offset = self.context_slot_offset(block_start, slot); + nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + + let mut out = vec![]; + for (i, chunk) in buf + .chunks_exact(BLOCK_CONTEXT_SLOT_SIZE_BYTES as usize) + .enumerate() + { + let ctx: Option = + bincode::deserialize(chunk)?; + out.push(ctx.map(|c| DownstairsBlockContext { + block: block_start + i as u64, + block_context: c.block_context, + on_disk_hash: c.on_disk_hash, + })); + } + self.buf.set(buf); + Ok(out) + } + + /// Write out the active context array and metadata section of the file + /// + /// This is done in a single write, so it should be atomic. + fn write_active_context_and_metadata( + &self, + file: &File, + active_context: &[ContextSlot], + dirty: bool, + flush_number: u64, + gen_number: u64, + ) -> Result<()> { + // Serialize bitpacked active slot values + let mut buf = self.buf.take(); + buf.clear(); + for c in active_context.chunks(8) { + let mut v = 0; + for (i, slot) in c.iter().enumerate() { + v |= (*slot as u8) << i; + } + buf.push(v); + } + + let d = OnDiskMeta { + dirty, + flush_number, + gen_number, + ext_version: EXTENT_META_RAW, + }; + let mut meta = [0u8; BLOCK_META_SIZE_BYTES as usize]; + bincode::serialize_into(meta.as_mut_slice(), &d)?; + buf.extend(meta); + + let offset = self.active_context_offset(); + + nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64) + .map_err(|e| CrucibleError::IoError(e.to_string()))?; + self.buf.set(buf); + + Ok(()) + } + + /// Decodes the active contexts from the given file + /// + /// The file descriptor offset is not changed by this function + fn get_active_contexts(&self, file: &File) -> Result> { + let mut buf = self.buf.take(); + buf.resize(self.active_context_size() as usize, 0u8); + let offset = self.active_context_offset(); + nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64)?; + + let mut active_context = vec![]; + for bit in buf + .iter() + .flat_map(|b| (0..8).map(move |i| b & (1 << i) == 0)) + .take(self.block_count() as usize) + { + // Unpack bits from each byte + active_context.push(if bit { + ContextSlot::A + } else { + ContextSlot::B + }); + } + assert_eq!(active_context.len(), self.block_count() as usize); + Ok(active_context) + } +} + #[cfg(test)] mod test { use super::*; diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 78efa6549..7890ac9ea 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1838,9 +1838,6 @@ pub(crate) mod test { region.extend(3).await?; let ddef = region.def(); - // Now, we're going to mess with the file on disk a little, to make sure - // that the migration happens from the old file (which we just exported) - // and not from the modified database. region .region_write( &[ @@ -1871,34 +1868,134 @@ pub(crate) mod test { // Manually calculate the migration from extent 1 let extent_file = extent_path(&dir, 1); - let exported = { - let mut inner = extent_inner_sqlite::SqliteInner::open( - &extent_file, - &ddef, - 1, + let mut inner = extent_inner_sqlite::SqliteInner::open( + &extent_file, + &ddef, + 1, + false, + &log, + )?; + use crate::extent::ExtentInner; + let ctxs = inner.export_contexts()?; + let dirty = inner.dirty()?; + let flush_number = inner.flush_number()?; + let gen_number = inner.gen_number()?; + drop(inner); + + let mut file = OpenOptions::new() + .read(true) + .write(true) + .open(&extent_file)?; + println!("about to call `import` on {:?}", extent_file); + extent_inner_raw::RawInner::import( + &mut file, + &ddef, + ctxs, + dirty, + flush_number, + gen_number, + )?; + println!("dooone"); + // At this point, we have manually written the file, but have not + // deleted the `.db` on disk. As such, migration should restart when + // the extent is reopened. + + let region = + Region::open(&dir, new_region_options(), true, false, &log).await?; + let out = region + .region_read( + &[ + ReadRequest { + eid: 1, + offset: Block::new_512(0), + }, + ReadRequest { + eid: 2, + offset: Block::new_512(0), + }, + ], + JobId(0), + ) + .await?; + assert_eq!(out[0].data.as_ref(), [1; 512]); + assert_eq!(out[1].data.as_ref(), [2; 512]); + + Ok(()) + } + + #[tokio::test] + async fn reopen_extent_partial_migration_corrupt() -> Result<()> { + let log = csl(); + let dir = tempdir()?; + let mut region = + Region::create_sqlite(&dir, new_region_options(), log.clone()) + .await?; + region.extend(3).await?; + let ddef = region.def(); + + // Make some writes, which we'll check after migration + region + .region_write( + &[ + crucible_protocol::Write { + eid: 1, + offset: Block::new_512(0), + data: Bytes::from(vec![1u8; 512]), + block_context: BlockContext { + encryption_context: None, + hash: 8717892996238908351, // hash for all 1s + }, + }, + crucible_protocol::Write { + eid: 2, + offset: Block::new_512(0), + data: Bytes::from(vec![2u8; 512]), + block_context: BlockContext { + encryption_context: None, + hash: 2192425179333611943, // hash for all 2s + }, + }, + ], + JobId(0), false, - &log, - )?; - use crate::extent::ExtentInner; - let ctxs = inner.export_contexts()?; - let dirty = inner.dirty()?; - let flush_number = inner.flush_number()?; - let gen_number = inner.gen_number()?; - extent_inner_raw::RawInner::import( - ctxs, - dirty, - flush_number, - gen_number, - )? - }; + ) + .await?; + drop(region); - { - let mut f = OpenOptions::new() - .write(true) - .append(true) - .open(&extent_file)?; - f.write_all(&exported)?; - } + // Manually calculate the migration from extent 1, but deliberately mess + // with the context values (simulating a migration that didn't manage to + // write everything to disk). + let extent_file = extent_path(&dir, 1); + let mut inner = extent_inner_sqlite::SqliteInner::open( + &extent_file, + &ddef, + 1, + false, + &log, + )?; + use crate::extent::ExtentInner; + let ctxs = inner.export_contexts()?.into_iter().map(|_| None).collect(); + let dirty = inner.dirty()?; + let flush_number = inner.flush_number()?; + let gen_number = inner.gen_number()?; + drop(inner); + + // Stage the corrupted migration + let mut file = OpenOptions::new() + .read(true) + .write(true) + .open(&extent_file)?; + extent_inner_raw::RawInner::import( + &mut file, + &ddef, + ctxs, + dirty, + flush_number, + gen_number, + )?; + // At this point, we have manually written the file, but have not + // deleted the `.db` on disk. As such, migration should restart when + // the extent is reopened, and we should recover from corruption. let region = Region::open(&dir, new_region_options(), true, false, &log).await?; From c3ed14d89687606928f659c9dcf285421f6258d3 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 09:41:42 -0400 Subject: [PATCH 38/62] Add docstring --- downstairs/src/extent_inner_raw.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 5f210e547..b69574d1b 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1154,6 +1154,11 @@ impl RawLayout { } } + /// Sets the dirty flag in the file true + /// + /// This unconditionally writes to the file; to avoid extra syscalls, it + /// would be wise to cache this at a higher level and only write if it has + /// changed. fn set_dirty(&self, file: &File) -> Result<(), CrucibleError> { let offset = self.metadata_offset(); nix::sys::uio::pwrite(file.as_raw_fd(), &[1u8], offset as i64) From 9642e66eb54071c235354ffbafa7fb62a793166d Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 11:12:05 -0400 Subject: [PATCH 39/62] Add support for alternate backends to integration tests --- common/src/region.rs | 2 +- downstairs/Cargo.toml | 1 + downstairs/src/extent.rs | 4 +- downstairs/src/extent_inner_sqlite.rs | 2 +- downstairs/src/lib.rs | 81 ++++++++++++++++-- downstairs/src/region.rs | 113 ++++++++++++++++---------- integration_tests/Cargo.toml | 2 +- integration_tests/src/lib.rs | 17 +++- 8 files changed, 165 insertions(+), 57 deletions(-) diff --git a/common/src/region.rs b/common/src/region.rs index 1595d48cd..46c2f4371 100644 --- a/common/src/region.rs +++ b/common/src/region.rs @@ -274,7 +274,7 @@ impl RegionDefinition { } } -#[derive(Deserialize, Serialize, Clone, Debug, PartialEq)] +#[derive(Serialize, Clone, Debug, PartialEq)] pub struct RegionOptions { /** * The size of each block in bytes. Must be a power of 2, minimum 512. diff --git a/downstairs/Cargo.toml b/downstairs/Cargo.toml index e938a38a1..2cc93866e 100644 --- a/downstairs/Cargo.toml +++ b/downstairs/Cargo.toml @@ -70,3 +70,4 @@ version_check = "0.9.4" asm = ["usdt/asm"] default = [] zfs_snapshot = [] +integration-tests = [] # Enables creating SQLite volumes diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index d248d57ca..e14792efc 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -125,7 +125,7 @@ pub struct ExtentMeta { /// This is no longer used when creating new extents, but we support opening /// existing SQLite-based extents because snapshot images are on read-only /// volumes, so we can't migrate them. -#[cfg(test)] // This constant is only actually used in unit tests +#[cfg(any(test, feature = "integration-tests"))] pub const EXTENT_META_SQLITE: u32 = 1; /// Extent version for raw-file-backed metadata @@ -497,7 +497,7 @@ impl Extent { /// Identical to `create`, but using the SQLite backend /// /// This is only allowed in unit tests - #[cfg(test)] + #[cfg(any(test, feature = "integration-tests"))] pub fn create_sqlite( dir: &Path, def: &RegionDefinition, diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index f3afb5236..3638623ae 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -593,7 +593,7 @@ impl SqliteInner { // We should never create a new SQLite-backed extent in production code, // because we should be using raw extents everywhere. However, we'll create // them during tests to check that our automatic migration system works. - #[cfg(test)] + #[cfg(any(test, feature = "integration-tests"))] pub fn create( dir: &Path, def: &RegionDefinition, diff --git a/downstairs/src/lib.rs b/downstairs/src/lib.rs index cd833a077..2cd421568 100644 --- a/downstairs/src/lib.rs +++ b/downstairs/src/lib.rs @@ -3030,6 +3030,18 @@ enum WrappedStream { Https(tokio_rustls::server::TlsStream), } +/// On-disk backend for downstairs storage +/// +/// Normally, we only allow the most recent backend. However, for integration +/// tests, it can be useful to create volumes using older backends. +#[derive(Copy, Clone, Debug, PartialEq)] +pub enum Backend { + RawFile, + + #[cfg(any(test, feature = "integration-tests"))] + SQLite, +} + pub async fn create_region( block_size: u64, data: PathBuf, @@ -3038,41 +3050,96 @@ pub async fn create_region( uuid: Uuid, encrypted: bool, log: Logger, +) -> Result { + create_region_with_backend( + data, + Block { + value: extent_size, + shift: block_size.trailing_zeros(), + }, + extent_count, + uuid, + encrypted, + Backend::RawFile, + log, + ) + .await +} + +pub async fn create_region_with_backend( + data: PathBuf, + extent_size: Block, + extent_count: u64, + uuid: Uuid, + encrypted: bool, + backend: Backend, + log: Logger, ) -> Result { /* * Create the region options, then the region. */ let mut region_options: crucible_common::RegionOptions = Default::default(); - region_options.set_block_size(block_size); - region_options - .set_extent_size(Block::new(extent_size, block_size.trailing_zeros())); + region_options.set_block_size(extent_size.block_size_in_bytes().into()); + region_options.set_extent_size(extent_size); region_options.set_uuid(uuid); region_options.set_encrypted(encrypted); - let mut region = Region::create(data, region_options, log).await?; + let mut region = + Region::create_with_backend(data, region_options, backend, log).await?; region.extend(extent_count as u32).await?; Ok(region) } +pub async fn build_downstairs_for_region( + data: &Path, + lossy: bool, + read_errors: bool, + write_errors: bool, + flush_errors: bool, + read_only: bool, + log_request: Option, +) -> Result>> { + build_downstairs_for_region_with_backend( + data, + lossy, + read_errors, + write_errors, + flush_errors, + read_only, + Backend::RawFile, + log_request, + ) + .await +} + // Build the downstairs struct given a region directory and some additional // needed information. If a logger is passed in, we will use that, otherwise // a logger will be created. -pub async fn build_downstairs_for_region( +#[allow(clippy::too_many_arguments)] +pub async fn build_downstairs_for_region_with_backend( data: &Path, lossy: bool, read_errors: bool, write_errors: bool, flush_errors: bool, read_only: bool, + backend: Backend, log_request: Option, ) -> Result>> { let log = match log_request { Some(log) => log, None => build_logger(), }; - let region = - Region::open(data, Default::default(), true, read_only, &log).await?; + let region = Region::open_with_backend( + data, + Default::default(), + true, + read_only, + backend, + &log, + ) + .await?; info!(log, "UUID: {:?}", region.def().uuid()); info!( diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 7890ac9ea..9c7991cdf 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -90,7 +90,7 @@ pub struct Region { /// Use the SQLite extent backend when creating new extents /// /// This is only allowed in tests; all new extents must be raw files - #[cfg(test)] + #[cfg(any(test, feature = "integration-tests"))] prefer_sqlite_backend: bool, } @@ -188,14 +188,13 @@ impl Region { options: RegionOptions, log: Logger, ) -> Result { - let mut region = Self::create_inner(dir, options, log)?; - region.open_extents(true).await?; - Ok(region) + Self::create_with_backend(dir, options, Backend::RawFile, log).await } - pub fn create_inner>( + pub async fn create_with_backend>( dir: P, options: RegionOptions, + backend: Backend, log: Logger, ) -> Result { options.validate()?; @@ -217,42 +216,47 @@ impl Region { write_json(&cp, &def, false)?; info!(log, "Created new region file {:?}", cp); - Ok(Region { + let mut region = Region { dir: dir.as_ref().to_path_buf(), def, extents: Vec::new(), dirty_extents: HashSet::new(), read_only: false, log, - #[cfg(test)] - prefer_sqlite_backend: false, - }) + #[cfg(any(test, feature = "integration-tests"))] + prefer_sqlite_backend: backend == Backend::SQLite, + }; + region.open_extents(true).await?; + Ok(region) } /** - * Create a new SQLite-backed region based on the given RegionOptions + * Open an existing region file */ - #[cfg(test)] - pub async fn create_sqlite>( + pub async fn open>( dir: P, options: RegionOptions, - log: Logger, + verbose: bool, + read_only: bool, + log: &Logger, ) -> Result { - let mut region = Self::create_inner(dir, options, log)?; - region.prefer_sqlite_backend = true; - region.open_extents(true).await?; - - Ok(region) + Self::open_with_backend( + dir, + options, + verbose, + read_only, + Backend::RawFile, + log, + ) + .await } - /** - * Open an existing region file - */ - pub async fn open>( + pub async fn open_with_backend>( dir: P, options: RegionOptions, verbose: bool, read_only: bool, + backend: Backend, log: &Logger, ) -> Result { options.validate()?; @@ -314,8 +318,8 @@ impl Region { read_only, log: log.clone(), - #[cfg(test)] - prefer_sqlite_backend: false, + #[cfg(any(test, feature = "integration-tests"))] + prefer_sqlite_backend: backend == Backend::SQLite, }; region.open_extents(false).await?; @@ -355,13 +359,13 @@ impl Region { for eid in eid_range { let extent = if create { - #[cfg(test)] + #[cfg(any(test, feature = "integration-tests"))] if self.prefer_sqlite_backend { Extent::create_sqlite(&self.dir, &self.def, eid)? } else { Extent::create(&self.dir, &self.def, eid)? } - #[cfg(not(test))] + #[cfg(not(any(test, feature = "integration-tests")))] Extent::create(&self.dir, &self.def, eid)? } else { let extent = Extent::open( @@ -1538,8 +1542,13 @@ pub(crate) mod test { // metadata files. // Create the region, make three extents let dir = tempdir()?; - let mut region = - Region::create_sqlite(&dir, new_region_options(), csl()).await?; + let mut region = Region::create_with_backend( + &dir, + new_region_options(), + Backend::SQLite, + csl(), + ) + .await?; region.extend(3).await?; // Close extent 1 @@ -1661,8 +1670,13 @@ pub(crate) mod test { // extent after the reopen has cleaned them up. // Create the region, make three extents let dir = tempdir()?; - let mut region = - Region::create_sqlite(&dir, new_region_options(), csl()).await?; + let mut region = Region::create_with_backend( + &dir, + new_region_options(), + Backend::SQLite, + csl(), + ) + .await?; region.extend(3).await?; // Close extent 1 @@ -1743,8 +1757,13 @@ pub(crate) mod test { // Create the region, make three extents let dir = tempdir()?; - let mut region = - Region::create_sqlite(&dir, new_region_options(), csl()).await?; + let mut region = Region::create_with_backend( + &dir, + new_region_options(), + Backend::SQLite, + csl(), + ) + .await?; region.extend(3).await?; // Make copy directory for this extent @@ -1832,9 +1851,13 @@ pub(crate) mod test { async fn reopen_extent_partial_migration() -> Result<()> { let log = csl(); let dir = tempdir()?; - let mut region = - Region::create_sqlite(&dir, new_region_options(), log.clone()) - .await?; + let mut region = Region::create_with_backend( + &dir, + new_region_options(), + Backend::SQLite, + csl(), + ) + .await?; region.extend(3).await?; let ddef = region.def(); @@ -1927,9 +1950,13 @@ pub(crate) mod test { async fn reopen_extent_partial_migration_corrupt() -> Result<()> { let log = csl(); let dir = tempdir()?; - let mut region = - Region::create_sqlite(&dir, new_region_options(), log.clone()) - .await?; + let mut region = Region::create_with_backend( + &dir, + new_region_options(), + Backend::SQLite, + csl(), + ) + .await?; region.extend(3).await?; let ddef = region.def(); @@ -2324,9 +2351,13 @@ pub(crate) mod test { async fn test_big_write_migrate() -> Result<()> { let log = csl(); let dir = tempdir()?; - let mut region = - Region::create_sqlite(&dir, new_region_options(), log.clone()) - .await?; + let mut region = Region::create_with_backend( + &dir, + new_region_options(), + Backend::SQLite, + csl(), + ) + .await?; region.extend(3).await?; let ddef = region.def(); diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index 68b8e405c..113173f12 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -16,7 +16,7 @@ bytes.workspace = true crucible-client-types.workspace = true # importantly, don't use features = ["zfs_snapshot"] here, this will cause # cleanup issues! -crucible-downstairs.workspace = true +crucible-downstairs = { workspace = true, features = ["integration-tests"] } crucible-pantry-client.workspace = true crucible-pantry.workspace = true crucible.workspace = true diff --git a/integration_tests/src/lib.rs b/integration_tests/src/lib.rs index aae0fea73..4115560ff 100644 --- a/integration_tests/src/lib.rs +++ b/integration_tests/src/lib.rs @@ -44,30 +44,35 @@ mod test { blocks_per_extent: u64, extent_count: u32, problematic: bool, + backend: Backend, ) -> Result { let tempdir = tempfile::Builder::new() .prefix(&"downstairs-") .rand_bytes(8) .tempdir()?; - let _region = create_region( - 512, /* block_size */ + let _region = create_region_with_backend( tempdir.path().to_path_buf(), - blocks_per_extent, + Block { + value: blocks_per_extent, + shift: 9, + }, extent_count.into(), Uuid::new_v4(), encrypted, + backend, csl(), ) .await?; - let downstairs = build_downstairs_for_region( + let downstairs = build_downstairs_for_region_with_backend( tempdir.path(), problematic, /* lossy */ problematic, /* read errors */ problematic, /* write errors */ problematic, /* flush errors */ read_only, + backend, Some(csl()), ) .await?; @@ -191,6 +196,7 @@ mod test { blocks_per_extent, extent_count, problematic, + Backend::RawFile, ) .await?; let downstairs2 = TestDownstairs::new( @@ -200,6 +206,7 @@ mod test { blocks_per_extent, extent_count, problematic, + Backend::RawFile, ) .await?; let downstairs3 = TestDownstairs::new( @@ -209,6 +216,7 @@ mod test { blocks_per_extent, extent_count, problematic, + Backend::RawFile, ) .await?; @@ -281,6 +289,7 @@ mod test { self.blocks_per_extent, self.extent_count, false, + Backend::RawFile, ) .await } From 786d4981fe58a9477fe8734de3273b1af94c8dbc Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 11:50:58 -0400 Subject: [PATCH 40/62] Add raw-on-SQLite integration test --- downstairs/src/extent.rs | 50 +++----- downstairs/src/region.rs | 26 ++-- integration_tests/src/lib.rs | 231 ++++++++++++++++++++++++++++++++++- 3 files changed, 257 insertions(+), 50 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index e14792efc..a6062e09a 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -281,6 +281,7 @@ impl Extent { def: &RegionDefinition, number: u32, read_only: bool, + backend: Backend, log: &Logger, ) -> Result { /* @@ -349,7 +350,13 @@ impl Extent { // `Extent::open` is called, we will restart the migration (if we // haven't gotten to deleting the `.db` file). let mut has_sqlite = path.with_extension("db").exists(); - if has_sqlite && !read_only { + let force_sqlite_backend = match backend { + Backend::RawFile => false, + #[cfg(any(test, feature = "integration-tests"))] + Backend::SQLite => true, + }; + let should_migrate = has_sqlite && !read_only && !force_sqlite_backend; + if should_migrate { info!(log, "Migrating extent {number}"); // Truncate the file to the length of the extent data { @@ -414,7 +421,7 @@ impl Extent { // as-is. let inner: Box = { if has_sqlite { - assert!(read_only); + assert!(read_only || force_sqlite_backend); let inner = extent_inner_sqlite::SqliteInner::open( &path, def, number, read_only, log, )?; @@ -463,6 +470,7 @@ impl Extent { dir: &Path, def: &RegionDefinition, number: u32, + backend: Backend, ) -> Result { /* * Store extent data in files within a directory hierarchy so that @@ -480,8 +488,15 @@ impl Extent { } remove_copy_cleanup_dir(dir, number)?; - // All new extents are created using the raw backend - let inner = extent_inner_raw::RawInner::create(dir, def, number)?; + let inner: Box = match backend { + Backend::RawFile => { + Box::new(extent_inner_raw::RawInner::create(dir, def, number)?) + } + #[cfg(any(test, feature = "integration-tests"))] + Backend::SQLite => Box::new( + extent_inner_sqlite::SqliteInner::create(dir, def, number)?, + ), + }; /* * Complete the construction of our new extent @@ -490,32 +505,7 @@ impl Extent { number, read_only: false, iov_max: Extent::get_iov_max()?, - inner: Mutex::new(Box::new(inner)), - }) - } - - /// Identical to `create`, but using the SQLite backend - /// - /// This is only allowed in unit tests - #[cfg(any(test, feature = "integration-tests"))] - pub fn create_sqlite( - dir: &Path, - def: &RegionDefinition, - number: u32, - ) -> Result { - { - let path = extent_path(dir, number); - if Path::new(&path).exists() { - bail!("Extent file already exists {:?}", path); - } - } - remove_copy_cleanup_dir(dir, number)?; - let inner = extent_inner_sqlite::SqliteInner::create(dir, def, number)?; - Ok(Extent { - number, - read_only: false, - iov_max: Extent::get_iov_max()?, - inner: Mutex::new(Box::new(inner)), + inner: Mutex::new(inner), }) } diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 9c7991cdf..529b590bc 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -87,11 +87,11 @@ pub struct Region { read_only: bool, log: Logger, - /// Use the SQLite extent backend when creating new extents + /// Select the backend to use when creating and opening extents /// - /// This is only allowed in tests; all new extents must be raw files - #[cfg(any(test, feature = "integration-tests"))] - prefer_sqlite_backend: bool, + /// The SQLite backend is only allowed in tests; all new extents must be raw + /// files + backend: Backend, } impl Region { @@ -223,8 +223,7 @@ impl Region { dirty_extents: HashSet::new(), read_only: false, log, - #[cfg(any(test, feature = "integration-tests"))] - prefer_sqlite_backend: backend == Backend::SQLite, + backend, }; region.open_extents(true).await?; Ok(region) @@ -317,9 +316,7 @@ impl Region { dirty_extents: HashSet::new(), read_only, log: log.clone(), - - #[cfg(any(test, feature = "integration-tests"))] - prefer_sqlite_backend: backend == Backend::SQLite, + backend, }; region.open_extents(false).await?; @@ -359,20 +356,14 @@ impl Region { for eid in eid_range { let extent = if create { - #[cfg(any(test, feature = "integration-tests"))] - if self.prefer_sqlite_backend { - Extent::create_sqlite(&self.dir, &self.def, eid)? - } else { - Extent::create(&self.dir, &self.def, eid)? - } - #[cfg(not(any(test, feature = "integration-tests")))] - Extent::create(&self.dir, &self.def, eid)? + Extent::create(&self.dir, &self.def, eid, self.backend)? } else { let extent = Extent::open( &self.dir, &self.def, eid, self.read_only, + self.backend, &self.log, )?; @@ -441,6 +432,7 @@ impl Region { &self.def, eid as u32, self.read_only, + Backend::RawFile, &self.log, )?; diff --git a/integration_tests/src/lib.rs b/integration_tests/src/lib.rs index 4115560ff..1403a4981 100644 --- a/integration_tests/src/lib.rs +++ b/integration_tests/src/lib.rs @@ -150,6 +150,24 @@ mod test { blocks_per_extent, extent_count, false, + Backend::RawFile, + ) + .await + } + + /// Spin off three SQLite downstairs, with a 5120b region + pub async fn small_sqlite( + read_only: bool, + ) -> Result { + // 5 * 2 * 512 = 5120b + let blocks_per_extent = 5; + let extent_count = 2; + TestDownstairsSet::new_with_flag( + read_only, + blocks_per_extent, + extent_count, + false, + Backend::SQLite, ) .await } @@ -164,6 +182,7 @@ mod test { blocks_per_extent, extent_count, false, + Backend::RawFile, ) .await } @@ -178,6 +197,7 @@ mod test { blocks_per_extent, extent_count, true, // problematic + Backend::RawFile, ) .await } @@ -188,6 +208,7 @@ mod test { blocks_per_extent: u64, extent_count: u32, problematic: bool, + backend: Backend, ) -> Result { let downstairs1 = TestDownstairs::new( "127.0.0.1".parse()?, @@ -196,7 +217,7 @@ mod test { blocks_per_extent, extent_count, problematic, - Backend::RawFile, + backend, ) .await?; let downstairs2 = TestDownstairs::new( @@ -206,7 +227,7 @@ mod test { blocks_per_extent, extent_count, problematic, - Backend::RawFile, + backend, ) .await?; let downstairs3 = TestDownstairs::new( @@ -216,7 +237,7 @@ mod test { blocks_per_extent, extent_count, problematic, - Backend::RawFile, + backend, ) .await?; @@ -2246,6 +2267,210 @@ mod test { Ok(()) } + #[tokio::test] + async fn integration_test_sqlite_backed_vol() -> Result<()> { + // Test using an old SQLite backend as a read-only parent. + + const BLOCK_SIZE: usize = 512; + + // boot three downstairs, write some data to them, then change to + // read-only. + let mut test_downstairs_set = + TestDownstairsSet::small_sqlite(false).await?; + // This must be a SQLite extent! + println!( + "{:?}", + test_downstairs_set + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + ); + let out = std::process::Command::new("tree") + .arg(test_downstairs_set.downstairs1.tempdir.path()) + .output() + .expect("failed to execute process"); + println!("{}", std::str::from_utf8(&out.stdout).unwrap()); + + assert!(test_downstairs_set + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + .exists()); + + let mut volume = Volume::new(BLOCK_SIZE as u64, csl()); + volume + .add_subvolume_create_guest( + test_downstairs_set.opts(), + volume::RegionExtentInfo { + block_size: BLOCK_SIZE as u64, + blocks_per_extent: test_downstairs_set.blocks_per_extent(), + extent_count: test_downstairs_set.extent_count(), + }, + 1, + None, + ) + .await?; + + volume.activate().await?; + + let random_buffer = { + let mut random_buffer = + vec![0u8; volume.total_size().await? as usize]; + rand::thread_rng().fill(&mut random_buffer[..]); + random_buffer + }; + + volume + .write( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + Bytes::from(random_buffer.clone()), + ) + .await?; + + volume.deactivate().await?; + + drop(volume); + + test_downstairs_set.reboot_read_only().await?; + // This must still be a SQLite backend! + assert!(test_downstairs_set + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + .exists()); + + // Validate that this now accepts reads and flushes, but rejects writes + { + let mut volume = Volume::new(BLOCK_SIZE as u64, csl()); + volume + .add_subvolume_create_guest( + test_downstairs_set.opts(), + volume::RegionExtentInfo { + block_size: BLOCK_SIZE as u64, + blocks_per_extent: test_downstairs_set + .blocks_per_extent(), + extent_count: test_downstairs_set.extent_count(), + }, + 2, + None, + ) + .await?; + + volume.activate().await?; + + let buffer = Buffer::new(volume.total_size().await? as usize); + volume + .read( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + buffer.clone(), + ) + .await?; + + assert_eq!(*buffer.as_vec().await, random_buffer); + + assert!(volume + .write( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + Bytes::from(vec![0u8; BLOCK_SIZE]), + ) + .await + .is_err()); + + volume.flush(None).await?; + } + + // create a new volume, layering a new set of downstairs on top of the + // read-only one we just (re)booted + let top_layer_tds = TestDownstairsSet::small(false).await?; + let top_layer_opts = top_layer_tds.opts(); + let bottom_layer_opts = test_downstairs_set.opts(); + // The new volume is **not** using the SQLite backend! + assert!(!top_layer_tds + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + .exists()); + + let vcr: VolumeConstructionRequest = + VolumeConstructionRequest::Volume { + id: Uuid::new_v4(), + block_size: BLOCK_SIZE as u64, + sub_volumes: vec![VolumeConstructionRequest::Region { + block_size: BLOCK_SIZE as u64, + blocks_per_extent: top_layer_tds.blocks_per_extent(), + extent_count: top_layer_tds.extent_count(), + opts: top_layer_opts, + gen: 3, + }], + read_only_parent: Some(Box::new( + VolumeConstructionRequest::Volume { + id: Uuid::new_v4(), + block_size: BLOCK_SIZE as u64, + sub_volumes: vec![VolumeConstructionRequest::Region { + block_size: BLOCK_SIZE as u64, + blocks_per_extent: test_downstairs_set + .blocks_per_extent(), + extent_count: test_downstairs_set.extent_count(), + opts: bottom_layer_opts, + gen: 3, + }], + read_only_parent: None, + }, + )), + }; + + let volume = Volume::construct(vcr, None, csl()).await?; + volume.activate().await?; + + // Validate that source blocks originally come from the read-only parent + { + let buffer = Buffer::new(volume.total_size().await? as usize); + volume + .read( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + buffer.clone(), + ) + .await?; + + assert_eq!(*buffer.as_vec().await, random_buffer); + } + + // Validate a flush works + volume.flush(None).await?; + + // Write one block of 0x00 in, validate with a read + volume + .write( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + Bytes::from(vec![0u8; BLOCK_SIZE]), + ) + .await?; + + { + let buffer = Buffer::new(volume.total_size().await? as usize); + volume + .read( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + buffer.clone(), + ) + .await?; + + let buffer_vec = buffer.as_vec().await; + + assert_eq!(buffer_vec[..BLOCK_SIZE], vec![0u8; BLOCK_SIZE]); + assert_eq!(buffer_vec[BLOCK_SIZE..], random_buffer[BLOCK_SIZE..]); + } + + // Validate a flush still works + volume.flush(None).await?; + + Ok(()) + } + #[tokio::test] async fn integration_test_volume_replace_downstairs() -> Result<()> { // Replace a downstairs with a new one From bd917781088fa9dff6d6a0afb6785eca29343d36 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 12:16:26 -0400 Subject: [PATCH 41/62] Add migration test --- integration_tests/src/lib.rs | 143 +++++++++++++++++++++++++++++++++++ 1 file changed, 143 insertions(+) diff --git a/integration_tests/src/lib.rs b/integration_tests/src/lib.rs index 1403a4981..652df9f52 100644 --- a/integration_tests/src/lib.rs +++ b/integration_tests/src/lib.rs @@ -123,6 +123,33 @@ mod test { Ok(()) } + pub async fn reboot_read_write(&mut self) -> Result<()> { + self.downstairs = build_downstairs_for_region( + self.tempdir.path(), + false, /* lossy */ + false, /* read errors */ + false, /* write errors */ + false, /* flush errors */ + false, + Some(csl()), + ) + .await?; + + let _join_handle = start_downstairs( + self.downstairs.clone(), + self.address, + None, /* oximeter */ + 0, /* any port */ + 0, /* any rport */ + None, /* cert_pem */ + None, /* key_pem */ + None, /* root_cert_pem */ + ) + .await?; + + Ok(()) + } + pub async fn address(&self) -> SocketAddr { // If start_downstairs returned Ok, then address will be populated self.downstairs.lock().await.address.unwrap() @@ -302,6 +329,21 @@ mod test { Ok(()) } + pub async fn reboot_read_write(&mut self) -> Result<()> { + assert!(!self.crucible_opts.read_only); + self.downstairs1.reboot_read_write().await?; + self.downstairs2.reboot_read_write().await?; + self.downstairs3.reboot_read_write().await?; + + self.crucible_opts.target = vec![ + self.downstairs1.address().await, + self.downstairs2.address().await, + self.downstairs3.address().await, + ]; + + Ok(()) + } + pub async fn new_downstairs(&self) -> Result { TestDownstairs::new( "127.0.0.1".parse()?, @@ -2471,6 +2513,107 @@ mod test { Ok(()) } + #[tokio::test] + async fn integration_test_sqlite_migration() -> Result<()> { + // Test using an old SQLite backend as a read-only parent. + + const BLOCK_SIZE: usize = 512; + + // boot three downstairs, write some data to them, then change to + // read-only. + let mut test_downstairs_set = + TestDownstairsSet::small_sqlite(false).await?; + // This must be a SQLite extent! + println!( + "{:?}", + test_downstairs_set + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + ); + let out = std::process::Command::new("tree") + .arg(test_downstairs_set.downstairs1.tempdir.path()) + .output() + .expect("failed to execute process"); + println!("{}", std::str::from_utf8(&out.stdout).unwrap()); + + assert!(test_downstairs_set + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + .exists()); + + let mut volume = Volume::new(BLOCK_SIZE as u64, csl()); + volume + .add_subvolume_create_guest( + test_downstairs_set.opts(), + volume::RegionExtentInfo { + block_size: BLOCK_SIZE as u64, + blocks_per_extent: test_downstairs_set.blocks_per_extent(), + extent_count: test_downstairs_set.extent_count(), + }, + 1, + None, + ) + .await?; + + volume.activate().await?; + + let random_buffer = { + let mut random_buffer = + vec![0u8; volume.total_size().await? as usize]; + rand::thread_rng().fill(&mut random_buffer[..]); + random_buffer + }; + + volume + .write( + Block::new(0, BLOCK_SIZE.trailing_zeros()), + Bytes::from(random_buffer.clone()), + ) + .await?; + + volume.deactivate().await?; + + drop(volume); + + test_downstairs_set.reboot_read_write().await?; + // This should now be migrated, and the DB file should be deleted + assert!(!test_downstairs_set + .downstairs1 + .tempdir + .path() + .join("00/000/000.db") + .exists()); + + let mut volume = Volume::new(BLOCK_SIZE as u64, csl()); + volume + .add_subvolume_create_guest( + test_downstairs_set.opts(), + volume::RegionExtentInfo { + block_size: BLOCK_SIZE as u64, + blocks_per_extent: test_downstairs_set.blocks_per_extent(), + extent_count: test_downstairs_set.extent_count(), + }, + 2, + None, + ) + .await?; + + volume.activate().await?; + // Validate that source blocks are the same + let buffer = Buffer::new(volume.total_size().await? as usize); + volume + .read(Block::new(0, BLOCK_SIZE.trailing_zeros()), buffer.clone()) + .await?; + + assert_eq!(*buffer.as_vec().await, random_buffer); + + Ok(()) + } + #[tokio::test] async fn integration_test_volume_replace_downstairs() -> Result<()> { // Replace a downstairs with a new one From 26b6b6d838e866d8acf91d0a289cef0fa6c6e14c Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 13:44:21 -0400 Subject: [PATCH 42/62] serialize_into should be infallible --- downstairs/src/extent_inner_raw.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index b69574d1b..9825e722f 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1248,11 +1248,7 @@ impl RawLayout { block_context: b.block_context, on_disk_hash: b.on_disk_hash, }); - bincode::serialize_into(&mut buf[n..], &d).map_err(|e| { - CrucibleError::IoError(format!( - "could not serialize context: {e}" - )) - })?; + bincode::serialize_into(&mut buf[n..], &d).unwrap(); } let offset = self.context_slot_offset(block_start, slot); nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64) @@ -1321,7 +1317,7 @@ impl RawLayout { ext_version: EXTENT_META_RAW, }; let mut meta = [0u8; BLOCK_META_SIZE_BYTES as usize]; - bincode::serialize_into(meta.as_mut_slice(), &d)?; + bincode::serialize_into(meta.as_mut_slice(), &d).unwrap(); buf.extend(meta); let offset = self.active_context_offset(); From 88cb382f1603bfdcdc0299ccb9834b091cca17cd Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 13:48:52 -0400 Subject: [PATCH 43/62] Make open() and create() consistent (taking directory) --- downstairs/src/extent.rs | 6 +++--- downstairs/src/extent_inner_raw.rs | 5 +++-- downstairs/src/extent_inner_sqlite.rs | 10 +++++----- downstairs/src/region.rs | 2 +- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index a6062e09a..72cbbbf63 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -367,7 +367,7 @@ impl Extent { // Compute supplemental data from the SQLite extent let mut inner = extent_inner_sqlite::SqliteInner::open( - &path, def, number, read_only, log, + dir, def, number, read_only, log, )?; let ctxs = inner.export_contexts()?; let dirty = inner.dirty()?; @@ -423,12 +423,12 @@ impl Extent { if has_sqlite { assert!(read_only || force_sqlite_backend); let inner = extent_inner_sqlite::SqliteInner::open( - &path, def, number, read_only, log, + dir, def, number, read_only, log, )?; Box::new(inner) } else { let inner = extent_inner_raw::RawInner::open( - &path, def, number, read_only, log, + dir, def, number, read_only, log, )?; Box::new(inner) } diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 9825e722f..d4ba625c3 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -619,12 +619,13 @@ impl RawInner { /// Constructs a new `Inner` object from files that already exist on disk pub fn open( - path: &Path, + dir: &Path, def: &RegionDefinition, extent_number: u32, read_only: bool, log: &Logger, ) -> Result { + let path = extent_path(dir, extent_number); let extent_size = def.extent_size(); let layout = RawLayout::new(extent_size); let size = layout.file_size(); @@ -633,7 +634,7 @@ impl RawInner { * Open the extent file and verify the size is as we expect. */ let file = - match OpenOptions::new().read(true).write(!read_only).open(path) { + match OpenOptions::new().read(true).write(!read_only).open(&path) { Err(e) => { error!( log, diff --git a/downstairs/src/extent_inner_sqlite.rs b/downstairs/src/extent_inner_sqlite.rs index 3638623ae..34056c7b6 100644 --- a/downstairs/src/extent_inner_sqlite.rs +++ b/downstairs/src/extent_inner_sqlite.rs @@ -1,7 +1,7 @@ // Copyright 2023 Oxide Computer Company use crate::{ cdt, crucible_bail, - extent::{check_input, DownstairsBlockContext, ExtentInner}, + extent::{check_input, extent_path, DownstairsBlockContext, ExtentInner}, integrity_hash, region::{BatchedPwritev, JobOrReconciliationId}, Block, BlockContext, CrucibleError, JobId, ReadResponse, RegionDefinition, @@ -600,7 +600,7 @@ impl SqliteInner { extent_number: u32, ) -> Result { use crate::{ - extent::{extent_path, ExtentMeta, EXTENT_META_SQLITE}, + extent::{ExtentMeta, EXTENT_META_SQLITE}, mkdir_for_file, }; let mut path = extent_path(dir, extent_number); @@ -749,12 +749,13 @@ impl SqliteInner { } pub fn open( - path: &Path, + dir: &Path, def: &RegionDefinition, extent_number: u32, read_only: bool, log: &Logger, ) -> Result { + let mut path = extent_path(dir, extent_number); let bcount = def.extent_size().value; let size = def.block_size().checked_mul(bcount).unwrap(); @@ -762,7 +763,7 @@ impl SqliteInner { * Open the extent file and verify the size is as we expect. */ let file = - match OpenOptions::new().read(true).write(!read_only).open(path) { + match OpenOptions::new().read(true).write(!read_only).open(&path) { Err(e) => { error!( log, @@ -789,7 +790,6 @@ impl SqliteInner { /* * Open a connection to the metadata db */ - let mut path = path.to_path_buf(); path.set_extension("db"); let metadb = match open_sqlite_connection(&path) { Err(e) => { diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 529b590bc..fe91c193e 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1986,7 +1986,7 @@ pub(crate) mod test { // write everything to disk). let extent_file = extent_path(&dir, 1); let mut inner = extent_inner_sqlite::SqliteInner::open( - &extent_file, + dir.path(), &ddef, 1, false, From 7b883ba46df170e99ca9353c765478f4daaf848e Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 14:18:00 -0400 Subject: [PATCH 44/62] Fix last failing test --- downstairs/src/region.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index fe91c193e..991ea28ec 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1884,7 +1884,7 @@ pub(crate) mod test { // Manually calculate the migration from extent 1 let extent_file = extent_path(&dir, 1); let mut inner = extent_inner_sqlite::SqliteInner::open( - &extent_file, + dir.path(), &ddef, 1, false, From 8583774d5143dab8de2d2ff0fcd5417b1343bfe8 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 27 Oct 2023 17:22:53 -0400 Subject: [PATCH 45/62] More specific errors --- common/src/lib.rs | 9 ++ downstairs/src/extent_inner_raw.rs | 135 ++++++++++++++++++----------- 2 files changed, 93 insertions(+), 51 deletions(-) diff --git a/common/src/lib.rs b/common/src/lib.rs index e278b0d4b..c68571e5c 100644 --- a/common/src/lib.rs +++ b/common/src/lib.rs @@ -146,6 +146,15 @@ pub enum CrucibleError { #[error("Invalid downstairs replace {0}")] ReplaceRequestInvalid(String), + + #[error("missing context slot for block {0}")] + MissingContextSlot(u64), + + #[error("metadata deserialization failed: {0}")] + BadMetadata(String), + + #[error("context slot deserialization failed: {0}")] + BadContextSlot(String), } impl From for CrucibleError { diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index d4ba625c3..b117baf73 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1,6 +1,6 @@ // Copyright 2023 Oxide Computer Company use crate::{ - cdt, crucible_bail, + cdt, extent::{ check_input, extent_path, DownstairsBlockContext, ExtentInner, EXTENT_META_RAW, @@ -10,7 +10,6 @@ use crate::{ Block, BlockContext, CrucibleError, JobId, ReadResponse, RegionDefinition, }; -use anyhow::{bail, Result}; use itertools::Itertools; use serde::{Deserialize, Serialize}; use slog::{error, Logger}; @@ -401,7 +400,12 @@ impl ExtentInner for RawInner { &mut iovecs, first_req.offset.value as i64 * block_size as i64, ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + .map_err(|e| { + CrucibleError::IoError(format!( + "extent {}: read failed: {e}", + self.extent_number + )) + })?; cdt::extent__read__file__done!(|| { (job_id.0, self.extent_number, n_contiguous_requests as u64) @@ -470,12 +474,10 @@ impl ExtentInner for RawInner { /* * XXX Retry? Mark extent as broken? */ - crucible_bail!( - IoError, - "extent {}: fsync 1 failure: {:?}", + return Err(CrucibleError::IoError(format!( + "extent {}: fsync 1 failure: {e:?}", self.extent_number, - e - ); + ))); } self.sync_index += 1; cdt::extent__flush__file__done!(|| { @@ -569,7 +571,7 @@ impl RawInner { dir: &Path, def: &RegionDefinition, extent_number: u32, - ) -> Result { + ) -> Result { let path = extent_path(dir, extent_number); let extent_size = def.extent_size(); let layout = RawLayout::new(extent_size); @@ -609,10 +611,9 @@ impl RawInner { // Sync the file to disk, to avoid any questions if let Err(e) = out.file.sync_all() { return Err(CrucibleError::IoError(format!( - "extent {}: fsync 1 failure during initial sync: {e:?}", + "extent {}: fsync 1 failure during initial sync: {e}", out.extent_number, - )) - .into()); + ))); } Ok(out) } @@ -624,7 +625,7 @@ impl RawInner { extent_number: u32, read_only: bool, log: &Logger, - ) -> Result { + ) -> Result { let path = extent_path(dir, extent_number); let extent_size = def.extent_size(); let layout = RawLayout::new(extent_size); @@ -641,18 +642,17 @@ impl RawInner { "Open of {path:?} for extent#{extent_number} \ returned: {e}", ); - bail!( - "Open of {path:?} for extent#{extent_number} \ - returned: {e}", - ); + return Err(CrucibleError::IoError(format!( + "extent {extent_number}: open of {path:?} failed: {e}", + ))); } Ok(f) => { let cur_size = f.metadata().unwrap().len(); if size != cur_size { - bail!( - "File size {size:?} does not match \ - expected {cur_size:?}", - ); + return Err(CrucibleError::IoError(format!( + "extent {extent_number}: file size {cur_size:?} \ + does not match expected {size:?}", + ))); } f } @@ -663,10 +663,9 @@ impl RawInner { if !read_only { if let Err(e) = file.sync_all() { return Err(CrucibleError::IoError(format!( - "extent {extent_number}: - fsync 1 failure during initial rehash: {e:?}", - )) - .into()); + "extent {extent_number}: \ + fsync 1 failure during initial rehash: {e}", + ))); } } @@ -743,9 +742,7 @@ impl RawInner { } matching_slot.or(empty_slot).ok_or( - CrucibleError::IoError(format!( - "open: no slot found for {block}" - )), + CrucibleError::MissingContextSlot(block as u64), )? }; active_context.push(slot); @@ -798,7 +795,12 @@ impl RawInner { &mut buf, (block_size as u64 * block) as i64, ) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + .map_err(|e| { + CrucibleError::IoError(format!( + "extent {}: reading block {block} data failed: {e}", + self.extent_number + )) + })?; let hash = integrity_hash(&[&buf]); // Then, read the slot data and decide if either slot @@ -822,9 +824,9 @@ impl RawInner { empty_slot = Some(slot); } } - let value = matching_slot.or(empty_slot).ok_or( - CrucibleError::IoError(format!("no slot found for {block}")), - )?; + let value = matching_slot + .or(empty_slot) + .ok_or(CrucibleError::MissingContextSlot(block))?; self.active_context[block as usize] = value; Ok(()) } @@ -832,7 +834,7 @@ impl RawInner { fn set_block_contexts( &mut self, block_contexts: &[DownstairsBlockContext], - ) -> Result<()> { + ) -> Result<(), CrucibleError> { // If any of these block contexts will be overwriting an unsyched // context slot, then we insert a sync here. let needs_sync = block_contexts.iter().any(|block_context| { @@ -847,7 +849,7 @@ impl RawInner { if needs_sync { self.file.sync_all().map_err(|e| { CrucibleError::IoError(format!( - "extent {}: fsync 1 failure: {e:?}", + "extent {}: fsync 1 failure: {e}", self.extent_number, )) })?; @@ -893,7 +895,7 @@ impl RawInner { fn set_block_contexts_contiguous( &mut self, block_contexts: &[DownstairsBlockContext], - ) -> Result { + ) -> Result { for (a, b) in block_contexts.iter().zip(block_contexts.iter().skip(1)) { assert_eq!(a.block + 1, b.block, "blocks must be contiguous"); } @@ -930,7 +932,11 @@ impl RawInner { } /// Update the flush number, generation number, and clear the dirty bit - fn set_flush_number(&mut self, new_flush: u64, new_gen: u64) -> Result<()> { + fn set_flush_number( + &mut self, + new_flush: u64, + new_gen: u64, + ) -> Result<(), CrucibleError> { self.layout.write_active_context_and_metadata( &self.file, &self.active_context, @@ -947,7 +953,7 @@ impl RawInner { &mut self, block: u64, count: u64, - ) -> Result>> { + ) -> Result>, CrucibleError> { let mut out = vec![]; let mut reads = 0u64; for (slot, group) in (block..block + count) @@ -1006,7 +1012,7 @@ impl RawInner { fn get_block_context( &mut self, block: u64, - ) -> Result> { + ) -> Result, CrucibleError> { let mut out = self.get_block_contexts(block, 1)?; assert_eq!(out.len(), 1); Ok(out.pop().unwrap()) @@ -1163,7 +1169,11 @@ impl RawLayout { fn set_dirty(&self, file: &File) -> Result<(), CrucibleError> { let offset = self.metadata_offset(); nix::sys::uio::pwrite(file.as_raw_fd(), &[1u8], offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + .map_err(|e| { + CrucibleError::IoError(format!( + "writing dirty byte failed: {e}", + )) + })?; Ok(()) } @@ -1223,9 +1233,11 @@ impl RawLayout { let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; let offset = self.metadata_offset(); nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + .map_err(|e| { + CrucibleError::IoError(format!("reading metadata failed: {e}")) + })?; let out: OnDiskMeta = bincode::deserialize(&buf) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + .map_err(|e| CrucibleError::BadMetadata(e.to_string()))?; Ok(out) } @@ -1252,8 +1264,13 @@ impl RawLayout { bincode::serialize_into(&mut buf[n..], &d).unwrap(); } let offset = self.context_slot_offset(block_start, slot); - nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64).map_err( + |e| { + CrucibleError::IoError(format!( + "writing context slots failed: {e}" + )) + }, + )?; self.buf.set(buf); Ok(()) } @@ -1264,13 +1281,17 @@ impl RawLayout { block_start: u64, block_count: u64, slot: ContextSlot, - ) -> Result>> { + ) -> Result>, CrucibleError> { let mut buf = self.buf.take(); buf.resize((BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count) as usize, 0u8); let offset = self.context_slot_offset(block_start, slot); nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + .map_err(|e| { + CrucibleError::IoError(format!( + "reading context slots failed: {e}" + )) + })?; let mut out = vec![]; for (i, chunk) in buf @@ -1278,7 +1299,9 @@ impl RawLayout { .enumerate() { let ctx: Option = - bincode::deserialize(chunk)?; + bincode::deserialize(chunk).map_err(|e| { + CrucibleError::BadContextSlot(e.to_string()) + })?; out.push(ctx.map(|c| DownstairsBlockContext { block: block_start + i as u64, block_context: c.block_context, @@ -1299,7 +1322,7 @@ impl RawLayout { dirty: bool, flush_number: u64, gen_number: u64, - ) -> Result<()> { + ) -> Result<(), CrucibleError> { // Serialize bitpacked active slot values let mut buf = self.buf.take(); buf.clear(); @@ -1323,8 +1346,9 @@ impl RawLayout { let offset = self.active_context_offset(); - nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64) - .map_err(|e| CrucibleError::IoError(e.to_string()))?; + nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64).map_err( + |e| CrucibleError::IoError(format!("writing metadata failed: {e}")), + )?; self.buf.set(buf); Ok(()) @@ -1333,11 +1357,19 @@ impl RawLayout { /// Decodes the active contexts from the given file /// /// The file descriptor offset is not changed by this function - fn get_active_contexts(&self, file: &File) -> Result> { + fn get_active_contexts( + &self, + file: &File, + ) -> Result, CrucibleError> { let mut buf = self.buf.take(); buf.resize(self.active_context_size() as usize, 0u8); let offset = self.active_context_offset(); - nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64)?; + nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) + .map_err(|e| { + CrucibleError::IoError(format!( + "could not read active contexts: {e}" + )) + })?; let mut active_context = vec![]; for bit in buf @@ -1360,6 +1392,7 @@ impl RawLayout { #[cfg(test)] mod test { use super::*; + use anyhow::Result; use bytes::{Bytes, BytesMut}; use crucible_protocol::EncryptionContext; use crucible_protocol::ReadRequest; From ade73921e9c0f6e9681ae10054cdb195b694aa69 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 09:56:22 -0400 Subject: [PATCH 46/62] Remove debug stuff --- integration_tests/src/lib.rs | 28 ---------------------------- 1 file changed, 28 deletions(-) diff --git a/integration_tests/src/lib.rs b/integration_tests/src/lib.rs index 652df9f52..66836def8 100644 --- a/integration_tests/src/lib.rs +++ b/integration_tests/src/lib.rs @@ -2320,20 +2320,6 @@ mod test { let mut test_downstairs_set = TestDownstairsSet::small_sqlite(false).await?; // This must be a SQLite extent! - println!( - "{:?}", - test_downstairs_set - .downstairs1 - .tempdir - .path() - .join("00/000/000.db") - ); - let out = std::process::Command::new("tree") - .arg(test_downstairs_set.downstairs1.tempdir.path()) - .output() - .expect("failed to execute process"); - println!("{}", std::str::from_utf8(&out.stdout).unwrap()); - assert!(test_downstairs_set .downstairs1 .tempdir @@ -2524,20 +2510,6 @@ mod test { let mut test_downstairs_set = TestDownstairsSet::small_sqlite(false).await?; // This must be a SQLite extent! - println!( - "{:?}", - test_downstairs_set - .downstairs1 - .tempdir - .path() - .join("00/000/000.db") - ); - let out = std::process::Command::new("tree") - .arg(test_downstairs_set.downstairs1.tempdir.path()) - .output() - .expect("failed to execute process"); - println!("{}", std::str::from_utf8(&out.stdout).unwrap()); - assert!(test_downstairs_set .downstairs1 .tempdir From 7c6c217a15eec0e27bce0b145474d163ad1799c6 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 13:55:47 -0400 Subject: [PATCH 47/62] Fix comment --- downstairs/src/extent.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/downstairs/src/extent.rs b/downstairs/src/extent.rs index 72cbbbf63..345751233 100644 --- a/downstairs/src/extent.rs +++ b/downstairs/src/extent.rs @@ -332,8 +332,7 @@ impl Extent { // .db file) // - Open the extent using our existing SQLite extent code // - Using standard extent APIs, find the metadata and encryption - // context for each block. Encode this in the new raw file format - // (as a Vec) + // context for each block. Append this to the existing data file. // - Close the (SQLite) extent // - Open the extent data file in append mode, and append the new // raw metadata + block contexts to the end of the file. From ec5cb52e02e92c006c3791af271c7dd1aa30e979 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 13:56:24 -0400 Subject: [PATCH 48/62] Typo --- downstairs/src/extent_inner_raw.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index b117baf73..a3079febb 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -835,7 +835,7 @@ impl RawInner { &mut self, block_contexts: &[DownstairsBlockContext], ) -> Result<(), CrucibleError> { - // If any of these block contexts will be overwriting an unsyched + // If any of these block contexts will be overwriting an unsynched // context slot, then we insert a sync here. let needs_sync = block_contexts.iter().any(|block_context| { let block = block_context.block as usize; From 856c18b89415cbe2adf550886b1d839e88d2a38e Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 13:59:05 -0400 Subject: [PATCH 49/62] More comments --- integration_tests/src/lib.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/lib.rs b/integration_tests/src/lib.rs index 66836def8..4c90d3760 100644 --- a/integration_tests/src/lib.rs +++ b/integration_tests/src/lib.rs @@ -2501,12 +2501,10 @@ mod test { #[tokio::test] async fn integration_test_sqlite_migration() -> Result<()> { - // Test using an old SQLite backend as a read-only parent. - const BLOCK_SIZE: usize = 512; - // boot three downstairs, write some data to them, then change to - // read-only. + // boot three downstairs, write some data to them, then reopen as + // read-write (which will automatically migrate the extent) let mut test_downstairs_set = TestDownstairsSet::small_sqlite(false).await?; // This must be a SQLite extent! From b6ef63a56a97aa74ad973061854ce71251de4b73 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 14:53:57 -0400 Subject: [PATCH 50/62] Add some assertions --- downstairs/src/extent_inner_raw.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index a3079febb..eeb0d28a0 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -541,6 +541,8 @@ impl RawInner { ) -> Result<(), CrucibleError> { let layout = RawLayout::new(def.extent_size()); let block_count = layout.block_count() as usize; + assert_eq!(block_count, def.extent_size().value as usize); + assert_eq!(block_count, ctxs.len()); file.set_len(layout.file_size())?; layout.write_context_slots_contiguous( From dc81096969a3507d50ae457eb5719cd605b95375 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 15:11:33 -0400 Subject: [PATCH 51/62] Add assertion --- downstairs/src/extent_inner_raw.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index eeb0d28a0..b8459d8b1 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1317,6 +1317,10 @@ impl RawLayout { /// Write out the active context array and metadata section of the file /// /// This is done in a single write, so it should be atomic. + /// + /// # Panics + /// `active_context.len()` must match `self.block_count()`, and the function + /// will panic otherwise. fn write_active_context_and_metadata( &self, file: &File, @@ -1325,6 +1329,8 @@ impl RawLayout { flush_number: u64, gen_number: u64, ) -> Result<(), CrucibleError> { + assert_eq!(active_context.len(), self.block_count() as usize); + // Serialize bitpacked active slot values let mut buf = self.buf.take(); buf.clear(); From e94ea4e4fa5d20ceb8f4fa05afc0bad86c1110a8 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Mon, 30 Oct 2023 17:11:38 -0400 Subject: [PATCH 52/62] More defragment tests --- downstairs/src/extent_inner_raw.rs | 339 ++++++++++++++++++++++++++++- 1 file changed, 338 insertions(+), 1 deletion(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index b8459d8b1..0e4f8eae1 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1908,7 +1908,7 @@ mod test { } #[test] - fn test_defragment() -> Result<()> { + fn test_defragment_full() -> Result<()> { let dir = tempdir()?; let mut inner = RawInner::create(dir.as_ref(), &new_region_definition(), 0) @@ -2009,6 +2009,343 @@ mod test { Ok(()) } + #[test] + fn test_defragment_into_b() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write every other block, so that the active context slot alternates + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + + for i in 0..3 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i * 2), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + inner.write(JobId(30), &[&write], false, IOV_MAX_TEST)?; + } + // 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 + // --|---|---|---|---|---|---|---|---|--- + // B | A | B | A | B | A | A | A | A | A + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 4 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + assert_eq!(inner.extra_syscall_count, 0); + assert_eq!(inner.extra_syscall_denominator, 3); + inner.flush(10, 10, JobId(10).into())?; + assert_eq!(inner.sync_index, 1); + + // This should not have changed active context slots! + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 4 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + + // Now, do one big write, which will be forced to bounce between the + // context slots. + let mut writes = vec![]; + let data = Bytes::from(vec![0x33; 512]); + let hash = integrity_hash(&[&data[..]]); + for i in 0..10 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + writes.push(write); + } + // This write should toggled every single context slot: + // 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 + // --|---|---|---|---|---|---|---|---|--- + // A | B | A | B | A | B | B | B | B | B + let writes_ref: Vec<&_> = writes.iter().collect(); + inner.write(JobId(30), &writes_ref, false, IOV_MAX_TEST)?; + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 4 { + ContextSlot::A + } else { + ContextSlot::B + }, + "invalid context slot at {i}", + ); + } + assert!(inner.extra_syscall_count > 0); + assert_eq!(inner.extra_syscall_denominator, 1); + + // Do a flush! Because we had a bunch of extra syscalls, this should + // trigger defragmentation; after the flush, every context slot should + // be in array B (which minimizes copies) + assert_eq!(inner.sync_index, 1); + inner.flush(11, 11, JobId(11).into())?; + assert_eq!(inner.sync_index, 2); + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + ContextSlot::B, + "invalid context slot at {i}", + ); + } + + Ok(()) + } + + #[test] + fn test_defragment_into_a() -> Result<()> { + // Identical to `test_defragment_a`, except that we force the + // defragmentation to copy into the A slots + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write every other block, so that the active context slot alternates + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + + for i in 0..3 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i * 2), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + inner.write(JobId(30), &[&write], false, IOV_MAX_TEST)?; + } + // 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 + // --|---|---|---|---|---|---|---|---|--- + // B | A | B | A | B | A | A | A | A | A + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 4 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + assert_eq!(inner.extra_syscall_count, 0); + assert_eq!(inner.extra_syscall_denominator, 3); + inner.flush(10, 10, JobId(10).into())?; + assert_eq!(inner.sync_index, 1); + + // This should not have changed active context slots! + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 4 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + + // Now, do two big writes, which will be forced to bounce between the + // context slots. + let mut writes = vec![]; + let data = Bytes::from(vec![0x33; 512]); + let hash = integrity_hash(&[&data[..]]); + for i in 0..10 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + writes.push(write); + } + let writes_ref: Vec<&_> = writes.iter().collect(); + inner.write(JobId(30), &writes_ref, false, IOV_MAX_TEST)?; + inner.write(JobId(30), &writes_ref, false, IOV_MAX_TEST)?; + // This write should toggled every single context slot: + // 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 + // --|---|---|---|---|---|---|---|---|--- + // B | A | B | A | B | A | A | A | A | A + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 4 { + ContextSlot::B + } else { + ContextSlot::A + }, + "invalid context slot at {i}", + ); + } + assert!(inner.extra_syscall_count > 0); + assert_eq!(inner.extra_syscall_denominator, 2); + + // Do a flush! Because we had a bunch of extra syscalls, this should + // trigger defragmentation; after the flush, every context slot should + // be in array A (which minimizes copies) + assert_eq!(inner.sync_index, 1); + inner.flush(11, 11, JobId(11).into())?; + assert_eq!(inner.sync_index, 2); + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + ContextSlot::A, + "invalid context slot at {i}", + ); + } + + Ok(()) + } + + #[test] + fn test_defragment_not_enough() -> Result<()> { + // Identical to `test_defragment_a`, except that we force the + // defragmentation to copy into the A slots + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write every other block, so that the active context slot alternates + let data = Bytes::from(vec![0x55; 512]); + let hash = integrity_hash(&[&data[..]]); + + for i in 0..2 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i * 2), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + inner.write(JobId(30), &[&write], false, IOV_MAX_TEST)?; + } + // 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 + // --|---|---|---|---|---|---|---|---|--- + // B | A | B | A | A | A | A | A | A | A + + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 2 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + assert_eq!(inner.extra_syscall_count, 0); + assert_eq!(inner.extra_syscall_denominator, 2); + inner.flush(10, 10, JobId(10).into())?; + assert_eq!(inner.sync_index, 1); + + // This should not have changed active context slots! + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 2 { + ContextSlot::B + } else { + ContextSlot::A + } + ); + } + + // Now, do two big writes, which will be forced to bounce between the + // context slots. + let mut writes = vec![]; + let data = Bytes::from(vec![0x33; 512]); + let hash = integrity_hash(&[&data[..]]); + for i in 0..10 { + let write = crucible_protocol::Write { + eid: 0, + offset: Block::new_512(i), + data: data.clone(), + block_context: BlockContext { + encryption_context: None, + hash, + }, + }; + writes.push(write); + } + let writes_ref: Vec<&_> = writes.iter().collect(); + inner.write(JobId(30), &writes_ref, false, IOV_MAX_TEST)?; + inner.write(JobId(30), &writes_ref, false, IOV_MAX_TEST)?; + // This write should toggled every single context slot: + // 0 | 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 + // --|---|---|---|---|---|---|---|---|--- + // B | A | B | A | A | A | A | A | A | A + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 2 { + ContextSlot::B + } else { + ContextSlot::A + }, + "invalid context slot at {i}", + ); + } + assert!(inner.extra_syscall_count > 0); + assert_eq!(inner.extra_syscall_denominator, 2); + + // This write didn't add enough extra syscalls to trigger + // defragmentation. + assert_eq!(inner.sync_index, 1); + inner.flush(11, 11, JobId(11).into())?; + assert_eq!(inner.sync_index, 2); + + // These should be the same! + for i in 0..10 { + assert_eq!( + inner.active_context[i], + if i % 2 == 0 && i <= 2 { + ContextSlot::B + } else { + ContextSlot::A + }, + "invalid context slot at {i}", + ); + } + + Ok(()) + } + #[test] fn test_serialized_sizes() { let c = OnDiskDownstairsBlockContext { From fb59d22f8dcfc2ccc981bc8fc84ea1ea5d2d72bf Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Fri, 3 Nov 2023 10:28:54 -0400 Subject: [PATCH 53/62] Check that defragment is only called on a clean extent --- downstairs/src/extent_inner_raw.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 0e4f8eae1..6ed9e1579 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1053,6 +1053,8 @@ impl RawInner { // We track the number of A vs B slots, as well as the range covered by // the slots. It's that range that we'll need to read + write, so we // want to pick whichever slot does less work. + assert!(!self.dirty); // This can only be called after a flush! + #[derive(Copy, Clone)] struct Counter { count: usize, From b66519ddd77a233d96a427379814b6cc450067b8 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 09:49:19 -0500 Subject: [PATCH 54/62] Add multiple-write-to-same-block test --- downstairs/src/extent_inner_raw.rs | 510 +++++++++++++++++++---------- 1 file changed, 340 insertions(+), 170 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 6ed9e1579..b055ad7b2 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -121,6 +121,12 @@ pub struct RawInner { /// Denominator corresponding to `extra_syscall_count` extra_syscall_denominator: u64, + + /// Scratch data for doing a quick block uniqueness check + block_unique_scratch: Vec, + + /// Scratch index for doing a quick block uniqueness check + block_unique_index: u8, } #[derive(Copy, Clone, Debug, Eq, PartialEq)] @@ -159,180 +165,39 @@ impl ExtentInner for RawInner { only_write_unwritten: bool, iov_max: usize, ) -> Result<(), CrucibleError> { - /* - * In order to be crash consistent, perform the following steps in - * order: - * - * 1) set the dirty bit - * 2) for each write: - * a) write out encryption context and hashes first - * b) write out extent data second - * - * If encryption context is written after the extent data, a crash or - * interruption before extent data is written would potentially leave - * data on the disk that cannot be decrypted. - * - * If hash is written after extent data, same thing - a crash or - * interruption would leave data on disk that would fail the - * integrity hash check. - * - * Note that writing extent data here does not assume that it is - * durably on disk - the only guarantee of that is returning - * ok from fsync. The data is only potentially on disk and - * this depends on operating system implementation. - * - * To minimize the performance hit of sending many transactions to the - * filesystem, as much as possible is written at the same time. This - * means multiple loops are required. The steps now look like: - * - * 1) set the dirty bit - * 2) gather and write all encryption contexts + hashes - * 3) write all extent data - * - * If "only_write_unwritten" is true, then we only issue a write for - * a block if that block has not been written to yet. Note - * that we can have a write that is "sparse" if the range of - * blocks it contains has a mix of written an unwritten - * blocks. - * - * We define a block being written to or not has if that block has - * `Some(...)` with a matching checksum serialized into a context slot - * or not. So it is required that a written block has a checksum. - */ - - // If `only_write_written`, we need to skip writing to blocks that - // already contain data. We'll first query the metadata to see which - // blocks have hashes - let mut writes_to_skip = HashSet::new(); - if only_write_unwritten { - cdt::extent__write__get__hashes__start!(|| { - (job_id.0, self.extent_number, writes.len() as u64) - }); - let mut write_run_start = 0; - while write_run_start < writes.len() { - let first_write = writes[write_run_start]; - - // Starting from the first write in the potential run, we scan - // forward until we find a write with a block that isn't - // contiguous with the request before it. Since we're counting - // pairs, and the number of pairs is one less than the number of - // writes, we need to add 1 to get our actual run length. - let n_contiguous_writes = writes[write_run_start..] - .windows(2) - .take_while(|wr_pair| { - wr_pair[0].offset.value + 1 == wr_pair[1].offset.value - }) - .count() - + 1; - - // Query hashes for the write range. - let block_contexts = self.get_block_contexts( - first_write.offset.value, - n_contiguous_writes as u64, - )?; - - for (i, block_contexts) in block_contexts.iter().enumerate() { - if block_contexts.is_some() { - let _ = writes_to_skip - .insert(i as u64 + first_write.offset.value); - } - } - - write_run_start += n_contiguous_writes; - } - cdt::extent__write__get__hashes__done!(|| { - (job_id.0, self.extent_number, writes.len() as u64) - }); - - if writes_to_skip.len() == writes.len() { - // Nothing to do - return Ok(()); - } - } - - self.set_dirty()?; - - // Write all the context data to the raw file - // - // TODO right now we're including the integrity_hash() time in the - // measured time. Is it small enough to be ignored? - cdt::extent__write__raw__context__insert__start!(|| { - (job_id.0, self.extent_number, writes.len() as u64) - }); - - // Compute block contexts, then write them to disk - let block_ctx: Vec<_> = writes - .iter() - .filter(|write| !writes_to_skip.contains(&write.offset.value)) - .map(|write| { - // TODO it would be nice if we could profile what % of time we're - // spending on hashes locally vs writing to disk - let on_disk_hash = integrity_hash(&[&write.data[..]]); - - DownstairsBlockContext { - block_context: write.block_context, - block: write.offset.value, - on_disk_hash, - } - }) - .collect(); - - self.set_block_contexts(&block_ctx)?; - - cdt::extent__write__raw__context__insert__done!(|| { - (job_id.0, self.extent_number, writes.len() as u64) - }); - - // PERFORMANCE TODO: - // - // Something worth considering for small writes is that, based on - // my memory of conversations we had with propolis folks about what - // OSes expect out of an NVMe driver, I believe our contract with the - // upstairs doesn't require us to have the writes inside the file - // until after a flush() returns. If that is indeed true, we could - // buffer a certain amount of writes, only actually writing that - // buffer when either a flush is issued or the buffer exceeds some - // set size (based on our memory constraints). This would have - // benefits on any workload that frequently writes to the same block - // between flushes, would have benefits for small contiguous writes - // issued over multiple write commands by letting us batch them into - // a larger write, and (speculation) may benefit non-contiguous writes - // by cutting down the number of metadata writes. But, it introduces - // complexity. The time spent implementing that would probably better be - // spent switching to aio or something like that. - cdt::extent__write__file__start!(|| { - (job_id.0, self.extent_number, writes.len() as u64) - }); - - let r = self.write_inner(writes, &writes_to_skip, iov_max); - - if r.is_err() { - for write in writes.iter() { - let block = write.offset.value; - if !writes_to_skip.contains(&block) { - // Try to recompute the context slot from the file. If this - // fails, then we _really_ can't recover, so bail out - // unceremoniously. - self.recompute_slot_from_file(block).unwrap(); - } - } + // If the same block is written multiple times in a single write, then + // (1) that's weird, and (2) we need to handle it specially. + if self.are_blocks_unique(writes.iter().map(|v| v.offset.value)) { + self.write_without_overlaps( + job_id, + writes, + only_write_unwritten, + iov_max, + ) } else { - // Now that writes have gone through, update active context slots - for write in writes.iter() { - let block = write.offset.value; - if !writes_to_skip.contains(&block) { - // We always write to the inactive slot, so just swap it - self.active_context[block as usize] = - !self.active_context[block as usize]; + // Special handle for writes which touch the same block multiple + // times! We split the `writes` slice into sub-slices of unique + // writes. + let mut seen = HashSet::new(); + let mut start = 0; + for i in 0..=writes.len() { + // If this value is a duplicate or we have reached the end of + // the list, then write everything up to this point and adjust + // our starting point and `seen` array + if i == writes.len() || !seen.insert(writes[i].offset.value) { + self.write_without_overlaps( + job_id, + &writes[start..i], + only_write_unwritten, + iov_max, + )?; + seen.clear(); + seen.extend(writes.get(i).map(|w| w.offset.value)); + start = i; } } + Ok(()) } - - cdt::extent__write__file__done!(|| { - (job_id.0, self.extent_number, writes.len() as u64) - }); - - Ok(()) } fn read( @@ -605,6 +470,8 @@ impl RawInner { sync_index: 0, extra_syscall_count: 0, extra_syscall_denominator: 0, + block_unique_scratch: vec![0; def.extent_size().value as usize], + block_unique_index: 0, }; // Setting the flush number also writes the extent version, since // they're serialized together in the same block. @@ -766,6 +633,8 @@ impl RawInner { extra_syscall_count: 0, extra_syscall_denominator: 0, sync_index: 0, + block_unique_scratch: vec![0; def.extent_size().value as usize], + block_unique_index: 0, }) } @@ -1135,6 +1004,213 @@ impl RawInner { } r.map(|_| ()) } + + /// Implementation details for `ExtentInner::write` + /// + /// This function requires that `writes` not have any overlapping writes, + /// i.e. blocks that are written multiple times. We write the contexts + /// first, then block data; if a single block is written multiple times, + /// then we'd write multiple contexts, then multiple block data, and it + /// would be possible for them to get out of sync. + fn write_without_overlaps( + &mut self, + job_id: JobId, + writes: &[&crucible_protocol::Write], + only_write_unwritten: bool, + iov_max: usize, + ) -> Result<(), CrucibleError> { + /* + * In order to be crash consistent, perform the following steps in + * order: + * + * 1) set the dirty bit + * 2) for each write: + * a) write out encryption context and hashes first + * b) write out extent data second + * + * If encryption context is written after the extent data, a crash or + * interruption before extent data is written would potentially leave + * data on the disk that cannot be decrypted. + * + * If hash is written after extent data, same thing - a crash or + * interruption would leave data on disk that would fail the + * integrity hash check. + * + * Note that writing extent data here does not assume that it is + * durably on disk - the only guarantee of that is returning + * ok from fsync. The data is only potentially on disk and + * this depends on operating system implementation. + * + * To minimize the performance hit of sending many transactions to the + * filesystem, as much as possible is written at the same time. This + * means multiple loops are required. The steps now look like: + * + * 1) set the dirty bit + * 2) gather and write all encryption contexts + hashes + * 3) write all extent data + * + * If "only_write_unwritten" is true, then we only issue a write for + * a block if that block has not been written to yet. Note + * that we can have a write that is "sparse" if the range of + * blocks it contains has a mix of written an unwritten + * blocks. + * + * We define a block being written to or not has if that block has + * `Some(...)` with a matching checksum serialized into a context slot + * or not. So it is required that a written block has a checksum. + */ + + // If `only_write_written`, we need to skip writing to blocks that + // already contain data. We'll first query the metadata to see which + // blocks have hashes + let mut writes_to_skip = HashSet::new(); + if only_write_unwritten { + cdt::extent__write__get__hashes__start!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + let mut write_run_start = 0; + while write_run_start < writes.len() { + let first_write = writes[write_run_start]; + + // Starting from the first write in the potential run, we scan + // forward until we find a write with a block that isn't + // contiguous with the request before it. Since we're counting + // pairs, and the number of pairs is one less than the number of + // writes, we need to add 1 to get our actual run length. + let n_contiguous_writes = writes[write_run_start..] + .windows(2) + .take_while(|wr_pair| { + wr_pair[0].offset.value + 1 == wr_pair[1].offset.value + }) + .count() + + 1; + + // Query hashes for the write range. + let block_contexts = self.get_block_contexts( + first_write.offset.value, + n_contiguous_writes as u64, + )?; + + for (i, block_contexts) in block_contexts.iter().enumerate() { + if block_contexts.is_some() { + let _ = writes_to_skip + .insert(i as u64 + first_write.offset.value); + } + } + + write_run_start += n_contiguous_writes; + } + cdt::extent__write__get__hashes__done!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + if writes_to_skip.len() == writes.len() { + // Nothing to do + return Ok(()); + } + } + + self.set_dirty()?; + + // Write all the context data to the raw file + // + // TODO right now we're including the integrity_hash() time in the + // measured time. Is it small enough to be ignored? + cdt::extent__write__raw__context__insert__start!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + // Compute block contexts, then write them to disk + let block_ctx: Vec<_> = writes + .iter() + .filter(|write| !writes_to_skip.contains(&write.offset.value)) + .map(|write| { + // TODO it would be nice if we could profile what % of time we're + // spending on hashes locally vs writing to disk + let on_disk_hash = integrity_hash(&[&write.data[..]]); + + DownstairsBlockContext { + block_context: write.block_context, + block: write.offset.value, + on_disk_hash, + } + }) + .collect(); + + self.set_block_contexts(&block_ctx)?; + + cdt::extent__write__raw__context__insert__done!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + // PERFORMANCE TODO: + // + // Something worth considering for small writes is that, based on + // my memory of conversations we had with propolis folks about what + // OSes expect out of an NVMe driver, I believe our contract with the + // upstairs doesn't require us to have the writes inside the file + // until after a flush() returns. If that is indeed true, we could + // buffer a certain amount of writes, only actually writing that + // buffer when either a flush is issued or the buffer exceeds some + // set size (based on our memory constraints). This would have + // benefits on any workload that frequently writes to the same block + // between flushes, would have benefits for small contiguous writes + // issued over multiple write commands by letting us batch them into + // a larger write, and (speculation) may benefit non-contiguous writes + // by cutting down the number of metadata writes. But, it introduces + // complexity. The time spent implementing that would probably better be + // spent switching to aio or something like that. + cdt::extent__write__file__start!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + let r = self.write_inner(writes, &writes_to_skip, iov_max); + + if r.is_err() { + for write in writes.iter() { + let block = write.offset.value; + if !writes_to_skip.contains(&block) { + // Try to recompute the context slot from the file. If this + // fails, then we _really_ can't recover, so bail out + // unceremoniously. + self.recompute_slot_from_file(block).unwrap(); + } + } + } else { + // Now that writes have gone through, update active context slots + for write in writes.iter() { + let block = write.offset.value; + if !writes_to_skip.contains(&block) { + // We always write to the inactive slot, so just swap it + self.active_context[block as usize] = + !self.active_context[block as usize]; + } + } + } + + cdt::extent__write__file__done!(|| { + (job_id.0, self.extent_number, writes.len() as u64) + }); + + Ok(()) + } + + /// Cheap test that a set of blocks are unique + /// + /// Under rare circumstances, the test may return a false negative – saying + /// that blocks are not unique when they in fact are. That's fine for our + /// use case. + fn are_blocks_unique>(&mut self, iter: I) -> bool { + self.block_unique_index = self.block_unique_index.wrapping_add(1); + for i in iter { + if self.block_unique_scratch[i as usize] == self.block_unique_index + { + return false; + } + self.block_unique_scratch[i as usize] = self.block_unique_index; + } + true + } } /// Data structure that implements the on-disk layout of a raw extent file @@ -2372,4 +2448,98 @@ mod test { let mut meta_buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; bincode::serialize_into(meta_buf.as_mut_slice(), &Some(m)).unwrap(); } + + /// Test that multiple writes to the same location work + #[test] + fn test_multiple_writes_to_same_location_raw() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + + // Write the same block four times in the same write command. + + let writes: Vec<_> = (0..4) + .map(|i| { + let data = Bytes::from(vec![i as u8; 512]); + let hash = integrity_hash(&[&data[..]]); + + crucible_protocol::Write { + eid: 0, + offset: Block::new_512(0), + data, + block_context: BlockContext { + encryption_context: None, + hash, + }, + } + }) + .collect(); + + let write_refs: Vec<_> = writes.iter().collect(); + + assert_eq!(inner.sync_index, 0); + assert_eq!(inner.context_slot_synched_at[0], [0, 0]); + inner.write(JobId(30), &write_refs, false, IOV_MAX_TEST)?; + + // The write should be split into four separate calls to + // `write_without_overlaps`, triggering one bonus fsync. + assert_eq!(inner.sync_index, 1); + assert_eq!(inner.context_slot_synched_at[0], [2, 2]); + + // Block 0 should be 0x03 repeated. + let mut resp = Vec::new(); + let read = ReadRequest { + eid: 0, + offset: Block::new_512(0), + }; + + inner.read(JobId(31), &[&read], &mut resp, IOV_MAX_TEST)?; + + let data = Bytes::from(vec![0x03; 512]); + let hash = integrity_hash(&[&data[..]]); + let block_context = BlockContext { + encryption_context: None, + hash, + }; + + assert_eq!( + resp, + vec![ReadResponse { + eid: 0, + offset: Block::new_512(0), + data: BytesMut::from(data.as_ref()), + // Only the most recent block context should be returned + block_contexts: vec![block_context], + }] + ); + + Ok(()) + } + + #[test] + fn test_are_blocks_unique() -> Result<()> { + let dir = tempdir()?; + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + assert!(inner.are_blocks_unique([0, 1, 2, 3].into_iter())); + assert!(!inner.are_blocks_unique([0, 1, 2, 0].into_iter())); + assert!(!inner.are_blocks_unique([0, 1, 1, 4].into_iter())); + assert!(inner.are_blocks_unique([4, 3, 2, 1].into_iter())); + + // Test the false-negative behavior. It's not load-bearing, but this is + // a good check for our understanding of the data structure. + let mut inner = + RawInner::create(dir.as_ref(), &new_region_definition(), 0) + .unwrap(); + for _ in 0..u8::MAX { + assert!(inner.are_blocks_unique([1].into_iter())); + } + assert!(!inner.are_blocks_unique([0].into_iter())); + + // After a single false negative, we should see correct values + assert!(inner.are_blocks_unique([0].into_iter())); + Ok(()) + } } From 57a52b67518807dadb1301dc703413b5876146b5 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 10:01:48 -0500 Subject: [PATCH 55/62] Post-rebase fixes --- common/src/lib.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/common/src/lib.rs b/common/src/lib.rs index c68571e5c..b7f6aef47 100644 --- a/common/src/lib.rs +++ b/common/src/lib.rs @@ -395,7 +395,10 @@ impl From for dropshot::HttpError { | CrucibleError::SubvolumeSizeMismatch | CrucibleError::UpstairsAlreadyActive | CrucibleError::UpstairsDeactivating - | CrucibleError::UuidMismatch => { + | CrucibleError::UuidMismatch + | CrucibleError::MissingContextSlot(..) + | CrucibleError::BadMetadata(..) + | CrucibleError::BadContextSlot(..) => { dropshot::HttpError::for_internal_error(e.to_string()) } } From a8b3705f33851d1c4a6d66916b066f0d5e299b02 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 10:37:22 -0500 Subject: [PATCH 56/62] Restore buf after `get_active_contexts` --- downstairs/src/extent_inner_raw.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index b055ad7b2..0b140f049 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1471,6 +1471,7 @@ impl RawLayout { }); } assert_eq!(active_context.len(), self.block_count() as usize); + self.buf.set(buf); Ok(active_context) } } From 416094efa98920643147ede936977b446f3db262 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 10:39:47 -0500 Subject: [PATCH 57/62] Update comment --- downstairs/src/region.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 991ea28ec..61804ea5c 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -27,7 +27,7 @@ use crate::extent::{ }; /** - * Validate a list of sorted repair files. + * Validate the repair file. * * There is only one repair file: the raw file itself (which also contains * structured context and metadata at the end). From 2a5ed27c32caa2922e620a5e5da7cb467ceeff4f Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 11:23:53 -0500 Subject: [PATCH 58/62] Add some handling of partial pread/pwrite(v) --- downstairs/src/extent_inner_raw.rs | 109 ++++++++++++++++++++--------- downstairs/src/region.rs | 11 ++- 2 files changed, 85 insertions(+), 35 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 0b140f049..8ce820648 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -249,9 +249,11 @@ impl ExtentInner for RawInner { } // Create what amounts to an iovec for each response data buffer. + let mut expected_bytes = 0; for resp in &mut responses[resp_run_start..][..n_contiguous_requests] { + expected_bytes += resp.data.len(); iovecs.push(IoSliceMut::new(&mut resp.data[..])); } @@ -260,7 +262,10 @@ impl ExtentInner for RawInner { (job_id.0, self.extent_number, n_contiguous_requests as u64) }); - nix::sys::uio::preadv( + // Perform the bulk read, then check against the expected number of + // bytes. We could do more robust error handling here (e.g. + // retrying in a loop), but for now, simply bailing out seems wise. + let num_bytes = nix::sys::uio::preadv( self.file.as_raw_fd(), &mut iovecs, first_req.offset.value as i64 * block_size as i64, @@ -271,6 +276,13 @@ impl ExtentInner for RawInner { self.extent_number )) })?; + if num_bytes != expected_bytes { + return Err(CrucibleError::IoError(format!( + "extent {}: incomplete read \ + (expected {expected_bytes}, got {num_bytes})", + self.extent_number + ))); + } cdt::extent__read__file__done!(|| { (job_id.0, self.extent_number, n_contiguous_requests as u64) @@ -661,7 +673,7 @@ impl RawInner { // Read the block data itself: let block_size = self.extent_size.block_size_in_bytes(); let mut buf = vec![0; block_size as usize]; - nix::sys::uio::pread( + pread_all( self.file.as_raw_fd(), &mut buf, (block_size as u64 * block) as i64, @@ -1248,12 +1260,9 @@ impl RawLayout { /// changed. fn set_dirty(&self, file: &File) -> Result<(), CrucibleError> { let offset = self.metadata_offset(); - nix::sys::uio::pwrite(file.as_raw_fd(), &[1u8], offset as i64) - .map_err(|e| { - CrucibleError::IoError(format!( - "writing dirty byte failed: {e}", - )) - })?; + pwrite_all(file.as_raw_fd(), &[1u8], offset as i64).map_err(|e| { + CrucibleError::IoError(format!("writing dirty byte failed: {e}",)) + })?; Ok(()) } @@ -1312,10 +1321,9 @@ impl RawLayout { fn get_metadata(&self, file: &File) -> Result { let mut buf = [0u8; BLOCK_META_SIZE_BYTES as usize]; let offset = self.metadata_offset(); - nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| { - CrucibleError::IoError(format!("reading metadata failed: {e}")) - })?; + pread_all(file.as_raw_fd(), &mut buf, offset as i64).map_err(|e| { + CrucibleError::IoError(format!("reading metadata failed: {e}")) + })?; let out: OnDiskMeta = bincode::deserialize(&buf) .map_err(|e| CrucibleError::BadMetadata(e.to_string()))?; Ok(out) @@ -1344,13 +1352,9 @@ impl RawLayout { bincode::serialize_into(&mut buf[n..], &d).unwrap(); } let offset = self.context_slot_offset(block_start, slot); - nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64).map_err( - |e| { - CrucibleError::IoError(format!( - "writing context slots failed: {e}" - )) - }, - )?; + pwrite_all(file.as_raw_fd(), &buf, offset as i64).map_err(|e| { + CrucibleError::IoError(format!("writing context slots failed: {e}")) + })?; self.buf.set(buf); Ok(()) } @@ -1366,12 +1370,9 @@ impl RawLayout { buf.resize((BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count) as usize, 0u8); let offset = self.context_slot_offset(block_start, slot); - nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| { - CrucibleError::IoError(format!( - "reading context slots failed: {e}" - )) - })?; + pread_all(file.as_raw_fd(), &mut buf, offset as i64).map_err(|e| { + CrucibleError::IoError(format!("reading context slots failed: {e}")) + })?; let mut out = vec![]; for (i, chunk) in buf @@ -1432,9 +1433,9 @@ impl RawLayout { let offset = self.active_context_offset(); - nix::sys::uio::pwrite(file.as_raw_fd(), &buf, offset as i64).map_err( - |e| CrucibleError::IoError(format!("writing metadata failed: {e}")), - )?; + pwrite_all(file.as_raw_fd(), &buf, offset as i64).map_err(|e| { + CrucibleError::IoError(format!("writing metadata failed: {e}")) + })?; self.buf.set(buf); Ok(()) @@ -1450,12 +1451,11 @@ impl RawLayout { let mut buf = self.buf.take(); buf.resize(self.active_context_size() as usize, 0u8); let offset = self.active_context_offset(); - nix::sys::uio::pread(file.as_raw_fd(), &mut buf, offset as i64) - .map_err(|e| { - CrucibleError::IoError(format!( - "could not read active contexts: {e}" - )) - })?; + pread_all(file.as_raw_fd(), &mut buf, offset as i64).map_err(|e| { + CrucibleError::IoError(format!( + "could not read active contexts: {e}" + )) + })?; let mut active_context = vec![]; for bit in buf @@ -1476,6 +1476,47 @@ impl RawLayout { } } +/// Call `pread` repeatedly to read an entire buffer +/// +/// Quoth the standard, +/// +/// > The value returned may be less than nbyte if the number of bytes left in +/// > the file is less than nbyte, if the read() request was interrupted by a +/// > signal, or if the file is a pipe or FIFO or special file and has fewer +/// > than nbyte bytes immediately available for reading. For example, a read() +/// > from a file associated with a terminal may return one typed line of data. +/// +/// We don't have to worry about most of these conditions, but it may be +/// possible for Crucible to be interrupted by a signal, so let's play it safe. +fn pread_all( + fd: std::os::fd::RawFd, + mut buf: &mut [u8], + mut offset: i64, +) -> Result<(), nix::errno::Errno> { + while !buf.is_empty() { + let n = nix::sys::uio::pread(fd, buf, offset)?; + offset += n as i64; + buf = &mut buf[n..]; + } + Ok(()) +} + +/// Call `pwrite` repeatedly to write an entire buffer +/// +/// See details for why this is necessary in [`pread_all`] +fn pwrite_all( + fd: std::os::fd::RawFd, + mut buf: &[u8], + mut offset: i64, +) -> Result<(), nix::errno::Errno> { + while !buf.is_empty() { + let n = nix::sys::uio::pwrite(fd, buf, offset)?; + offset += n as i64; + buf = &buf[n..]; + } + Ok(()) +} + #[cfg(test)] mod test { use super::*; diff --git a/downstairs/src/region.rs b/downstairs/src/region.rs index 61804ea5c..ecd65f1eb 100644 --- a/downstairs/src/region.rs +++ b/downstairs/src/region.rs @@ -1074,6 +1074,7 @@ struct BatchedPwritevState<'a> { byte_offset: u64, iovecs: Vec>, next_block_in_run: u64, + expected_bytes: usize, } pub(crate) struct BatchedPwritev<'a> { @@ -1137,6 +1138,7 @@ impl<'a> BatchedPwritev<'a> { assert_eq!(block, state.next_block_in_run); state.iovecs.push(IoSlice::new(&write.data)); state.next_block_in_run += 1; + state.expected_bytes += write.data.len(); } else { // start fresh self.state = Some(BatchedPwritevState { @@ -1146,6 +1148,7 @@ impl<'a> BatchedPwritev<'a> { iovecs.push(IoSlice::new(&write.data)); iovecs }, + expected_bytes: write.data.len(), next_block_in_run: block + 1, }); } @@ -1158,12 +1161,18 @@ impl<'a> BatchedPwritev<'a> { if let Some(state) = &mut self.state { assert!(!state.iovecs.is_empty()); - nix::sys::uio::pwritev( + let n = nix::sys::uio::pwritev( self.fd, &state.iovecs[..], state.byte_offset as i64, ) .map_err(|e| CrucibleError::IoError(e.to_string()))?; + if n != state.expected_bytes { + return Err(CrucibleError::IoError(format!( + "pwritev incomplete (expected {}, got {n} bytes)", + state.expected_bytes + ))); + } self.state = None; } From 971ed1e97e24e38bbd24e6991abde4467126b5db Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 11:25:02 -0500 Subject: [PATCH 59/62] Explain dirty / context slot mismatch --- downstairs/src/extent_inner_raw.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 8ce820648..4364dfd81 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1013,6 +1013,14 @@ impl RawInner { for block in counter.min_block..=counter.max_block { self.active_context[block as usize] = !copy_from; } + // At this point, the `dirty` bit is not set, but values in + // `self.active_context` disagree with the active context slot + // stored in the file. Normally, this is bad: if we crashed and + // reloaded the file from disk right at this moment, we'd end up + // with different values in `self.active_context`. In this case, + // though, it's okay: the values that have changed have the **same + // data** in both context slots, so it would still be a valid state + // for the file. } r.map(|_| ()) } From 868bca4c5d87c413bdb005da6c3e962c1a3361a5 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 18:43:02 +0000 Subject: [PATCH 60/62] Remove buf (it doesn't make things faster and uses up RAM) --- downstairs/src/extent_inner_raw.rs | 29 ++++++----------------------- 1 file changed, 6 insertions(+), 23 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 4364dfd81..97a53f32b 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -1236,13 +1236,6 @@ impl RawInner { /// Data structure that implements the on-disk layout of a raw extent file struct RawLayout { extent_size: Block, - - /// Miscellaneous buffer for reading and writing - /// - /// This is simply to avoid churning through memory allocations. It is the - /// user's responsibility to take `buf` out of the cell when it's in use, - /// and return it when they're done with it. - buf: std::cell::Cell>, } impl std::fmt::Debug for RawLayout { @@ -1255,10 +1248,7 @@ impl std::fmt::Debug for RawLayout { impl RawLayout { fn new(extent_size: Block) -> Self { - RawLayout { - extent_size, - buf: std::cell::Cell::default(), - } + RawLayout { extent_size } } /// Sets the dirty flag in the file true @@ -1347,8 +1337,7 @@ impl RawLayout { where I: Iterator>, { - let mut buf = self.buf.take(); - buf.clear(); + let mut buf = vec![]; for block_context in iter { let n = buf.len(); @@ -1363,7 +1352,6 @@ impl RawLayout { pwrite_all(file.as_raw_fd(), &buf, offset as i64).map_err(|e| { CrucibleError::IoError(format!("writing context slots failed: {e}")) })?; - self.buf.set(buf); Ok(()) } @@ -1374,8 +1362,8 @@ impl RawLayout { block_count: u64, slot: ContextSlot, ) -> Result>, CrucibleError> { - let mut buf = self.buf.take(); - buf.resize((BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count) as usize, 0u8); + let mut buf = + vec![0u8; (BLOCK_CONTEXT_SLOT_SIZE_BYTES * block_count) as usize]; let offset = self.context_slot_offset(block_start, slot); pread_all(file.as_raw_fd(), &mut buf, offset as i64).map_err(|e| { @@ -1397,7 +1385,6 @@ impl RawLayout { on_disk_hash: c.on_disk_hash, })); } - self.buf.set(buf); Ok(out) } @@ -1419,8 +1406,7 @@ impl RawLayout { assert_eq!(active_context.len(), self.block_count() as usize); // Serialize bitpacked active slot values - let mut buf = self.buf.take(); - buf.clear(); + let mut buf = vec![]; for c in active_context.chunks(8) { let mut v = 0; for (i, slot) in c.iter().enumerate() { @@ -1444,7 +1430,6 @@ impl RawLayout { pwrite_all(file.as_raw_fd(), &buf, offset as i64).map_err(|e| { CrucibleError::IoError(format!("writing metadata failed: {e}")) })?; - self.buf.set(buf); Ok(()) } @@ -1456,8 +1441,7 @@ impl RawLayout { &self, file: &File, ) -> Result, CrucibleError> { - let mut buf = self.buf.take(); - buf.resize(self.active_context_size() as usize, 0u8); + let mut buf = vec![0u8; self.active_context_size() as usize]; let offset = self.active_context_offset(); pread_all(file.as_raw_fd(), &mut buf, offset as i64).map_err(|e| { CrucibleError::IoError(format!( @@ -1479,7 +1463,6 @@ impl RawLayout { }); } assert_eq!(active_context.len(), self.block_count() as usize); - self.buf.set(buf); Ok(active_context) } } From 0d4f5731cfd27fbaa628e9c593d25457b033752a Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 14:17:06 -0500 Subject: [PATCH 61/62] Simplify dirty context slot tracking to reduce RAM --- downstairs/src/extent_inner_raw.rs | 141 ++++++++++++++--------------- 1 file changed, 70 insertions(+), 71 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index 97a53f32b..c024bd77f 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -103,18 +103,14 @@ pub struct RawInner { /// This allows us to only write the flag when the value changes dirty: bool, - /// Monotonically increasing sync index + /// Marks whether the given context slot is dirty /// - /// This is unrelated to `flush_number` or `gen_number`; it's purely - /// internal to this data structure and is transient. We use this value to - /// determine whether a context slot has been persisted to disk or not. - sync_index: u64, - - /// The value of `sync_index` when the current value of a context slot was - /// synched to disk. When the value of a context slot changes, the value in - /// this array is set to `self.sync_index + 1` indicates that the slot has - /// not yet been synched. - context_slot_synched_at: Vec<[u64; 2]>, + /// A dirty context slot has not yet been saved to disk, and must be + /// synched before being overwritten. + /// + /// Context slots are stored as a 2-bit field, with bit 0 marking + /// `ContextSlot::A` and bit 1 marking `ContextSlot::B`. + context_slot_dirty: Vec, /// Total number of extra syscalls due to context slot fragmentation extra_syscall_count: u64, @@ -356,7 +352,7 @@ impl ExtentInner for RawInner { self.extent_number, ))); } - self.sync_index += 1; + self.context_slot_dirty.fill(0); cdt::extent__flush__file__done!(|| { (job_id.get(), self.extent_number, 0) }); @@ -475,11 +471,7 @@ impl RawInner { ContextSlot::A; // both slots are empty, so this is fine def.extent_size().value as usize ], - context_slot_synched_at: vec![ - [0, 0]; - def.extent_size().value as usize - ], - sync_index: 0, + context_slot_dirty: vec![0; def.extent_size().value as usize], extra_syscall_count: 0, extra_syscall_denominator: 0, block_unique_scratch: vec![0; def.extent_size().value as usize], @@ -638,13 +630,9 @@ impl RawInner { extent_number, extent_size: def.extent_size(), layout: RawLayout::new(def.extent_size()), - context_slot_synched_at: vec![ - [0, 0]; - def.extent_size().value as usize - ], + context_slot_dirty: vec![0; def.extent_size().value as usize], extra_syscall_count: 0, extra_syscall_denominator: 0, - sync_index: 0, block_unique_scratch: vec![0; def.extent_size().value as usize], block_unique_index: 0, }) @@ -724,10 +712,7 @@ impl RawInner { let block = block_context.block as usize; // We'll be writing to the inactive slot let slot = !self.active_context[block]; - let last_sync = self.context_slot_synched_at[block][slot as usize]; - // We should never be > 1 sync ahead! - assert!(last_sync <= self.sync_index + 1); - last_sync > self.sync_index + (self.context_slot_dirty[block] & (1 << slot as usize)) != 0 }); if needs_sync { self.file.sync_all().map_err(|e| { @@ -736,7 +721,7 @@ impl RawInner { self.extent_number, )) })?; - self.sync_index += 1; + self.context_slot_dirty.fill(0); } // Mark the to-be-written slots as unsynched on disk // @@ -746,8 +731,7 @@ impl RawInner { for block_context in block_contexts { let block = block_context.block as usize; let slot = !self.active_context[block]; - self.context_slot_synched_at[block][slot as usize] = - self.sync_index + 1; + self.context_slot_dirty[block] |= 1 << (slot as usize); } let mut start = 0; @@ -992,8 +976,7 @@ impl RawInner { // Mark this slot as unsynched, so that we don't overwrite // it later on without a sync - self.context_slot_synched_at[block][!copy_from as usize] = - self.sync_index + 1; + self.context_slot_dirty[block] |= 1 << (!copy_from as usize); } } let r = self.layout.write_context_slots_contiguous( @@ -1842,17 +1825,39 @@ mod test { hash, }, }; - // The context should be written to slot 0 + + // Write block 1, so that we can notice when a sync happens. The write + // should go to slot B. + let write1 = crucible_protocol::Write { + offset: Block::new_512(1), + ..write.clone() + }; + inner.write(JobId(10), &[&write1], false, IOV_MAX_TEST)?; + assert_eq!(inner.context_slot_dirty[0], 0b00); + assert_eq!(inner.active_context[0], ContextSlot::A); + assert_eq!(inner.context_slot_dirty[1], 0b10); + assert_eq!(inner.active_context[1], ContextSlot::B); + + // The context should be written to block 0, slot B inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 0); + assert_eq!(inner.context_slot_dirty[0], 0b10); + assert_eq!(inner.active_context[0], ContextSlot::B); + assert_eq!(inner.context_slot_dirty[1], 0b10); // unchanged + assert_eq!(inner.active_context[1], ContextSlot::B); // unchanged - // The context should be written to slot 1 + // The context should be written to block 0, slot A inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 0); + assert_eq!(inner.context_slot_dirty[0], 0b11); + assert_eq!(inner.active_context[0], ContextSlot::A); + assert_eq!(inner.context_slot_dirty[1], 0b10); // unchanged + assert_eq!(inner.active_context[1], ContextSlot::B); // unchanged - // The context should be written to slot 0, forcing a sync + // The context should be written to slot B, forcing a sync inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.context_slot_dirty[0], 0b10); + assert_eq!(inner.active_context[0], ContextSlot::B); + assert_eq!(inner.context_slot_dirty[1], 0b00); + assert_eq!(inner.active_context[1], ContextSlot::B); // unchanged Ok(()) } @@ -1879,27 +1884,27 @@ mod test { // The context should be written to slot B inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::B); - assert_eq!(inner.sync_index, 0); + assert_eq!(inner.context_slot_dirty[0], 0b10); - // Flush! This will bump the sync number, marking slot B as synched + // Flush! This will mark all slots as synched inner.flush(12, 12, JobId(11).into())?; assert_eq!(inner.active_context[0], ContextSlot::B); - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.context_slot_dirty[0], 0b00); // The context should be written to slot A inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::A); - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.context_slot_dirty[0], 0b01); // The context should be written to slot B inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::B); - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.context_slot_dirty[0], 0b11); // The context should be written to slot A, forcing a sync inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; assert_eq!(inner.active_context[0], ContextSlot::A); - assert_eq!(inner.sync_index, 2); + assert_eq!(inner.context_slot_dirty[0], 0b01); Ok(()) } @@ -1923,28 +1928,35 @@ mod test { hash, }, }; - // The context should be written to slot 0 + // The context should be written to slot B inner.write(JobId(10), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 0); + assert_eq!(inner.active_context[0], ContextSlot::B); + assert_eq!(inner.context_slot_dirty[0], 0b10); - // The context should be written to slot 1 + // The context should be written to slot A inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 0); + assert_eq!(inner.active_context[0], ContextSlot::A); + assert_eq!(inner.context_slot_dirty[0], 0b11); - // Flush, which should bump the sync number (marking slot 0 as synched) + // Flush! This will mark all slots as synched inner.flush(12, 12, JobId(11).into())?; + assert_eq!(inner.active_context[0], ContextSlot::A); + assert_eq!(inner.context_slot_dirty[0], 0b00); - // The context should be written to slot 0 + // The context should be written to slot B inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.active_context[0], ContextSlot::B); + assert_eq!(inner.context_slot_dirty[0], 0b10); - // The context should be written to slot 1 + // The context should be written to slot A inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 1); + assert_eq!(inner.active_context[0], ContextSlot::A); + assert_eq!(inner.context_slot_dirty[0], 0b11); - // The context should be written to slot 0, forcing a sync - inner.write(JobId(12), &[&write], false, IOV_MAX_TEST)?; - assert_eq!(inner.sync_index, 2); + // The context should be written to slot B, forcing a sync + inner.write(JobId(11), &[&write], false, IOV_MAX_TEST)?; + assert_eq!(inner.active_context[0], ContextSlot::B); + assert_eq!(inner.context_slot_dirty[0], 0b10); Ok(()) } @@ -2054,7 +2066,7 @@ mod test { assert_eq!(inner.extra_syscall_count, 0); assert_eq!(inner.extra_syscall_denominator, 5); inner.flush(10, 10, JobId(10).into())?; - assert_eq!(inner.sync_index, 1); + assert!(inner.context_slot_dirty.iter().all(|v| *v == 0)); // This should not have changed active context slots! for i in 0..10 { @@ -2105,9 +2117,7 @@ mod test { // Do a flush! Because we had a bunch of extra syscalls, this should // trigger defragmentation; after the flush, every context slot should // be in array A. - assert_eq!(inner.sync_index, 1); inner.flush(11, 11, JobId(11).into())?; - assert_eq!(inner.sync_index, 2); for i in 0..10 { assert_eq!( @@ -2160,7 +2170,6 @@ mod test { assert_eq!(inner.extra_syscall_count, 0); assert_eq!(inner.extra_syscall_denominator, 3); inner.flush(10, 10, JobId(10).into())?; - assert_eq!(inner.sync_index, 1); // This should not have changed active context slots! for i in 0..10 { @@ -2214,9 +2223,7 @@ mod test { // Do a flush! Because we had a bunch of extra syscalls, this should // trigger defragmentation; after the flush, every context slot should // be in array B (which minimizes copies) - assert_eq!(inner.sync_index, 1); inner.flush(11, 11, JobId(11).into())?; - assert_eq!(inner.sync_index, 2); for i in 0..10 { assert_eq!( @@ -2271,7 +2278,6 @@ mod test { assert_eq!(inner.extra_syscall_count, 0); assert_eq!(inner.extra_syscall_denominator, 3); inner.flush(10, 10, JobId(10).into())?; - assert_eq!(inner.sync_index, 1); // This should not have changed active context slots! for i in 0..10 { @@ -2326,9 +2332,7 @@ mod test { // Do a flush! Because we had a bunch of extra syscalls, this should // trigger defragmentation; after the flush, every context slot should // be in array A (which minimizes copies) - assert_eq!(inner.sync_index, 1); inner.flush(11, 11, JobId(11).into())?; - assert_eq!(inner.sync_index, 2); for i in 0..10 { assert_eq!( @@ -2383,7 +2387,6 @@ mod test { assert_eq!(inner.extra_syscall_count, 0); assert_eq!(inner.extra_syscall_denominator, 2); inner.flush(10, 10, JobId(10).into())?; - assert_eq!(inner.sync_index, 1); // This should not have changed active context slots! for i in 0..10 { @@ -2437,9 +2440,7 @@ mod test { // This write didn't add enough extra syscalls to trigger // defragmentation. - assert_eq!(inner.sync_index, 1); inner.flush(11, 11, JobId(11).into())?; - assert_eq!(inner.sync_index, 2); // These should be the same! for i in 0..10 { @@ -2511,14 +2512,12 @@ mod test { let write_refs: Vec<_> = writes.iter().collect(); - assert_eq!(inner.sync_index, 0); - assert_eq!(inner.context_slot_synched_at[0], [0, 0]); + assert_eq!(inner.context_slot_dirty[0], 0b00); inner.write(JobId(30), &write_refs, false, IOV_MAX_TEST)?; // The write should be split into four separate calls to // `write_without_overlaps`, triggering one bonus fsync. - assert_eq!(inner.sync_index, 1); - assert_eq!(inner.context_slot_synched_at[0], [2, 2]); + assert_eq!(inner.context_slot_dirty[0], 0b11); // Block 0 should be 0x03 repeated. let mut resp = Vec::new(); From 8fe775b07a9881c8b9fe21418800424a7b173668 Mon Sep 17 00:00:00 2001 From: Matt Keeter Date: Tue, 7 Nov 2023 21:00:10 +0000 Subject: [PATCH 62/62] Remove more premature optimization --- downstairs/src/extent_inner_raw.rs | 100 ++++++----------------------- 1 file changed, 20 insertions(+), 80 deletions(-) diff --git a/downstairs/src/extent_inner_raw.rs b/downstairs/src/extent_inner_raw.rs index c024bd77f..7f00089ad 100644 --- a/downstairs/src/extent_inner_raw.rs +++ b/downstairs/src/extent_inner_raw.rs @@ -117,12 +117,6 @@ pub struct RawInner { /// Denominator corresponding to `extra_syscall_count` extra_syscall_denominator: u64, - - /// Scratch data for doing a quick block uniqueness check - block_unique_scratch: Vec, - - /// Scratch index for doing a quick block uniqueness check - block_unique_index: u8, } #[derive(Copy, Clone, Debug, Eq, PartialEq)] @@ -162,38 +156,31 @@ impl ExtentInner for RawInner { iov_max: usize, ) -> Result<(), CrucibleError> { // If the same block is written multiple times in a single write, then - // (1) that's weird, and (2) we need to handle it specially. - if self.are_blocks_unique(writes.iter().map(|v| v.offset.value)) { - self.write_without_overlaps( - job_id, - writes, - only_write_unwritten, - iov_max, - ) - } else { - // Special handle for writes which touch the same block multiple - // times! We split the `writes` slice into sub-slices of unique - // writes. - let mut seen = HashSet::new(); - let mut start = 0; - for i in 0..=writes.len() { - // If this value is a duplicate or we have reached the end of - // the list, then write everything up to this point and adjust - // our starting point and `seen` array - if i == writes.len() || !seen.insert(writes[i].offset.value) { - self.write_without_overlaps( - job_id, - &writes[start..i], - only_write_unwritten, - iov_max, - )?; + // (1) that's weird, and (2) we need to handle it specially. To handle + // such cases, we split the `writes` slice into sub-slices of unique + // writes. + let mut seen = HashSet::new(); + let mut start = 0; + for i in 0..=writes.len() { + // If this value is a duplicate or we have reached the end of + // the list, then write everything up to this point and adjust + // our starting point and `seen` array + if i == writes.len() || !seen.insert(writes[i].offset.value) { + self.write_without_overlaps( + job_id, + &writes[start..i], + only_write_unwritten, + iov_max, + )?; + // Keep going, resetting the hashmap and start position + if i != writes.len() { seen.clear(); - seen.extend(writes.get(i).map(|w| w.offset.value)); + seen.insert(writes[i].offset.value); start = i; } } - Ok(()) } + Ok(()) } fn read( @@ -474,8 +461,6 @@ impl RawInner { context_slot_dirty: vec![0; def.extent_size().value as usize], extra_syscall_count: 0, extra_syscall_denominator: 0, - block_unique_scratch: vec![0; def.extent_size().value as usize], - block_unique_index: 0, }; // Setting the flush number also writes the extent version, since // they're serialized together in the same block. @@ -633,8 +618,6 @@ impl RawInner { context_slot_dirty: vec![0; def.extent_size().value as usize], extra_syscall_count: 0, extra_syscall_denominator: 0, - block_unique_scratch: vec![0; def.extent_size().value as usize], - block_unique_index: 0, }) } @@ -1197,23 +1180,6 @@ impl RawInner { Ok(()) } - - /// Cheap test that a set of blocks are unique - /// - /// Under rare circumstances, the test may return a false negative – saying - /// that blocks are not unique when they in fact are. That's fine for our - /// use case. - fn are_blocks_unique>(&mut self, iter: I) -> bool { - self.block_unique_index = self.block_unique_index.wrapping_add(1); - for i in iter { - if self.block_unique_scratch[i as usize] == self.block_unique_index - { - return false; - } - self.block_unique_scratch[i as usize] = self.block_unique_index; - } - true - } } /// Data structure that implements the on-disk layout of a raw extent file @@ -2548,30 +2514,4 @@ mod test { Ok(()) } - - #[test] - fn test_are_blocks_unique() -> Result<()> { - let dir = tempdir()?; - let mut inner = - RawInner::create(dir.as_ref(), &new_region_definition(), 0) - .unwrap(); - assert!(inner.are_blocks_unique([0, 1, 2, 3].into_iter())); - assert!(!inner.are_blocks_unique([0, 1, 2, 0].into_iter())); - assert!(!inner.are_blocks_unique([0, 1, 1, 4].into_iter())); - assert!(inner.are_blocks_unique([4, 3, 2, 1].into_iter())); - - // Test the false-negative behavior. It's not load-bearing, but this is - // a good check for our understanding of the data structure. - let mut inner = - RawInner::create(dir.as_ref(), &new_region_definition(), 0) - .unwrap(); - for _ in 0..u8::MAX { - assert!(inner.are_blocks_unique([1].into_iter())); - } - assert!(!inner.are_blocks_unique([0].into_iter())); - - // After a single false negative, we should see correct values - assert!(inner.are_blocks_unique([0].into_iter())); - Ok(()) - } }