fxfs/object_store/journal/
super_block.rs

1// Copyright 2021 The Fuchsia Authors. All rights reserved.
2// Use of this source code is governed by a BSD-style license that can be
3// found in the LICENSE file.
4
5//! We currently store two of these super-blocks (A/B) starting at offset 0 and 512kB.
6//!
7//! Immediately following the serialized `SuperBlockHeader` structure below is a stream of
8//! serialized operations that are replayed into the root parent `ObjectStore`. Note that the root
9//! parent object store exists entirely in RAM until serialized back into the super-block.
10//!
11//! Super-blocks are updated alternately with a monotonically increasing generation number.
12//! At mount time, the super-block used is the valid `SuperBlock` with the highest generation
13//! number.
14//!
15//! Note the asymmetry here regarding load/save:
16//!   * We load a superblock from a Device/SuperBlockInstance and return a
17//!     (SuperBlockHeader, ObjectStore) pair. The ObjectStore is populated directly from device.
18//!   * We save a superblock from a (SuperBlockHeader, Vec<ObjectItem>) pair to a WriteObjectHandle.
19//!
20//! This asymmetry is required for consistency.
21//! The Vec<ObjectItem> is produced by scanning the root_parent_store. This is the responsibility
22//! of the journal code, which must hold a lock to avoid concurrent updates. However, this lock
23//! must NOT be held when saving the superblock as additional extents may need to be allocated as
24//! part of the save process.
25use crate::errors::FxfsError;
26use crate::filesystem::{ApplyContext, ApplyMode, FxFilesystem, JournalingObject};
27use crate::log::*;
28use crate::lsm_tree::types::LayerIterator;
29use crate::lsm_tree::{LSMTree, LayerSet, Query};
30use crate::metrics;
31use crate::object_handle::ObjectHandle as _;
32use crate::object_store::allocator::Reservation;
33use crate::object_store::journal::bootstrap_handle::BootstrapObjectHandle;
34use crate::object_store::journal::reader::{JournalReader, ReadResult};
35use crate::object_store::journal::writer::JournalWriter;
36use crate::object_store::journal::{JournalCheckpoint, JournalCheckpointV32, BLOCK_SIZE};
37use crate::object_store::object_record::{ObjectItem, ObjectItemV40, ObjectItemV41, ObjectItemV43};
38use crate::object_store::transaction::{AssocObj, Options};
39use crate::object_store::tree::MajorCompactable;
40use crate::object_store::{
41    DataObjectHandle, HandleOptions, HandleOwner, Mutation, ObjectKey, ObjectStore, ObjectValue,
42};
43use crate::range::RangeExt;
44use crate::serialized_types::{
45    migrate_to_version, Migrate, Version, Versioned, VersionedLatest, EARLIEST_SUPPORTED_VERSION,
46    FIRST_EXTENT_IN_SUPERBLOCK_VERSION, SMALL_SUPERBLOCK_VERSION,
47};
48use anyhow::{bail, ensure, Context, Error};
49use fprint::TypeFingerprint;
50use fuchsia_inspect::{Property as _, UintProperty};
51use fuchsia_sync::Mutex;
52use futures::FutureExt;
53use rustc_hash::FxHashMap as HashMap;
54use serde::{Deserialize, Serialize};
55use std::collections::VecDeque;
56use std::fmt;
57use std::io::{Read, Write};
58use std::ops::Range;
59use std::sync::Arc;
60use std::time::SystemTime;
61use storage_device::Device;
62use uuid::Uuid;
63
64// These only exist in the root store.
65const SUPER_BLOCK_A_OBJECT_ID: u64 = 1;
66const SUPER_BLOCK_B_OBJECT_ID: u64 = 2;
67
68/// The superblock is extended in units of `SUPER_BLOCK_CHUNK_SIZE` as required.
69pub const SUPER_BLOCK_CHUNK_SIZE: u64 = 65536;
70
71/// Each superblock is one block but may contain records that extend its own length.
72const MIN_SUPER_BLOCK_SIZE: u64 = 4096;
73/// The first 2 * 512 KiB on the disk used to be reserved for two A/B super-blocks.
74const LEGACY_MIN_SUPER_BLOCK_SIZE: u64 = 524_288;
75
76/// All superblocks start with the magic bytes "FxfsSupr".
77const SUPER_BLOCK_MAGIC: &[u8; 8] = b"FxfsSupr";
78
79/// An enum representing one of our super-block instances.
80///
81/// This provides hard-coded constants related to the location and properties of the super-blocks
82/// that are required to bootstrap the filesystem.
83#[derive(Copy, Clone, Debug)]
84pub enum SuperBlockInstance {
85    A,
86    B,
87}
88
89impl SuperBlockInstance {
90    /// Returns the next [SuperBlockInstance] for use in round-robining writes across super-blocks.
91    pub fn next(&self) -> SuperBlockInstance {
92        match self {
93            SuperBlockInstance::A => SuperBlockInstance::B,
94            SuperBlockInstance::B => SuperBlockInstance::A,
95        }
96    }
97
98    pub fn object_id(&self) -> u64 {
99        match self {
100            SuperBlockInstance::A => SUPER_BLOCK_A_OBJECT_ID,
101            SuperBlockInstance::B => SUPER_BLOCK_B_OBJECT_ID,
102        }
103    }
104
105    /// Returns the byte range where the first extent of the [SuperBlockInstance] is stored.
106    /// (Note that a [SuperBlockInstance] may still have multiple extents.)
107    pub fn first_extent(&self) -> Range<u64> {
108        match self {
109            SuperBlockInstance::A => 0..MIN_SUPER_BLOCK_SIZE,
110            SuperBlockInstance::B => 524288..524288 + MIN_SUPER_BLOCK_SIZE,
111        }
112    }
113
114    /// We used to allocate 512kB to superblocks but this was almost always more than needed.
115    pub fn legacy_first_extent(&self) -> Range<u64> {
116        match self {
117            SuperBlockInstance::A => 0..LEGACY_MIN_SUPER_BLOCK_SIZE,
118            SuperBlockInstance::B => LEGACY_MIN_SUPER_BLOCK_SIZE..2 * LEGACY_MIN_SUPER_BLOCK_SIZE,
119        }
120    }
121}
122
123pub type SuperBlockHeader = SuperBlockHeaderV32;
124
125#[derive(
126    Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize, TypeFingerprint, Versioned,
127)]
128pub struct SuperBlockHeaderV32 {
129    /// The globally unique identifier for the filesystem.
130    guid: UuidWrapperV32,
131
132    /// There are two super-blocks which are used in an A/B configuration. The super-block with the
133    /// greatest generation number is what is used when mounting an Fxfs image; the other is
134    /// discarded.
135    pub generation: u64,
136
137    /// The root parent store is an in-memory only store and serves as the backing store for the
138    /// root store and the journal.  The records for this store are serialized into the super-block
139    /// and mutations are also recorded in the journal.
140    pub root_parent_store_object_id: u64,
141
142    /// The root parent needs a graveyard and there's nowhere else to store it other than in the
143    /// super-block.
144    pub root_parent_graveyard_directory_object_id: u64,
145
146    /// The root object store contains all other metadata objects (including the allocator, the
147    /// journal and the super-blocks) and is the parent for all other object stores.
148    pub root_store_object_id: u64,
149
150    /// This is in the root object store.
151    pub allocator_object_id: u64,
152
153    /// This is in the root parent object store.
154    pub journal_object_id: u64,
155
156    /// Start checkpoint for the journal file.
157    pub journal_checkpoint: JournalCheckpointV32,
158
159    /// Offset of the journal file when the super-block was written.  If no entry is present in
160    /// journal_file_offsets for a particular object, then an object might have dependencies on the
161    /// journal from super_block_journal_file_offset onwards, but not earlier.
162    pub super_block_journal_file_offset: u64,
163
164    /// object id -> journal file offset. Indicates where each object has been flushed to.
165    pub journal_file_offsets: HashMap<u64, u64>,
166
167    /// Records the amount of borrowed metadata space as applicable at
168    /// `super_block_journal_file_offset`.
169    pub borrowed_metadata_space: u64,
170
171    /// The earliest version of Fxfs used to create any still-existing struct in the filesystem.
172    ///
173    /// Note: structs in the filesystem may had been made with various different versions of Fxfs.
174    pub earliest_version: Version,
175}
176
177type UuidWrapper = UuidWrapperV32;
178#[derive(Clone, Default, Eq, PartialEq)]
179struct UuidWrapperV32(Uuid);
180
181impl UuidWrapper {
182    fn new() -> Self {
183        Self(Uuid::new_v4())
184    }
185    #[cfg(test)]
186    fn nil() -> Self {
187        Self(Uuid::nil())
188    }
189}
190
191impl fmt::Debug for UuidWrapper {
192    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
193        // The UUID uniquely identifies the filesystem, so we should redact it so that we don't leak
194        // it in logs.
195        f.write_str("<redacted>")
196    }
197}
198
199impl TypeFingerprint for UuidWrapper {
200    fn fingerprint() -> String {
201        "<[u8;16]>".to_owned()
202    }
203}
204
205// Uuid serializes like a slice, but SuperBlockHeader used to contain [u8; 16] and we want to remain
206// compatible.
207impl Serialize for UuidWrapper {
208    fn serialize<S: serde::Serializer>(&self, serializer: S) -> Result<S::Ok, S::Error> {
209        self.0.as_bytes().serialize(serializer)
210    }
211}
212
213impl<'de> Deserialize<'de> for UuidWrapper {
214    fn deserialize<D: serde::Deserializer<'de>>(deserializer: D) -> Result<Self, D::Error> {
215        <[u8; 16]>::deserialize(deserializer).map(|bytes| UuidWrapperV32(Uuid::from_bytes(bytes)))
216    }
217}
218
219pub type SuperBlockRecord = SuperBlockRecordV43;
220
221#[allow(clippy::large_enum_variant)]
222#[derive(Debug, Serialize, Deserialize, TypeFingerprint, Versioned)]
223pub enum SuperBlockRecordV43 {
224    // When reading the super-block we know the initial extent, but not subsequent extents, so these
225    // records need to exist to allow us to completely read the super-block.
226    Extent(Range<u64>),
227
228    // Following the super-block header are ObjectItem records that are to be replayed into the root
229    // parent object store.
230    ObjectItem(ObjectItemV43),
231
232    // Marks the end of the full super-block.
233    End,
234}
235#[derive(Migrate, Serialize, Deserialize, TypeFingerprint, Versioned)]
236#[migrate_to_version(SuperBlockRecordV43)]
237pub enum SuperBlockRecordV41 {
238    Extent(Range<u64>),
239    ObjectItem(ObjectItemV41),
240    End,
241}
242
243#[derive(Migrate, Serialize, Deserialize, TypeFingerprint, Versioned)]
244#[migrate_to_version(SuperBlockRecordV41)]
245pub enum SuperBlockRecordV40 {
246    Extent(Range<u64>),
247    ObjectItem(ObjectItemV40),
248    End,
249}
250
251struct SuperBlockMetrics {
252    /// Time we wrote the most recent superblock in milliseconds since [`std::time::UNIX_EPOCH`].
253    /// Uses [`std::time::SystemTime`] as the clock source.
254    last_super_block_update_time_ms: UintProperty,
255
256    /// Offset of the most recent superblock we wrote in the journal.
257    last_super_block_offset: UintProperty,
258}
259
260impl Default for SuperBlockMetrics {
261    fn default() -> Self {
262        SuperBlockMetrics {
263            last_super_block_update_time_ms: metrics::detail()
264                .create_uint("last_super_block_update_time_ms", 0),
265            last_super_block_offset: metrics::detail().create_uint("last_super_block_offset", 0),
266        }
267    }
268}
269
270/// Reads an individual (A/B) super-block instance and root_parent_store from device.
271/// Users should use SuperBlockManager::load() instead.
272async fn read(
273    device: Arc<dyn Device>,
274    block_size: u64,
275    instance: SuperBlockInstance,
276) -> Result<(SuperBlockHeader, SuperBlockInstance, ObjectStore), Error> {
277    let (super_block_header, mut reader) = SuperBlockHeader::read_header(device.clone(), instance)
278        .await
279        .context("failed to read superblock")?;
280    let root_parent = ObjectStore::new_root_parent(
281        device,
282        block_size,
283        super_block_header.root_parent_store_object_id,
284    );
285    root_parent.set_graveyard_directory_object_id(
286        super_block_header.root_parent_graveyard_directory_object_id,
287    );
288
289    loop {
290        // TODO: Flatten a layer and move reader here?
291        let (mutation, sequence) = match reader.next_item().await? {
292            // RecordReader should filter out extent records.
293            SuperBlockRecord::Extent(_) => bail!("Unexpected extent record"),
294            SuperBlockRecord::ObjectItem(item) => {
295                (Mutation::insert_object(item.key, item.value), item.sequence)
296            }
297            SuperBlockRecord::End => break,
298        };
299        root_parent.apply_mutation(
300            mutation,
301            &ApplyContext {
302                mode: ApplyMode::Replay,
303                checkpoint: JournalCheckpoint { file_offset: sequence, ..Default::default() },
304            },
305            AssocObj::None,
306        )?;
307    }
308    Ok((super_block_header, instance, root_parent))
309}
310
311/// Write a super-block to the given file handle.
312/// Requires that the filesystem is fully loaded and writable as this may require allocation.
313async fn write<S: HandleOwner>(
314    super_block_header: &SuperBlockHeader,
315    items: LayerSet<ObjectKey, ObjectValue>,
316    handle: DataObjectHandle<S>,
317) -> Result<(), Error> {
318    let object_manager = handle.store().filesystem().object_manager().clone();
319    // TODO(https://fxbug.dev/42177407): Don't use the same code here for Journal and SuperBlock. They
320    // aren't the same things and it is already getting convoluted. e.g of diff stream content:
321    //   Superblock:  (Magic, Ver, Header(Ver), Extent(Ver)*, SuperBlockRecord(Ver)*, ...)
322    //   Journal:     (Ver, JournalRecord(Ver)*, RESET, Ver2, JournalRecord(Ver2)*, ...)
323    // We should abstract away the checksum code and implement these separately.
324
325    let mut writer =
326        SuperBlockWriter::new(handle, super_block_header, object_manager.metadata_reservation())
327            .await?;
328    let mut merger = items.merger();
329    let mut iter = LSMTree::major_iter(merger.query(Query::FullScan).await?).await?;
330    while let Some(item) = iter.get() {
331        writer.write_root_parent_item(item.cloned()).await?;
332        iter.advance().await?;
333    }
334    writer.finalize().await
335}
336
337// Compacts and returns the *old* snapshot of the root_parent store.
338// Must be performed whilst holding a writer lock.
339pub fn compact_root_parent(
340    root_parent_store: &ObjectStore,
341) -> Result<LayerSet<ObjectKey, ObjectValue>, Error> {
342    // The root parent always uses in-memory layers which shouldn't be async, so we can use
343    // `now_or_never`.
344    let tree = root_parent_store.tree();
345    let layer_set = tree.layer_set();
346    {
347        let mut merger = layer_set.merger();
348        let mut iter = LSMTree::major_iter(merger.query(Query::FullScan).now_or_never().unwrap()?)
349            .now_or_never()
350            .unwrap()?;
351        let new_layer = LSMTree::new_mutable_layer();
352        while let Some(item_ref) = iter.get() {
353            new_layer.insert(item_ref.cloned())?;
354            iter.advance().now_or_never().unwrap()?;
355        }
356        tree.set_mutable_layer(new_layer);
357    }
358    Ok(layer_set)
359}
360
361/// This encapsulates the A/B alternating super-block logic.
362/// All super-block load/save operations should be via the methods on this type.
363pub struct SuperBlockManager {
364    next_instance: Arc<Mutex<SuperBlockInstance>>,
365    metrics: SuperBlockMetrics,
366}
367
368impl SuperBlockManager {
369    pub fn new() -> Self {
370        Self {
371            next_instance: Arc::new(Mutex::new(SuperBlockInstance::A)),
372            metrics: Default::default(),
373        }
374    }
375
376    /// Loads both A/B super-blocks and root_parent ObjectStores and and returns the newest valid
377    /// pair. Also ensures the next superblock updated via |save| will be the other instance.
378    pub async fn load(
379        &self,
380        device: Arc<dyn Device>,
381        block_size: u64,
382    ) -> Result<(SuperBlockHeader, ObjectStore), Error> {
383        // Superblocks consume a minimum of one block. We currently hard code the length of
384        // this first extent. It should work with larger block sizes, but has not been tested.
385        // TODO(https://fxbug.dev/42063349): Consider relaxing this.
386        debug_assert!(MIN_SUPER_BLOCK_SIZE == block_size);
387
388        let (super_block, current_super_block, root_parent) = match futures::join!(
389            read(device.clone(), block_size, SuperBlockInstance::A),
390            read(device.clone(), block_size, SuperBlockInstance::B)
391        ) {
392            (Err(e1), Err(e2)) => {
393                bail!("Failed to load both superblocks due to {:?}\nand\n{:?}", e1, e2)
394            }
395            (Ok(result), Err(_)) => result,
396            (Err(_), Ok(result)) => result,
397            (Ok(result1), Ok(result2)) => {
398                // Break the tie by taking the super-block with the greatest generation.
399                if result2.0.generation > result1.0.generation {
400                    result2
401                } else {
402                    result1
403                }
404            }
405        };
406        info!(super_block:?, current_super_block:?; "loaded super-block");
407        *self.next_instance.lock() = current_super_block.next();
408        Ok((super_block, root_parent))
409    }
410
411    /// Writes the provided superblock and root_parent ObjectStore to the device.
412    /// Requires that the filesystem is fully loaded and writable as this may require allocation.
413    pub async fn save(
414        &self,
415        super_block_header: SuperBlockHeader,
416        filesystem: Arc<FxFilesystem>,
417        root_parent: LayerSet<ObjectKey, ObjectValue>,
418    ) -> Result<(), Error> {
419        let root_store = filesystem.root_store();
420        let object_id = {
421            let mut next_instance = self.next_instance.lock();
422            let object_id = next_instance.object_id();
423            *next_instance = next_instance.next();
424            object_id
425        };
426        let handle = ObjectStore::open_object(
427            &root_store,
428            object_id,
429            HandleOptions { skip_journal_checks: true, ..Default::default() },
430            None,
431        )
432        .await
433        .context("Failed to open superblock object")?;
434        write(&super_block_header, root_parent, handle).await?;
435        self.metrics
436            .last_super_block_offset
437            .set(super_block_header.super_block_journal_file_offset);
438        self.metrics.last_super_block_update_time_ms.set(
439            SystemTime::now()
440                .duration_since(SystemTime::UNIX_EPOCH)
441                .unwrap()
442                .as_millis()
443                .try_into()
444                .unwrap_or(0u64),
445        );
446        Ok(())
447    }
448}
449
450impl SuperBlockHeader {
451    /// Creates a new instance with random GUID.
452    pub fn new(
453        root_parent_store_object_id: u64,
454        root_parent_graveyard_directory_object_id: u64,
455        root_store_object_id: u64,
456        allocator_object_id: u64,
457        journal_object_id: u64,
458        journal_checkpoint: JournalCheckpoint,
459        earliest_version: Version,
460    ) -> Self {
461        SuperBlockHeader {
462            guid: UuidWrapper::new(),
463            generation: 1u64,
464            root_parent_store_object_id,
465            root_parent_graveyard_directory_object_id,
466            root_store_object_id,
467            allocator_object_id,
468            journal_object_id,
469            journal_checkpoint,
470            earliest_version,
471            ..Default::default()
472        }
473    }
474
475    /// Read the super-block header, and return it and a reader that produces the records that are
476    /// to be replayed in to the root parent object store.
477    async fn read_header(
478        device: Arc<dyn Device>,
479        target_super_block: SuperBlockInstance,
480    ) -> Result<(SuperBlockHeader, RecordReader), Error> {
481        let handle = BootstrapObjectHandle::new(
482            target_super_block.object_id(),
483            device,
484            target_super_block.first_extent(),
485        );
486        let mut reader = JournalReader::new(handle, &JournalCheckpoint::default());
487        reader.set_eof_ok();
488
489        reader.fill_buf().await?;
490
491        let mut super_block_header;
492        let super_block_version;
493        reader.consume({
494            let mut cursor = std::io::Cursor::new(reader.buffer());
495            // Validate magic bytes.
496            let mut magic_bytes: [u8; 8] = [0; 8];
497            cursor.read_exact(&mut magic_bytes)?;
498            if magic_bytes.as_slice() != SUPER_BLOCK_MAGIC.as_slice() {
499                bail!("Invalid magic: {:?}", magic_bytes);
500            }
501            (super_block_header, super_block_version) =
502                SuperBlockHeader::deserialize_with_version(&mut cursor)?;
503
504            if super_block_version < EARLIEST_SUPPORTED_VERSION {
505                bail!("Unsupported SuperBlock version: {:?}", super_block_version);
506            }
507
508            // NOTE: It is possible that data was written to the journal with an old version
509            // but no compaction ever happened, so the journal version could potentially be older
510            // than the layer file versions.
511            if super_block_header.journal_checkpoint.version < EARLIEST_SUPPORTED_VERSION {
512                bail!(
513                    "Unsupported JournalCheckpoint version: {:?}",
514                    super_block_header.journal_checkpoint.version
515                );
516            }
517
518            if super_block_header.earliest_version < EARLIEST_SUPPORTED_VERSION {
519                bail!(
520                    "Filesystem contains struct with unsupported version: {:?}",
521                    super_block_header.earliest_version
522                );
523            }
524
525            cursor.position() as usize
526        });
527
528        // From version 45 superblocks describe their own extents (a noop here).
529        // At version 44, superblocks assume a 4kb first extent.
530        // Prior to version 44, superblocks assume a 512kb first extent.
531        if super_block_version < SMALL_SUPERBLOCK_VERSION {
532            reader.handle().push_extent(0, target_super_block.legacy_first_extent());
533        } else if super_block_version < FIRST_EXTENT_IN_SUPERBLOCK_VERSION {
534            reader.handle().push_extent(0, target_super_block.first_extent())
535        }
536
537        // If guid is zeroed (e.g. in a newly imaged system), assign one randomly.
538        if super_block_header.guid.0.is_nil() {
539            super_block_header.guid = UuidWrapper::new();
540        }
541        reader.set_version(super_block_version);
542        Ok((super_block_header, RecordReader { reader }))
543    }
544}
545
546struct SuperBlockWriter<'a, S: HandleOwner> {
547    handle: DataObjectHandle<S>,
548    writer: JournalWriter,
549    existing_extents: VecDeque<(u64, Range<u64>)>,
550    size: u64,
551    reservation: &'a Reservation,
552}
553
554impl<'a, S: HandleOwner> SuperBlockWriter<'a, S> {
555    /// Create a new writer, outputs FXFS magic, version and SuperBlockHeader.
556    /// On success, the writer is ready to accept root parent store mutations.
557    pub async fn new(
558        handle: DataObjectHandle<S>,
559        super_block_header: &SuperBlockHeader,
560        reservation: &'a Reservation,
561    ) -> Result<Self, Error> {
562        let existing_extents = handle.device_extents().await?;
563        let mut this = Self {
564            handle,
565            writer: JournalWriter::new(BLOCK_SIZE as usize, 0),
566            existing_extents: existing_extents.into_iter().collect(),
567            size: 0,
568            reservation,
569        };
570        this.writer.write_all(SUPER_BLOCK_MAGIC)?;
571        super_block_header.serialize_with_version(&mut this.writer)?;
572        Ok(this)
573    }
574
575    /// Internal helper function to pull ranges from a list of existing extents and tack
576    /// corresponding extent records onto the journal.
577    fn try_extend_existing(&mut self, target_size: u64) -> Result<(), Error> {
578        while self.size < target_size {
579            if let Some((offset, range)) = self.existing_extents.pop_front() {
580                ensure!(offset == self.size, "superblock file contains a hole.");
581                self.size += range.end - range.start;
582                SuperBlockRecord::Extent(range).serialize_into(&mut self.writer)?;
583            } else {
584                break;
585            }
586        }
587        Ok(())
588    }
589
590    pub async fn write_root_parent_item(&mut self, record: ObjectItem) -> Result<(), Error> {
591        let min_len = self.writer.journal_file_checkpoint().file_offset + SUPER_BLOCK_CHUNK_SIZE;
592        self.try_extend_existing(min_len)?;
593        if min_len > self.size {
594            // Need to allocate some more space.
595            let mut transaction = self
596                .handle
597                .new_transaction_with_options(Options {
598                    skip_journal_checks: true,
599                    borrow_metadata_space: true,
600                    allocator_reservation: Some(self.reservation),
601                    ..Default::default()
602                })
603                .await?;
604            let mut file_range = self.size..self.size + SUPER_BLOCK_CHUNK_SIZE;
605            let allocated = self
606                .handle
607                .preallocate_range(&mut transaction, &mut file_range)
608                .await
609                .context("preallocate superblock")?;
610            if file_range.start < file_range.end {
611                bail!("preallocate_range returned too little space");
612            }
613            transaction.commit().await?;
614            for device_range in allocated {
615                self.size += device_range.end - device_range.start;
616                SuperBlockRecord::Extent(device_range).serialize_into(&mut self.writer)?;
617            }
618        }
619        SuperBlockRecord::ObjectItem(record).serialize_into(&mut self.writer)?;
620        Ok(())
621    }
622
623    pub async fn finalize(mut self) -> Result<(), Error> {
624        SuperBlockRecord::End.serialize_into(&mut self.writer)?;
625        self.writer.pad_to_block()?;
626        let mut buf = self.handle.allocate_buffer(self.writer.flushable_bytes()).await;
627        let offset = self.writer.take_flushable(buf.as_mut());
628        self.handle.overwrite(offset, buf.as_mut(), false).await?;
629        let len =
630            std::cmp::max(MIN_SUPER_BLOCK_SIZE, self.writer.journal_file_checkpoint().file_offset)
631                + SUPER_BLOCK_CHUNK_SIZE;
632        self.handle
633            .truncate_with_options(
634                Options {
635                    skip_journal_checks: true,
636                    borrow_metadata_space: true,
637                    ..Default::default()
638                },
639                len,
640            )
641            .await?;
642        Ok(())
643    }
644}
645
646pub struct RecordReader {
647    reader: JournalReader,
648}
649
650impl RecordReader {
651    pub async fn next_item(&mut self) -> Result<SuperBlockRecord, Error> {
652        loop {
653            match self.reader.deserialize().await? {
654                ReadResult::Reset(_) => bail!("Unexpected reset"),
655                ReadResult::ChecksumMismatch => bail!("Checksum mismatch"),
656                ReadResult::Some(SuperBlockRecord::Extent(extent)) => {
657                    ensure!(extent.is_valid(), FxfsError::Inconsistent);
658                    self.reader.handle().push_extent(0, extent)
659                }
660                ReadResult::Some(x) => return Ok(x),
661            }
662        }
663    }
664}
665
666#[cfg(test)]
667mod tests {
668    use super::{
669        compact_root_parent, write, SuperBlockHeader, SuperBlockInstance, UuidWrapper,
670        MIN_SUPER_BLOCK_SIZE, SUPER_BLOCK_CHUNK_SIZE,
671    };
672    use crate::filesystem::{FxFilesystem, OpenFxFilesystem};
673    use crate::object_handle::ReadObjectHandle;
674    use crate::object_store::journal::JournalCheckpoint;
675    use crate::object_store::transaction::{lock_keys, Options};
676    use crate::object_store::{
677        DataObjectHandle, HandleOptions, ObjectHandle, ObjectKey, ObjectStore,
678    };
679    use crate::serialized_types::LATEST_VERSION;
680    use storage_device::fake_device::FakeDevice;
681    use storage_device::DeviceHolder;
682
683    // We require 512kiB each for A/B super-blocks, 256kiB for the journal (128kiB before flush)
684    // and compactions require double the layer size to complete.
685    const TEST_DEVICE_BLOCK_SIZE: u32 = 512;
686    const TEST_DEVICE_BLOCK_COUNT: u64 = 16384;
687
688    async fn filesystem_and_super_block_handles(
689    ) -> (OpenFxFilesystem, DataObjectHandle<ObjectStore>, DataObjectHandle<ObjectStore>) {
690        let device =
691            DeviceHolder::new(FakeDevice::new(TEST_DEVICE_BLOCK_COUNT, TEST_DEVICE_BLOCK_SIZE));
692        let fs = FxFilesystem::new_empty(device).await.expect("new_empty failed");
693        fs.close().await.expect("Close failed");
694        let device = fs.take_device().await;
695        device.reopen(false);
696        let fs = FxFilesystem::open(device).await.expect("open failed");
697
698        let handle_a = ObjectStore::open_object(
699            &fs.object_manager().root_store(),
700            SuperBlockInstance::A.object_id(),
701            HandleOptions::default(),
702            None,
703        )
704        .await
705        .expect("open superblock failed");
706
707        let handle_b = ObjectStore::open_object(
708            &fs.object_manager().root_store(),
709            SuperBlockInstance::B.object_id(),
710            HandleOptions::default(),
711            None,
712        )
713        .await
714        .expect("open superblock failed");
715        (fs, handle_a, handle_b)
716    }
717
718    #[fuchsia::test]
719    async fn test_read_written_super_block() {
720        let (fs, _handle_a, _handle_b) = filesystem_and_super_block_handles().await;
721        const JOURNAL_OBJECT_ID: u64 = 5;
722
723        // Confirm that the (first) super-block is expected size.
724        // It should be MIN_SUPER_BLOCK_SIZE + SUPER_BLOCK_CHUNK_SIZE.
725        assert_eq!(
726            ObjectStore::open_object(
727                &fs.root_store(),
728                SuperBlockInstance::A.object_id(),
729                HandleOptions::default(),
730                None,
731            )
732            .await
733            .expect("open_object failed")
734            .get_size(),
735            MIN_SUPER_BLOCK_SIZE + SUPER_BLOCK_CHUNK_SIZE
736        );
737
738        // Create a large number of objects in the root parent store so that we test growing
739        // of the super-block file, requiring us to add extents.
740        let mut created_object_ids = vec![];
741        const NUM_ENTRIES: u64 = 16384;
742        for _ in 0..NUM_ENTRIES {
743            let mut transaction = fs
744                .clone()
745                .new_transaction(lock_keys![], Options::default())
746                .await
747                .expect("new_transaction failed");
748            created_object_ids.push(
749                ObjectStore::create_object(
750                    &fs.object_manager().root_parent_store(),
751                    &mut transaction,
752                    HandleOptions::default(),
753                    None,
754                )
755                .await
756                .expect("create_object failed")
757                .object_id(),
758            );
759            transaction.commit().await.expect("commit failed");
760        }
761
762        // Note here that DataObjectHandle caches the size given to it at construction.
763        // If we want to know the true size after a super-block has been written, we need
764        // a new handle.
765        assert!(
766            ObjectStore::open_object(
767                &fs.root_store(),
768                SuperBlockInstance::A.object_id(),
769                HandleOptions::default(),
770                None,
771            )
772            .await
773            .expect("open_object failed")
774            .get_size()
775                > MIN_SUPER_BLOCK_SIZE + SUPER_BLOCK_CHUNK_SIZE
776        );
777
778        let written_super_block_a =
779            SuperBlockHeader::read_header(fs.device(), SuperBlockInstance::A)
780                .await
781                .expect("read failed");
782        let written_super_block_b =
783            SuperBlockHeader::read_header(fs.device(), SuperBlockInstance::B)
784                .await
785                .expect("read failed");
786
787        // Check that a non-zero GUID has been assigned.
788        assert!(!written_super_block_a.0.guid.0.is_nil());
789
790        // Depending on specific offsets is fragile so we just validate the fields we believe
791        // to be stable.
792        assert_eq!(written_super_block_a.0.guid, written_super_block_b.0.guid);
793        assert_eq!(written_super_block_a.0.guid, written_super_block_b.0.guid);
794        assert!(written_super_block_a.0.generation != written_super_block_b.0.generation);
795        assert_eq!(
796            written_super_block_a.0.root_parent_store_object_id,
797            written_super_block_b.0.root_parent_store_object_id
798        );
799        assert_eq!(
800            written_super_block_a.0.root_parent_graveyard_directory_object_id,
801            written_super_block_b.0.root_parent_graveyard_directory_object_id
802        );
803        assert_eq!(written_super_block_a.0.root_store_object_id, fs.root_store().store_object_id());
804        assert_eq!(
805            written_super_block_a.0.root_store_object_id,
806            written_super_block_b.0.root_store_object_id
807        );
808        assert_eq!(written_super_block_a.0.allocator_object_id, fs.allocator().object_id());
809        assert_eq!(
810            written_super_block_a.0.allocator_object_id,
811            written_super_block_b.0.allocator_object_id
812        );
813        assert_eq!(written_super_block_a.0.journal_object_id, JOURNAL_OBJECT_ID);
814        assert_eq!(
815            written_super_block_a.0.journal_object_id,
816            written_super_block_b.0.journal_object_id
817        );
818        assert!(
819            written_super_block_a.0.journal_checkpoint.file_offset
820                != written_super_block_b.0.journal_checkpoint.file_offset
821        );
822        assert!(
823            written_super_block_a.0.super_block_journal_file_offset
824                != written_super_block_b.0.super_block_journal_file_offset
825        );
826        // Nb: We skip journal_file_offsets and borrowed metadata space checks.
827        assert_eq!(written_super_block_a.0.earliest_version, LATEST_VERSION);
828        assert_eq!(
829            written_super_block_a.0.earliest_version,
830            written_super_block_b.0.earliest_version
831        );
832
833        // Nb: Skip comparison of root_parent store contents because we have no way of anticipating
834        // the extent offsets and it is reasonable that a/b differ.
835
836        // Delete all the objects we just made.
837        for object_id in created_object_ids {
838            let mut transaction = fs
839                .clone()
840                .new_transaction(lock_keys![], Options::default())
841                .await
842                .expect("new_transaction failed");
843            fs.object_manager()
844                .root_parent_store()
845                .adjust_refs(&mut transaction, object_id, -1)
846                .await
847                .expect("adjust_refs failed");
848            transaction.commit().await.expect("commit failed");
849            fs.object_manager()
850                .root_parent_store()
851                .tombstone_object(object_id, Options::default())
852                .await
853                .expect("tombstone failed");
854        }
855        // Write some stuff to the root store to ensure we rotate the journal and produce new
856        // super blocks.
857        for _ in 0..NUM_ENTRIES {
858            let mut transaction = fs
859                .clone()
860                .new_transaction(lock_keys![], Options::default())
861                .await
862                .expect("new_transaction failed");
863            ObjectStore::create_object(
864                &fs.object_manager().root_store(),
865                &mut transaction,
866                HandleOptions::default(),
867                None,
868            )
869            .await
870            .expect("create_object failed");
871            transaction.commit().await.expect("commit failed");
872        }
873
874        assert_eq!(
875            ObjectStore::open_object(
876                &fs.root_store(),
877                SuperBlockInstance::A.object_id(),
878                HandleOptions::default(),
879                None,
880            )
881            .await
882            .expect("open_object failed")
883            .get_size(),
884            MIN_SUPER_BLOCK_SIZE + SUPER_BLOCK_CHUNK_SIZE
885        );
886    }
887
888    #[fuchsia::test]
889    async fn test_guid_assign_on_read() {
890        let (fs, handle_a, _handle_b) = filesystem_and_super_block_handles().await;
891        const JOURNAL_OBJECT_ID: u64 = 5;
892        let mut super_block_header_a = SuperBlockHeader::new(
893            fs.object_manager().root_parent_store().store_object_id(),
894            /* root_parent_graveyard_directory_object_id: */ 1000,
895            fs.root_store().store_object_id(),
896            fs.allocator().object_id(),
897            JOURNAL_OBJECT_ID,
898            JournalCheckpoint { file_offset: 1234, checksum: 5678, version: LATEST_VERSION },
899            /* earliest_version: */ LATEST_VERSION,
900        );
901        // Ensure the superblock has no set GUID.
902        super_block_header_a.guid = UuidWrapper::nil();
903        write(
904            &super_block_header_a,
905            compact_root_parent(fs.object_manager().root_parent_store().as_ref())
906                .expect("scan failed"),
907            handle_a,
908        )
909        .await
910        .expect("write failed");
911        let super_block_header = SuperBlockHeader::read_header(fs.device(), SuperBlockInstance::A)
912            .await
913            .expect("read failed");
914        // Ensure a GUID has been assigned.
915        assert!(!super_block_header.0.guid.0.is_nil());
916    }
917
918    #[fuchsia::test]
919    async fn test_init_wipes_superblocks() {
920        let device = DeviceHolder::new(FakeDevice::new(8192, TEST_DEVICE_BLOCK_SIZE));
921
922        let fs = FxFilesystem::new_empty(device).await.expect("new_empty failed");
923        let root_store = fs.root_store();
924        // Generate enough work to induce a journal flush and thus a new superblock being written.
925        for _ in 0..6000 {
926            let mut transaction = fs
927                .clone()
928                .new_transaction(lock_keys![], Options::default())
929                .await
930                .expect("new_transaction failed");
931            ObjectStore::create_object(
932                &root_store,
933                &mut transaction,
934                HandleOptions::default(),
935                None,
936            )
937            .await
938            .expect("create_object failed");
939            transaction.commit().await.expect("commit failed");
940        }
941        fs.close().await.expect("Close failed");
942        let device = fs.take_device().await;
943        device.reopen(false);
944
945        SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::A)
946            .await
947            .expect("read failed");
948        let header = SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::B)
949            .await
950            .expect("read failed");
951
952        let old_guid = header.0.guid;
953
954        // Re-initialize the filesystem.  The A and B blocks should be for the new FS.
955        let fs = FxFilesystem::new_empty(device).await.expect("new_empty failed");
956        fs.close().await.expect("Close failed");
957        let device = fs.take_device().await;
958        device.reopen(false);
959
960        let a = SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::A)
961            .await
962            .expect("read failed");
963        let b = SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::B)
964            .await
965            .expect("read failed");
966
967        assert_eq!(a.0.guid, b.0.guid);
968        assert_ne!(old_guid, a.0.guid);
969    }
970
971    #[fuchsia::test]
972    async fn test_alternating_super_blocks() {
973        let device = DeviceHolder::new(FakeDevice::new(8192, TEST_DEVICE_BLOCK_SIZE));
974
975        let fs = FxFilesystem::new_empty(device).await.expect("new_empty failed");
976        fs.close().await.expect("Close failed");
977        let device = fs.take_device().await;
978        device.reopen(false);
979
980        let (super_block_header_a, _) =
981            SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::A)
982                .await
983                .expect("read failed");
984
985        // The second super-block won't be valid at this time so there's no point reading it.
986
987        let fs = FxFilesystem::open(device).await.expect("open failed");
988        let root_store = fs.root_store();
989        // Generate enough work to induce a journal flush.
990        for _ in 0..6000 {
991            let mut transaction = fs
992                .clone()
993                .new_transaction(lock_keys![], Options::default())
994                .await
995                .expect("new_transaction failed");
996            ObjectStore::create_object(
997                &root_store,
998                &mut transaction,
999                HandleOptions::default(),
1000                None,
1001            )
1002            .await
1003            .expect("create_object failed");
1004            transaction.commit().await.expect("commit failed");
1005        }
1006        fs.close().await.expect("Close failed");
1007        let device = fs.take_device().await;
1008        device.reopen(false);
1009
1010        let (super_block_header_a_after, _) =
1011            SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::A)
1012                .await
1013                .expect("read failed");
1014        let (super_block_header_b_after, _) =
1015            SuperBlockHeader::read_header(device.clone(), SuperBlockInstance::B)
1016                .await
1017                .expect("read failed");
1018
1019        // It's possible that multiple super-blocks were written, so cater for that.
1020
1021        // The sequence numbers should be one apart.
1022        assert_eq!(
1023            (super_block_header_b_after.generation as i64
1024                - super_block_header_a_after.generation as i64)
1025                .abs(),
1026            1
1027        );
1028
1029        // At least one super-block should have been written.
1030        assert!(
1031            std::cmp::max(
1032                super_block_header_a_after.generation,
1033                super_block_header_b_after.generation
1034            ) > super_block_header_a.generation
1035        );
1036
1037        // They should have the same oddness.
1038        assert_eq!(super_block_header_a_after.generation & 1, super_block_header_a.generation & 1);
1039    }
1040
1041    #[fuchsia::test]
1042    async fn test_root_parent_is_compacted() {
1043        let device = DeviceHolder::new(FakeDevice::new(8192, TEST_DEVICE_BLOCK_SIZE));
1044
1045        let fs = FxFilesystem::new_empty(device).await.expect("new_empty failed");
1046
1047        let mut transaction = fs
1048            .clone()
1049            .new_transaction(lock_keys![], Options::default())
1050            .await
1051            .expect("new_transaction failed");
1052        let store = fs.root_parent_store();
1053        let handle =
1054            ObjectStore::create_object(&store, &mut transaction, HandleOptions::default(), None)
1055                .await
1056                .expect("create_object failed");
1057        transaction.commit().await.expect("commit failed");
1058
1059        store
1060            .tombstone_object(handle.object_id(), Options::default())
1061            .await
1062            .expect("tombstone failed");
1063
1064        // Generate enough work to induce a journal flush.
1065        let root_store = fs.root_store();
1066        for _ in 0..6000 {
1067            let mut transaction = fs
1068                .clone()
1069                .new_transaction(lock_keys![], Options::default())
1070                .await
1071                .expect("new_transaction failed");
1072            ObjectStore::create_object(
1073                &root_store,
1074                &mut transaction,
1075                HandleOptions::default(),
1076                None,
1077            )
1078            .await
1079            .expect("create_object failed");
1080            transaction.commit().await.expect("commit failed");
1081        }
1082
1083        // The root parent store should have been compacted, so we shouldn't be able to find any
1084        // record referring to the object we tombstoned.
1085        assert_eq!(
1086            store.tree().find(&ObjectKey::object(handle.object_id())).await.expect("find failed"),
1087            None
1088        );
1089    }
1090}