diff --git a/doc/new_prepare_commit_design.md b/doc/new_prepare_commit_design.md new file mode 100644 index 0000000..a3968ea --- /dev/null +++ b/doc/new_prepare_commit_design.md @@ -0,0 +1,71 @@ +# Proposal: Replicate Root Page Refs and User Data to Simply Job Txns and Trimming + +## Problem Statement + +Currently, a multi-page transaction ("job" from hereafter) is considered +to be durable/committed only once all the following have occurred: + +1. PackedPrepareJob slot written/flushed +2. All new Page data written +3. All PageAllocator ref count updates written/flushed +4. PackedCommitJob slot written + +A PackedCommitJob is currently only a slot offset pointer to the +prepare slot it finalizes. Thus in order to present a committed job's +user data to the application, we need to reference information stored +in the prior PackedPrepareJob slot. This same dependency on both +slots is also present when the VolumeTrimmer is trim the Volume root +log, since the main function of the VolumeTrimmer is to update ref +counts in response to trimmed root refs. + +This current design presents several significant problems: + +1. Additional complexity when reading slots (currently implemented by + the VolumeSlotDemuxer class), since we must store a map from slot + offset to prepare job record until we see the corresponding commit, + in both reading and trimming workflows +2. This poses a dilemma to the trimmer: should we allow the trimming + of a prepare slot, but not its later commit slot? If we do, we + have a commit slot that is essentially useless from the standpoint + of the application. If we don't, this could introduce latency + spikes into trimming due to the fact that "interleaved" jobs + (prepare-1, prepare-2, commit-1, prepare-3, commit-2, prepare-4, + etc.) could indefinitely arrest trimming, up to the maximum + capacity of the log. This is possible even today, with serialized + jobs, since we allow pipelining of the job txn protocol steps + enumerated above. + +As of 2023-08-23 (when this is being written), the VolumeTrimmer +implements further problematic behavior, namely it will happily trim +the prepare slot for an _ongoing_ transaction! When it does so, it +attempts to roll-forward the information from the prepare slot that it +cares about (the root ref list), which has resulted in the +implementation of a complicated and buggy scheme of Grant reservation +and management. Given the complexity of this system, it is very +difficult to achieve high confidence in its correctness. + +Worse (and more to the point), the very scenario it presumes is +nonsensical: what use is it to trim a prepare _before the job has even +committed?_ Recall from above that the application is unable to +derive anything useful from just the commit slot, as it is just a +pointer to the slot offset of the prepare (where all useful +information is stored). + +## Solution + +The new design trades a small amount of write amplification for +drastically simpler system design by extending the PackedCommitJob +record to include the (opaque) user data and root page refs from the +prepare record. Typically the user data is small (<100 bytes), and +the increased I/O is largely offset by alleviating the need for +obscure hacks like reserving extra space in the PackedPrepareJob +record to balance the trimmer's current roll-forward scheme (this is +so that the Grant needed to roll prepare information forward can +always be reclaimed from the trimmed region itself). Since the commit +is no longer depend on the earlier prepare slot, the new design +eliminates the need to keep track of prepare slots as the root log is +read, and the trimmer no longer faces the dilemma described above: it +can simply treat commit slots as stand-alone records and take action +accordingly when they are trimmed. This means that log scanning gets +faster, and we can get rid of the `depends_on_slot` field in the +`SlotParse` structure, simplifying application code. diff --git a/src/llfs/appendable_job.cpp b/src/llfs/appendable_job.cpp index 097de0e..da8bdba 100644 --- a/src/llfs/appendable_job.cpp +++ b/src/llfs/appendable_job.cpp @@ -45,4 +45,27 @@ PrepareJob prepare(const AppendableJob& appendable) }; } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 AppendableJob::calculate_grant_size() const noexcept +{ + const usize user_data_size = packed_sizeof(this->user_data); + const usize root_refs_size = packed_array_size(this->job.root_count()); + + return // + packed_sizeof_slot_with_payload_size( // + sizeof(PackedPrepareJob) // + + user_data_size // + + root_refs_size // + + packed_array_size(this->job.new_page_count()) // + + packed_array_size(this->job.deleted_page_count()) // + + packed_array_size(this->job.page_device_count()) // + ) // + + packed_sizeof_slot_with_payload_size( // + sizeof(PackedCommitJob) // + + user_data_size // + + root_refs_size // + ); +} + } // namespace llfs diff --git a/src/llfs/appendable_job.hpp b/src/llfs/appendable_job.hpp index b4f7b81..6627f74 100644 --- a/src/llfs/appendable_job.hpp +++ b/src/llfs/appendable_job.hpp @@ -9,7 +9,7 @@ #ifndef LLFS_APPENDABLE_JOB_HPP #define LLFS_APPENDABLE_JOB_HPP -#include +#include #include #include #include @@ -29,6 +29,13 @@ struct PrepareJob; struct AppendableJob { CommittablePageCacheJob job; PackableRef user_data; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns the total grant size needed to append both the PrepareJob and CommitJob events + * for this job. + */ + u64 calculate_grant_size() const noexcept; }; // Construct an AppendableJob. diff --git a/src/llfs/committable_page_cache_job.cpp b/src/llfs/committable_page_cache_job.cpp new file mode 100644 index 0000000..eb1a39c --- /dev/null +++ b/src/llfs/committable_page_cache_job.cpp @@ -0,0 +1,579 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +#include +#include + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*static*/ StatusOr CommittablePageCacheJob::from( + std::unique_ptr job, u64 callers) +{ + StatusOr prune_status = job->prune(callers | Caller::PageCacheJob_finalize); + BATT_REQUIRE_OK(prune_status); + + // This job will no longer be changing, so unpin pages to save memory. + // + job->unpin_all(); + + auto committable_job = CommittablePageCacheJob{std::move(job)}; + + // Calculate page reference count updates for all devices. + // + BATT_ASSIGN_OK_RESULT(committable_job.ref_count_updates_, + committable_job.get_page_ref_count_updates(callers)); + + return committable_job; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +CommittablePageCacheJob::CommittablePageCacheJob( + std::unique_ptr finalized_job) noexcept + : job_{std::move(finalized_job)} + , tracker_{new FinalizedJobTracker{this->job_}} +{ + BATT_CHECK(this->job_->is_pruned()); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +CommittablePageCacheJob::~CommittablePageCacheJob() noexcept +{ + if (this->job_ && this->tracker_) { + this->tracker_->progress_.modify([](PageCacheJobProgress old) { + if (is_terminal_state(old)) { + return old; + } + return PageCacheJobProgress::kCancelled; + }); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 CommittablePageCacheJob::job_id() const +{ + return this->job_->job_id; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +BoxedSeq CommittablePageCacheJob::new_page_ids() const +{ + return as_seq(this->job_->get_new_pages().begin(), this->job_->get_new_pages().end()) // + | seq::map([](const auto& kv_pair) -> PageId { + return kv_pair.first; + }) // + | seq::boxed(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize CommittablePageCacheJob::new_page_count() const noexcept +{ + return this->job_->get_new_pages().size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +BoxedSeq CommittablePageCacheJob::deleted_page_ids() const +{ + return as_seq(this->job_->get_deleted_pages().begin(), this->job_->get_deleted_pages().end()) // + | seq::map([](const auto& kv_pair) -> PageId { + return kv_pair.first; + }) // + | seq::boxed(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize CommittablePageCacheJob::deleted_page_count() const noexcept +{ + return this->job_->get_deleted_pages().size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +BoxedSeq CommittablePageCacheJob::root_set_deltas() const +{ + return as_seq(this->job_->get_root_set_delta().begin(), + this->job_->get_root_set_delta().end()) // + | seq::map([](const auto& kv_pair) { + return PageRefCount{ + .page_id = kv_pair.first, + .ref_count = kv_pair.second, + }; + }) // + | seq::boxed(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize CommittablePageCacheJob::root_count() const noexcept +{ + usize count = 0; + for (const auto& [page_id, ref_count] : this->job_->get_root_set_delta()) { + count += ref_count; + } + return count; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +BoxedSeq CommittablePageCacheJob::page_device_ids() const +{ + // Make sure the ref_count_updates_ is initialized! + // + BATT_CHECK(this->ref_count_updates_.initialized); + + return as_seq(this->ref_count_updates_.per_device.begin(), + this->ref_count_updates_.per_device.end()) // + | seq::map([](const auto& kv_pair) -> page_device_id_int { + return kv_pair.first; + }) // + | seq::boxed(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize CommittablePageCacheJob::page_device_count() const noexcept +{ + return this->ref_count_updates_.per_device.size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void CommittablePageCacheJob::cancel() +{ + if (!this->tracker_) { + return; + } + + this->tracker_->cancel(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +FinalizedPageCacheJob CommittablePageCacheJob::finalized_job() const +{ + return FinalizedPageCacheJob{batt::make_copy(this->tracker_)}; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status commit(std::unique_ptr job, const JobCommitParams& params, u64 callers, + slot_offset_type prev_caller_slot, batt::Watch* durable_caller_slot) +{ + BATT_ASSIGN_OK_RESULT(auto committable_job, + CommittablePageCacheJob::from(std::move(job), callers)); + + return commit(std::move(committable_job), params, callers, prev_caller_slot, durable_caller_slot); +} + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +Status commit(CommittablePageCacheJob committable_job, const JobCommitParams& params, u64 callers, + slot_offset_type prev_caller_slot, batt::Watch* durable_caller_slot) +{ + return committable_job.commit_impl(params, callers, prev_caller_slot, durable_caller_slot); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status CommittablePageCacheJob::commit_impl(const JobCommitParams& params, u64 callers, + slot_offset_type prev_caller_slot, + batt::Watch* durable_caller_slot) +{ + BATT_CHECK_NOT_NULLPTR(params.caller_uuid); + + bool success = false; + const auto on_return = batt::finally([&] { + if (!success) { + this->tracker_->progress_.modify([](PageCacheJobProgress p) { + if (p == PageCacheJobProgress::kDurable) { + return p; + } + return PageCacheJobProgress::kCancelled; + }); + } + }); + + const PageCacheJob* job = this->job_.get(); + BATT_CHECK_NOT_NULLPTR(job); + + LLFS_VLOG(1) << "commit(PageCacheJob): entered"; + + // Make sure the job is pruned! + // + BATT_CHECK(job->is_pruned()); + + // Write new pages. + // + Status write_status = LLFS_COLLECT_LATENCY(job->cache().metrics().page_write_latency, + this->write_new_pages(params, callers)); + BATT_REQUIRE_OK(write_status); + + // Make sure the ref_count_updates_ is initialized! + // + BATT_CHECK(this->ref_count_updates_.initialized); + + //+++++++++++-+-+--+----- --- -- - - - - + // Wait until all previous commits in our pipeline have successfully updated ref counts. + // + Status pipeline_status = + LLFS_COLLECT_LATENCY(this->job_->cache().metrics().pipeline_wait_latency, // + this->job_->await_base_job_durable()); + BATT_REQUIRE_OK(pipeline_status); + + if (durable_caller_slot) { + BATT_CHECK(slot_less_than(prev_caller_slot, params.caller_slot)); + BATT_REQUIRE_OK(await_slot_offset(prev_caller_slot, *durable_caller_slot)); + } + + // Update ref counts, keeping track of the sync point for each device's allocator; this allows the + // updates to happen in parallel. We go through again below to synchronize them. + // + BATT_ASSIGN_OK_RESULT(DeadPages dead_pages, + LLFS_COLLECT_LATENCY(job->cache().metrics().update_ref_counts_latency, + this->start_ref_count_updates( + params, this->ref_count_updates_, callers))); + + // Wait for all ref count updates to complete. + // + Status ref_count_status = + LLFS_COLLECT_LATENCY(job->cache().metrics().ref_count_sync_latency, + this->await_ref_count_updates(this->ref_count_updates_)); + BATT_REQUIRE_OK(ref_count_status); + + //+++++++++++-+-+--+----- --- -- - - - - + // Now we can allow future commits in our pipeline to continue. + // + this->tracker_->progress_.set_value(PageCacheJobProgress::kDurable); + + if (durable_caller_slot) { + const slot_offset_type prev_durable_slot = durable_caller_slot->set_value(params.caller_slot); + BATT_CHECK_EQ(prev_durable_slot, prev_caller_slot); + } + + // If there are any dead pages, assign their ownership to the recycler. + // - TODO [tastolfi 2021-06-12] this can be moved to its own pipeline stage/task. + // + Status recycle_status = this->recycle_dead_pages(params, dead_pages); + BATT_REQUIRE_OK(recycle_status); + + // Drop any deleted pages from storage. + // + // IMPORTANT: this must be done after updating page ref counts; otherwise if we crash, we will + // never be able to recover the refcounts that must go down because the page is being dropped. + // `PageDevice::drop` is idempotent because of the generation number. + // + Status drop_status = this->drop_deleted_pages(callers); + BATT_REQUIRE_OK(drop_status); + + LLFS_VLOG(1) << "commit(PageCacheJob): done"; + + success = true; + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status CommittablePageCacheJob::write_new_pages(const JobCommitParams& params, u64 callers) +{ + LLFS_VLOG(1) << "commit(PageCacheJob): writing new pages"; + + if (this->job_->get_new_pages().empty()) { + return OkStatus(); + } + + const PageCacheJob* const job = this->job_.get(); + BATT_CHECK_NOT_NULLPTR(job); + + u64 op_count = 0; + u64 used_byte_count = 0; + u64 total_byte_count = 0; + + // Write the pages to their respective PageDevice asynchronously/concurrently to maximize + // throughput. + // + batt::Watch done_counter{0}; + const usize n_ops = job->get_new_pages().size(); + auto ops = PageWriteOp::allocate_array(n_ops, done_counter); + LLFS_VLOG(1) << "commit(PageCacheJob): writing new pages"; + { + usize i = 0; + for (auto& p : job->get_new_pages()) { + const PageId page_id = p.first; + + // There's no need to write recovered pages, since they are already durable; skip. + // + if (job->is_recovered_page(page_id)) { + ops[i].get_handler()(batt::OkStatus()); + continue; + } + + const PageCacheJob::NewPage& new_page = p.second; + std::shared_ptr new_page_view = new_page.view(); + BATT_CHECK_NOT_NULLPTR(new_page_view); + BATT_CHECK_EQ(page_id, new_page_view->page_id()); + BATT_CHECK(job->get_already_pinned(page_id) != None) << BATT_INSPECT(page_id); + + // Finalize the client uuid and slot that uniquely identifies this transaction, so we can + // guarantee exactly-once side effects in the presence of crashes. + { + std::shared_ptr mutable_page_buffer = new_page.buffer(); + BATT_CHECK_NOT_NULLPTR(mutable_page_buffer); + + PackedPageUserSlot& user_slot = mutable_page_header(mutable_page_buffer.get())->user_slot; + if (params.caller_uuid) { + user_slot.user_id = *params.caller_uuid; + } else { + std::memset(&user_slot.user_id, 0, sizeof(user_slot.user_id)); + } + user_slot.slot_offset = params.caller_slot; + } + + // We will need this information to update the metrics below. + // + const PackedPageHeader& page_header = new_page.const_page_header(); + const usize page_size = page_header.size; + const usize used_size = page_header.used_size(); + + ops[i].page_id = page_id; + + job->cache().arena_for_page_id(page_id).device().write(new_page.const_buffer(), + ops[i].get_handler()); + + total_byte_count += page_size; + used_byte_count += used_size; + op_count += page_size / 4096; + ++i; + } + } + + // Wait for all concurrent page writes to finish. + // + auto final_count = done_counter.await_true([&](i64 n) { + return n == (i64)n_ops; + }); + BATT_REQUIRE_OK(final_count); + + // Only proceed if all writes succeeded. + // + Status all_ops_status = OkStatus(); + for (auto& op : as_slice(ops.get(), n_ops)) { + job->cache().track_new_page_event(NewPageTracker{ + .ts = 0, + .job_id = job->job_id, + .page_id = op.page_id, + .callers = callers | Caller::PageCacheJob_commit_0, + .event_id = op.result.ok() ? (int)NewPageTracker::Event::kWrite_Ok + : (int)NewPageTracker::Event::kWrite_Fail, + }); + all_ops_status.Update(op.result); + } + BATT_REQUIRE_OK(all_ops_status); + + job->cache().metrics().total_bytes_written += total_byte_count; + job->cache().metrics().used_bytes_written += used_byte_count; + job->cache().metrics().total_write_ops += op_count; + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +auto CommittablePageCacheJob::start_ref_count_updates(const JobCommitParams& params, + PageRefCountUpdates& updates, u64 /*callers*/) + -> StatusOr +{ + LLFS_VLOG(1) << "commit(PageCacheJob): updating ref counts"; + + DeadPages dead_pages; + + for (auto& [device_id, device_state] : updates.per_device) { + // Hint to the cache that down-referenced pages will probably not be needed again soon. + // + this->hint_pages_obsolete(device_state.ref_count_updates); + + LLFS_VLOG(1) << "calling PageAllocator::update_page_ref_counts for device " << device_id << ";" + << BATT_INSPECT_RANGE(device_state.ref_count_updates); + + const PageArena& arena = this->job_->cache().arena_for_device_id(device_id); + device_state.p_arena = &arena; + + BATT_ASSIGN_OK_RESULT( + device_state.sync_point, + arena.allocator().update_page_ref_counts( + *params.caller_uuid, params.caller_slot, as_seq(device_state.ref_count_updates), + /*dead_page_fn=*/ + [&dead_pages, recycle_depth = params.recycle_depth](PageId dead_page_id) { + LLFS_VLOG(1) << "(recycle event) page is now dead: " << dead_page_id + << " depth=" << recycle_depth; + dead_pages.ids.emplace_back(dead_page_id); + })); + // + // ^^^ TODO [tastolfi 2021-09-13] deal with partial failure + } + + return dead_pages; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status CommittablePageCacheJob::await_ref_count_updates(const PageRefCountUpdates& updates) +{ + LLFS_VLOG(1) << "commit(PageCacheJob): waiting on ref count sync"; + + // Now wait for the allocator logs to flush. + // + for (const auto& [device_id, device_state] : updates.per_device) { + Status sync_status = device_state.p_arena->allocator().sync(device_state.sync_point); + BATT_REQUIRE_OK(sync_status); + } + // + // NOTE: this is the "true" point at which a transaction is durably committed. The commit slot + // in a Tablet WAL is merely a reflection of this fact. + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +auto CommittablePageCacheJob::get_page_ref_count_updates(u64 /*callers*/) const + -> StatusOr +{ + std::unordered_map ref_count_delta = this->job_->get_root_set_delta(); + + // New pages start with a ref count value of 2; 1 for the client doing the allocation, and 1 for + // the future garabage collector that will release any references held by that page. + // + for (const auto& p : this->job_->get_new_pages()) { + const PageId& id = p.first; + if (id) { + ref_count_delta[id] += 1; + } + } + + FinalizedPageCacheJob loader = this->finalized_job(); + + // Trace any new pages reachable from the root set and increment their ref count; existing pages + // are already accounted for existing ref counts (because pages are write-once). + // + Status trace_add_ref_status = this->job_->trace_new_roots(loader, [&ref_count_delta](PageId id) { + if (id) { + ref_count_delta[id] += 1; + } + }); + BATT_REQUIRE_OK(trace_add_ref_status); + + // Trace deleted pages non-recursively, decrementing the ref counts of all pages they directly + // reference. + // + for (const auto& p : this->job_->get_deleted_pages()) { + // Sanity check; deleted pages should have a ref_count_delta of kRefCount_1_to_0. + // + const PageId deleted_page_id = p.first; + { + auto iter = ref_count_delta.find(deleted_page_id); + BATT_CHECK_NE(iter, ref_count_delta.end()); + BATT_CHECK_EQ(iter->second, kRefCount_1_to_0); + } + + // Decrement ref counts. + // + p.second->trace_refs() | seq::for_each([&ref_count_delta, deleted_page_id](PageId id) { + if (id) { + LLFS_VLOG(1) << " decrementing ref count for page " << id + << " (because it was referenced from deleted page " << deleted_page_id << ")"; + ref_count_delta[id] -= 1; + } + }); + } + + // Build the final map of PageRefCount vectors, one per device. + // + PageRefCountUpdates updates; + for (const auto& p : ref_count_delta) { + if (p.second == 0) { + continue; + } + const auto device_id = PageIdFactory::get_device_id(p.first); + updates.per_device[device_id].ref_count_updates.emplace_back(PageRefCount{ + .page_id = p.first, + .ref_count = p.second, + }); + } + + updates.initialized = true; + + return updates; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void CommittablePageCacheJob::hint_pages_obsolete( + const std::vector& ref_count_updates) +{ + for (const PageRefCount& prc : ref_count_updates) { + if (prc.ref_count < 0) { + Optional already_pinned = this->job_->get_already_pinned(PageId{prc.page_id}); + if (already_pinned) { + already_pinned->hint_obsolete(); + } + } + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status CommittablePageCacheJob::recycle_dead_pages(const JobCommitParams& params, + const DeadPages& dead_pages) +{ + LLFS_VLOG(1) << "commit(PageCacheJob): recycling dead pages (count=" << dead_pages.ids.size() + << ")"; + + BATT_CHECK_NOT_NULLPTR(params.recycler.pointer()); + + BATT_ASSIGN_OK_RESULT( + slot_offset_type recycler_sync_point, + params.recycler.recycle_pages(as_slice(dead_pages.ids), params.recycle_grant, + params.recycle_depth + 1)); + + LLFS_VLOG(1) << "commit(PageCacheJob): waiting for PageRecycler sync point"; + + return params.recycler.await_flush(recycler_sync_point); + // + // IMPORTANT: we must only finalize the job after making sure the list of dead pages is flushed to + // the page recycler's log. +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status CommittablePageCacheJob::drop_deleted_pages(u64 callers) +{ + LLFS_VLOG(1) << "commit(PageCacheJob): dropping deleted pages"; + + const auto& deleted_pages = this->job_->get_deleted_pages(); + + return parallel_drop_pages(as_seq(deleted_pages.begin(), deleted_pages.end()) // + | seq::map([](const auto& kv_pair) -> PageId { + return kv_pair.first; + }) // + | seq::collect_vec(), + this->job_->cache(), this->job_->job_id, callers); +} + +} //namespace llfs diff --git a/src/llfs/committable_page_cache_job.hpp b/src/llfs/committable_page_cache_job.hpp new file mode 100644 index 0000000..cc5f170 --- /dev/null +++ b/src/llfs/committable_page_cache_job.hpp @@ -0,0 +1,203 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_COMMITTABLE_PAGE_CACHE_JOB_HPP +#define LLFS_COMMITTABLE_PAGE_CACHE_JOB_HPP + +#include +// +#include +#include +#include +#include +#include +#include + +#include + +#include + +namespace llfs { + +/** \brief A PageCacheJob that has been finalized and is ready to be committed to durable storage. + * + * This class is a move-only value type with unique_ptr-like semantics. The usage pattern is: + * + * 1. Convert a unique_ptr to CommittablePageCacheJob using + * CommittablePageCacheJob::from. + * 2. + */ +class CommittablePageCacheJob +{ + public: + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Converts the passed PageCacheJob to a CommittablePageCacheJob. No changes to the job + * (e.g., adding new pages) are allowed after conversion to CommittablePageCacheJob. + */ + static StatusOr from(std::unique_ptr job, u64 callers); + + // `commit` is implemented as a free function so that CommittablePageCacheJob can be moved into + // it, emphasizing that this is an operation that can only be performed once per job. Declare + // friendship so it can operate like a member function. + // + friend Status commit(CommittablePageCacheJob committable_job, const JobCommitParams& params, + u64 callers, slot_offset_type prev_caller_slot, + batt::Watch* durable_caller_slot); + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Constructs an empty/invalid CommittablePageCacheJob. + */ + CommittablePageCacheJob() = default; + + /** \brief This class is moveable but not copyable. + */ + CommittablePageCacheJob(const CommittablePageCacheJob&) = delete; + + /** \brief This class is moveable but not copyable. + */ + CommittablePageCacheJob& operator=(const CommittablePageCacheJob&) = delete; + + /** \brief This class is moveable but not copyable. + */ + CommittablePageCacheJob(CommittablePageCacheJob&&) = default; + + /** \brief This class is moveable but not copyable. + */ + CommittablePageCacheJob& operator=(CommittablePageCacheJob&&) = default; + + /** \brief Sets the tracker status to kAborted if not already in a terminal state. + */ + ~CommittablePageCacheJob() noexcept; + + /** \brief Creates a new FinalizedPageCacheJob that points to this job. + * + * The returned object can be used as the basis for future jobs (using + * PageCacheJob::set_base_job), so that we don't have to wait for all the new pages to be written + * to device in order to construct a dependent PageCacheJob. That is, requests to load pages via + * a PageCacheJob will automatically see any new pages written in the "base job" + * (FinalizedPageCacheJob) of that PageCacheJob, even if they haven't actually been written to + * storage yet. + */ + FinalizedPageCacheJob finalized_job() const; + + /** \brief Returns the process-unique serial number of this job; this will be the same as + * job->job_id, where job is the PageCacheJob used to create this. + */ + u64 job_id() const; + + /** \brief Returns a sequence of PageIds for the new pages to be written by this job. + */ + BoxedSeq new_page_ids() const; + + /** \brief Returns a sequence of PageIds for the dead pages to be dropped by this job. + */ + BoxedSeq deleted_page_ids() const; + + /** \brief Returns the page reference count updates for _root_ refs (i.e., those accessible + * directly from the WAL) for this job. + */ + BoxedSeq root_set_deltas() const; + + /** \brief Returns the PageDevice ids for which this job will update ref counts. + */ + BoxedSeq page_device_ids() const; + + /** \brief Returns the number of new pages to be written when this job is committed. + */ + usize new_page_count() const noexcept; + + /** \brief Returns the number of pages to be deleted when this job is committed. + */ + usize deleted_page_count() const noexcept; + + /** \brief Returns the number of root page references that will be written to the log when this + * job is committed. + */ + usize root_count() const noexcept; + + /** \brief Returns the number of distinct PageDevice ids for which this job will trigger page ref + * count updates. + */ + usize page_device_count() const noexcept; + + /** \brief Returns true iff this object holds a valid (unique) reference to a PageCacheJob. This + * will be false after passing a CommittablePageCacheJob by move to commit(). + */ + explicit operator bool() const + { + return this->job_ && this->tracker_; + } + + /** \brief Cancels this job, reporting batt::StatusCode::kCancelled to any status listeners. + */ + void cancel(); + + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - + private: + struct DeviceUpdateState { + std::vector ref_count_updates; + const PageArena* p_arena = nullptr; + slot_offset_type sync_point = 0; + }; + + struct PageRefCountUpdates { + std::unordered_map per_device; + bool initialized = false; + }; + + struct DeadPages { + std::vector ids; + }; + + //+++++++++++-+-+--+----- --- -- - - - - + + explicit CommittablePageCacheJob(std::unique_ptr finalized_job) noexcept; + + Status commit_impl(const JobCommitParams& params, u64 callers, slot_offset_type prev_caller_slot, + batt::Watch* durable_caller_slot); + + Status write_new_pages(const JobCommitParams& params, u64 callers); + + StatusOr get_page_ref_count_updates(u64 callers) const; + + StatusOr start_ref_count_updates(const JobCommitParams& params, + PageRefCountUpdates& updates, u64 callers); + + Status await_ref_count_updates(const PageRefCountUpdates& updates); + + Status recycle_dead_pages(const JobCommitParams& params, const DeadPages& dead_pages); + + void hint_pages_obsolete(const std::vector& prcs); + + Status drop_deleted_pages(u64 callers); + + //+++++++++++-+-+--+----- --- -- - - - - + + std::shared_ptr job_; + boost::intrusive_ptr tracker_; + PageRefCountUpdates ref_count_updates_; +}; + +/** \brief Write all changes in `job` to durable storage. This is guaranteed to be atomic. + */ +Status commit(CommittablePageCacheJob committable_job, const JobCommitParams& params, u64 callers, + slot_offset_type prev_caller_slot = 0, + batt::Watch* durable_caller_slot = nullptr); + +/** \brief Convenience shortcut for use cases where we do not pipeline job commits. + */ +Status commit(std::unique_ptr job, const JobCommitParams& params, u64 callers, + slot_offset_type prev_caller_slot = 0, + batt::Watch* durable_caller_slot = nullptr); + +} //namespace llfs + +#endif // LLFS_COMMITTABLE_PAGE_CACHE_JOB_HPP diff --git a/src/llfs/data_packer.cpp b/src/llfs/data_packer.cpp index 356a323..9b7af53 100644 --- a/src/llfs/data_packer.cpp +++ b/src/llfs/data_packer.cpp @@ -53,6 +53,13 @@ void DataPacker::invalidate() this->arena_.invalidate(); } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Optional DataPacker::reserve_front(usize byte_size) noexcept +{ + return this->arena_.allocate_front(byte_size); +} + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // const void* DataPacker::pack_data(const void* data, usize size) diff --git a/src/llfs/data_packer.hpp b/src/llfs/data_packer.hpp index 5bf1b5b..4e42401 100644 --- a/src/llfs/data_packer.hpp +++ b/src/llfs/data_packer.hpp @@ -57,7 +57,7 @@ class DataPacker struct AllocFrontPolicy; struct AllocBackPolicy; - /*! \brief Allocates from Arena in front-to-back order. + /** \brief Allocates from Arena in front-to-back order. */ struct AllocFrontPolicy { static Optional allocate_buffer(Arena* arena, usize size) @@ -71,7 +71,7 @@ class DataPacker } }; - /*! \brief Allocates from Arena in back-to-front order. + /** \brief Allocates from Arena in back-to-front order. */ struct AllocBackPolicy { static Optional allocate_buffer(Arena* arena, usize size) @@ -137,7 +137,7 @@ class DataPacker (rec_end <= this->buffer_end()); } - /*! \brief Allocate buffer from Data-packer's arena for the passed in Type. + /** \brief Allocate buffer from Data-packer's arena for the passed in Type. * * \param count It's specifying number of elements of type 'T' for which memory allocation is * requested. By default it is going to allocate space for 'one' element. @@ -154,13 +154,17 @@ class DataPacker return reinterpret_cast(buf->data()); } - /*! \brief Reserve space at the end of the buffer for later allocation. + /** \brief Reserve space at the end of the buffer for later allocation. * * DataPacker functions that allocate trailing buffer space (pack_data, pack_string, etc.) can be * passed a DataPacker::Arena to allocate portions of the reserved space later. */ [[nodiscard]] Optional reserve_arena(usize size); + /** \brief Reserve `byte_size` bytes of space at the beginning of the available region. + */ + [[nodiscard]] Optional reserve_front(usize byte_size) noexcept; + [[nodiscard]] const void* pack_data(const void* data, usize size); [[nodiscard]] const void* pack_data(const void* data, usize size, Arena* arena); diff --git a/src/llfs/finalized_job_tracker.cpp b/src/llfs/finalized_job_tracker.cpp new file mode 100644 index 0000000..c1dff74 --- /dev/null +++ b/src/llfs/finalized_job_tracker.cpp @@ -0,0 +1,72 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ FinalizedJobTracker::FinalizedJobTracker( + const std::shared_ptr& job) noexcept + : job_{job} + , progress_{PageCacheJobProgress::kPending} +{ +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status FinalizedJobTracker::await_durable() +{ + StatusOr seen = this->progress_.await_true([](PageCacheJobProgress value) { + return value == PageCacheJobProgress::kDurable || is_terminal_state(value); + }); + + BATT_REQUIRE_OK(seen); + + switch (*seen) { + case PageCacheJobProgress::kPending: + BATT_PANIC() << "Pending is not a terminal state!"; + BATT_UNREACHABLE(); + + case PageCacheJobProgress::kCancelled: + return batt::StatusCode::kCancelled; + + case PageCacheJobProgress::kDurable: + return OkStatus(); + } + + return batt::StatusCode::kUnknown; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +PageCacheJobProgress FinalizedJobTracker::get_progress() const +{ + return this->progress_.get_value(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void FinalizedJobTracker::cancel() +{ + this->progress_.set_value(PageCacheJobProgress::kCancelled); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +std::shared_ptr lock_job(const FinalizedJobTracker* tracker) +{ + if (!tracker) { + return nullptr; + } + return tracker->job_.lock(); +} + +} //namespace llfs diff --git a/src/llfs/finalized_job_tracker.hpp b/src/llfs/finalized_job_tracker.hpp new file mode 100644 index 0000000..e934a6e --- /dev/null +++ b/src/llfs/finalized_job_tracker.hpp @@ -0,0 +1,62 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_FINALIZED_JOB_TRACKER_HPP +#define LLFS_FINALIZED_JOB_TRACKER_HPP + +#include +// +#include +#include + +#include +#include + +#include + +namespace llfs { + +class CommittablePageCacheJob; +class FinalizedJobTracker; +class PageCacheJob; + +std::shared_ptr lock_job(const FinalizedJobTracker* tracker); + +class FinalizedJobTracker : public boost::intrusive_ref_counter +{ + public: + friend class CommittablePageCacheJob; + + using Self = FinalizedJobTracker; + + //+++++++++++-+-+--+----- --- -- - - - - + + friend std::shared_ptr lock_job(const Self*); + + //+++++++++++-+-+--+----- --- -- - - - - + + explicit FinalizedJobTracker(const std::shared_ptr& job) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + + Status await_durable(); + + PageCacheJobProgress get_progress() const; + + void cancel(); + + //+++++++++++-+-+--+----- --- -- - - - - + private: + std::weak_ptr job_; + batt::Watch progress_; +}; + +} //namespace llfs + +#endif // LLFS_FINALIZED_JOB_TRACKER_HPP diff --git a/src/llfs/finalized_page_cache_job.cpp b/src/llfs/finalized_page_cache_job.cpp index 80e7905..97ecf80 100644 --- a/src/llfs/finalized_page_cache_job.cpp +++ b/src/llfs/finalized_page_cache_job.cpp @@ -9,26 +9,11 @@ #include // +#include #include -#include -#include namespace llfs { -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// class FinalizedPageCacheJob -// - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -std::shared_ptr lock_job(const FinalizedJobTracker* tracker) -{ - if (!tracker) { - return nullptr; - } - return tracker->job_.lock(); -} - //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // std::ostream& operator<<(std::ostream& out, const FinalizedPageCacheJob& t) @@ -130,7 +115,7 @@ StatusOr FinalizedPageCacheJob::finalized_get( { const std::shared_ptr job = lock_job(this->tracker_.get()); if (job == nullptr) { - if (this->tracker_ && this->tracker_->get_progress() == PageCacheJobProgress::kAborted) { + if (this->tracker_ && this->tracker_->get_progress() == PageCacheJobProgress::kCancelled) { return Status{batt::StatusCode::kCancelled}; } return Status{batt::StatusCode::kUnavailable}; @@ -160,527 +145,4 @@ Status FinalizedPageCacheJob::await_durable() const return this->tracker_->await_durable(); } -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// class CommittablePageCacheJob -// - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -/*static*/ StatusOr CommittablePageCacheJob::from( - std::unique_ptr job, u64 callers) -{ - StatusOr prune_status = job->prune(callers | Caller::PageCacheJob_finalize); - BATT_REQUIRE_OK(prune_status); - - // This job will no longer be changing, so unpin pages to save memory. - // - job->unpin_all(); - - auto committable_job = CommittablePageCacheJob{std::move(job)}; - - // Calculate page reference count updates for all devices. - // - BATT_ASSIGN_OK_RESULT(committable_job.ref_count_updates_, - committable_job.get_page_ref_count_updates(callers)); - - return committable_job; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -CommittablePageCacheJob::CommittablePageCacheJob( - std::unique_ptr finalized_job) noexcept - : job_{std::move(finalized_job)} - , tracker_{new FinalizedJobTracker{this->job_}} -{ - BATT_CHECK(this->job_->is_pruned()); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -CommittablePageCacheJob::~CommittablePageCacheJob() noexcept -{ - if (this->job_ && this->tracker_) { - this->tracker_->progress_.modify([](PageCacheJobProgress old) { - if (is_terminal_state(old)) { - return old; - } - return PageCacheJobProgress::kAborted; - }); - } -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -u64 CommittablePageCacheJob::job_id() const -{ - return this->job_->job_id; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -BoxedSeq CommittablePageCacheJob::new_page_ids() const -{ - return as_seq(this->job_->get_new_pages().begin(), this->job_->get_new_pages().end()) // - | seq::map([](const auto& kv_pair) -> PageId { - return kv_pair.first; - }) // - | seq::boxed(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -BoxedSeq CommittablePageCacheJob::deleted_page_ids() const -{ - return as_seq(this->job_->get_deleted_pages().begin(), this->job_->get_deleted_pages().end()) // - | seq::map([](const auto& kv_pair) -> PageId { - return kv_pair.first; - }) // - | seq::boxed(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -BoxedSeq CommittablePageCacheJob::root_set_deltas() const -{ - return as_seq(this->job_->get_root_set_delta().begin(), - this->job_->get_root_set_delta().end()) // - | seq::map([](const auto& kv_pair) { - return PageRefCount{ - .page_id = kv_pair.first, - .ref_count = kv_pair.second, - }; - }) // - | seq::boxed(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -BoxedSeq CommittablePageCacheJob::page_device_ids() const -{ - // Make sure the ref_count_updates_ is initialized! - // - BATT_CHECK(this->ref_count_updates_.initialized); - - return as_seq(this->ref_count_updates_.per_device.begin(), - this->ref_count_updates_.per_device.end()) // - | seq::map([](const auto& kv_pair) -> page_device_id_int { - return kv_pair.first; - }) // - | seq::boxed(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -FinalizedPageCacheJob CommittablePageCacheJob::finalized_job() const -{ - return FinalizedPageCacheJob{batt::make_copy(this->tracker_)}; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status commit(std::unique_ptr job, const JobCommitParams& params, u64 callers, - slot_offset_type prev_caller_slot, batt::Watch* durable_caller_slot) -{ - BATT_ASSIGN_OK_RESULT(auto committable_job, - CommittablePageCacheJob::from(std::move(job), callers)); - - return commit(std::move(committable_job), params, callers, prev_caller_slot, durable_caller_slot); -} - -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// -Status commit(CommittablePageCacheJob committable_job, const JobCommitParams& params, u64 callers, - slot_offset_type prev_caller_slot, batt::Watch* durable_caller_slot) -{ - return committable_job.commit_impl(params, callers, prev_caller_slot, durable_caller_slot); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status CommittablePageCacheJob::commit_impl(const JobCommitParams& params, u64 callers, - slot_offset_type prev_caller_slot, - batt::Watch* durable_caller_slot) -{ - BATT_CHECK_NOT_NULLPTR(params.caller_uuid); - - bool success = false; - const auto on_return = batt::finally([&] { - if (!success) { - this->tracker_->progress_.modify([](PageCacheJobProgress p) { - if (p == PageCacheJobProgress::kDurable) { - return p; - } - return PageCacheJobProgress::kAborted; - }); - } - }); - - const PageCacheJob* job = this->job_.get(); - BATT_CHECK_NOT_NULLPTR(job); - - LLFS_VLOG(1) << "commit(PageCacheJob): entered"; - - // Make sure the job is pruned! - // - BATT_CHECK(job->is_pruned()); - - // Write new pages. - // - Status write_status = LLFS_COLLECT_LATENCY(job->cache().metrics().page_write_latency, - this->write_new_pages(params, callers)); - BATT_REQUIRE_OK(write_status); - - // Make sure the ref_count_updates_ is initialized! - // - BATT_CHECK(this->ref_count_updates_.initialized); - - //+++++++++++-+-+--+----- --- -- - - - - - // Wait until all previous commits in our pipeline have successfully updated ref counts. - // - Status pipeline_status = - LLFS_COLLECT_LATENCY(this->job_->cache().metrics().pipeline_wait_latency, // - this->job_->await_base_job_durable()); - BATT_REQUIRE_OK(pipeline_status); - - if (durable_caller_slot) { - BATT_CHECK(slot_less_than(prev_caller_slot, params.caller_slot)); - BATT_REQUIRE_OK(await_slot_offset(prev_caller_slot, *durable_caller_slot)); - } - - // Update ref counts, keeping track of the sync point for each device's allocator; this allows the - // updates to happen in parallel. We go through again below to synchronize them. - // - BATT_ASSIGN_OK_RESULT(DeadPages dead_pages, - LLFS_COLLECT_LATENCY(job->cache().metrics().update_ref_counts_latency, - this->start_ref_count_updates( - params, this->ref_count_updates_, callers))); - - // Wait for all ref count updates to complete. - // - Status ref_count_status = - LLFS_COLLECT_LATENCY(job->cache().metrics().ref_count_sync_latency, - this->await_ref_count_updates(this->ref_count_updates_)); - BATT_REQUIRE_OK(ref_count_status); - - //+++++++++++-+-+--+----- --- -- - - - - - // Now we can allow future commits in our pipeline to continue. - // - this->tracker_->progress_.set_value(PageCacheJobProgress::kDurable); - - if (durable_caller_slot) { - const slot_offset_type prev_durable_slot = durable_caller_slot->set_value(params.caller_slot); - BATT_CHECK_EQ(prev_durable_slot, prev_caller_slot); - } - - // If there are any dead pages, assign their ownership to the recycler. - // - TODO [tastolfi 2021-06-12] this can be moved to its own pipeline stage/task. - // - Status recycle_status = this->recycle_dead_pages(params, dead_pages); - BATT_REQUIRE_OK(recycle_status); - - // Drop any deleted pages from storage. - // - // IMPORTANT: this must be done after updating page ref counts; otherwise if we crash, we will - // never be able to recover the refcounts that must go down because the page is being dropped. - // `PageDevice::drop` is idempotent because of the generation number. - // - Status drop_status = this->drop_deleted_pages(callers); - BATT_REQUIRE_OK(drop_status); - - LLFS_VLOG(1) << "commit(PageCacheJob): done"; - - success = true; - - return OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status CommittablePageCacheJob::write_new_pages(const JobCommitParams& params, u64 callers) -{ - LLFS_VLOG(1) << "commit(PageCacheJob): writing new pages"; - - if (this->job_->get_new_pages().empty()) { - return OkStatus(); - } - - const PageCacheJob* const job = this->job_.get(); - BATT_CHECK_NOT_NULLPTR(job); - - u64 op_count = 0; - u64 used_byte_count = 0; - u64 total_byte_count = 0; - - // Write the pages to their respective PageDevice asynchronously/concurrently to maximize - // throughput. - // - batt::Watch done_counter{0}; - const usize n_ops = job->get_new_pages().size(); - auto ops = PageWriteOp::allocate_array(n_ops, done_counter); - LLFS_VLOG(1) << "commit(PageCacheJob): writing new pages"; - { - usize i = 0; - for (auto& p : job->get_new_pages()) { - const PageId page_id = p.first; - - // There's no need to write recovered pages, since they are already durable; skip. - // - if (job->is_recovered_page(page_id)) { - ops[i].get_handler()(batt::OkStatus()); - continue; - } - - const PageCacheJob::NewPage& new_page = p.second; - std::shared_ptr new_page_view = new_page.view(); - BATT_CHECK_NOT_NULLPTR(new_page_view); - BATT_CHECK_EQ(page_id, new_page_view->page_id()); - BATT_CHECK(job->get_already_pinned(page_id) != None) << BATT_INSPECT(page_id); - - // Finalize the client uuid and slot that uniquely identifies this transaction, so we can - // guarantee exactly-once side effects in the presence of crashes. - { - std::shared_ptr mutable_page_buffer = new_page.buffer(); - BATT_CHECK_NOT_NULLPTR(mutable_page_buffer); - - PackedPageUserSlot& user_slot = mutable_page_header(mutable_page_buffer.get())->user_slot; - if (params.caller_uuid) { - user_slot.user_id = *params.caller_uuid; - } else { - std::memset(&user_slot.user_id, 0, sizeof(user_slot.user_id)); - } - user_slot.slot_offset = params.caller_slot; - } - - // We will need this information to update the metrics below. - // - const PackedPageHeader& page_header = new_page.const_page_header(); - const usize page_size = page_header.size; - const usize used_size = page_header.used_size(); - - ops[i].page_id = page_id; - - job->cache().arena_for_page_id(page_id).device().write(new_page.const_buffer(), - ops[i].get_handler()); - - total_byte_count += page_size; - used_byte_count += used_size; - op_count += page_size / 4096; - ++i; - } - } - - // Wait for all concurrent page writes to finish. - // - auto final_count = done_counter.await_true([&](i64 n) { - return n == (i64)n_ops; - }); - BATT_REQUIRE_OK(final_count); - - // Only proceed if all writes succeeded. - // - Status all_ops_status = OkStatus(); - for (auto& op : as_slice(ops.get(), n_ops)) { - job->cache().track_new_page_event(NewPageTracker{ - .ts = 0, - .job_id = job->job_id, - .page_id = op.page_id, - .callers = callers | Caller::PageCacheJob_commit_0, - .event_id = op.result.ok() ? (int)NewPageTracker::Event::kWrite_Ok - : (int)NewPageTracker::Event::kWrite_Fail, - }); - all_ops_status.Update(op.result); - } - BATT_REQUIRE_OK(all_ops_status); - - job->cache().metrics().total_bytes_written += total_byte_count; - job->cache().metrics().used_bytes_written += used_byte_count; - job->cache().metrics().total_write_ops += op_count; - - return OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -auto CommittablePageCacheJob::start_ref_count_updates(const JobCommitParams& params, - PageRefCountUpdates& updates, u64 /*callers*/) - -> StatusOr -{ - LLFS_VLOG(1) << "commit(PageCacheJob): updating ref counts"; - - DeadPages dead_pages; - - for (auto& [device_id, device_state] : updates.per_device) { - // Hint to the cache that down-referenced pages will probably not be needed again soon. - // - this->hint_pages_obsolete(device_state.ref_count_updates); - - LLFS_VLOG(1) << "calling PageAllocator::update_page_ref_counts for device " << device_id << ";" - << BATT_INSPECT_RANGE(device_state.ref_count_updates); - - const PageArena& arena = this->job_->cache().arena_for_device_id(device_id); - device_state.p_arena = &arena; - - BATT_ASSIGN_OK_RESULT( - device_state.sync_point, - arena.allocator().update_page_ref_counts( - *params.caller_uuid, params.caller_slot, as_seq(device_state.ref_count_updates), - /*dead_page_fn=*/ - [&dead_pages, recycle_depth = params.recycle_depth](PageId dead_page_id) { - LLFS_VLOG(1) << "(recycle event) page is now dead: " << dead_page_id - << " depth=" << recycle_depth; - dead_pages.ids.emplace_back(dead_page_id); - })); - // - // ^^^ TODO [tastolfi 2021-09-13] deal with partial failure - } - - return dead_pages; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status CommittablePageCacheJob::await_ref_count_updates(const PageRefCountUpdates& updates) -{ - LLFS_VLOG(1) << "commit(PageCacheJob): waiting on ref count sync"; - - // Now wait for the allocator logs to flush. - // - for (const auto& [device_id, device_state] : updates.per_device) { - Status sync_status = device_state.p_arena->allocator().sync(device_state.sync_point); - BATT_REQUIRE_OK(sync_status); - } - // - // NOTE: this is the "true" point at which a transaction is durably committed. The commit slot - // in a Tablet WAL is merely a reflection of this fact. - - return OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -auto CommittablePageCacheJob::get_page_ref_count_updates(u64 /*callers*/) const - -> StatusOr -{ - std::unordered_map ref_count_delta = this->job_->get_root_set_delta(); - - // New pages start with a ref count value of 2; 1 for the client doing the allocation, and 1 for - // the future garabage collector that will release any references held by that page. - // - for (const auto& p : this->job_->get_new_pages()) { - const PageId& id = p.first; - if (id) { - ref_count_delta[id] += 1; - } - } - - FinalizedPageCacheJob loader = this->finalized_job(); - - // Trace any new pages reachable from the root set and increment their ref count; existing pages - // are already accounted for existing ref counts (because pages are write-once). - // - Status trace_add_ref_status = this->job_->trace_new_roots(loader, [&ref_count_delta](PageId id) { - if (id) { - ref_count_delta[id] += 1; - } - }); - BATT_REQUIRE_OK(trace_add_ref_status); - - // Trace deleted pages non-recursively, decrementing the ref counts of all pages they directly - // reference. - // - for (const auto& p : this->job_->get_deleted_pages()) { - // Sanity check; deleted pages should have a ref_count_delta of kRefCount_1_to_0. - // - const PageId deleted_page_id = p.first; - { - auto iter = ref_count_delta.find(deleted_page_id); - BATT_CHECK_NE(iter, ref_count_delta.end()); - BATT_CHECK_EQ(iter->second, kRefCount_1_to_0); - } - - // Decrement ref counts. - // - p.second->trace_refs() | seq::for_each([&ref_count_delta, deleted_page_id](PageId id) { - if (id) { - LLFS_VLOG(1) << " decrementing ref count for page " << id - << " (because it was referenced from deleted page " << deleted_page_id << ")"; - ref_count_delta[id] -= 1; - } - }); - } - - // Build the final map of PageRefCount vectors, one per device. - // - PageRefCountUpdates updates; - for (const auto& p : ref_count_delta) { - if (p.second == 0) { - continue; - } - const auto device_id = PageIdFactory::get_device_id(p.first); - updates.per_device[device_id].ref_count_updates.emplace_back(PageRefCount{ - .page_id = p.first, - .ref_count = p.second, - }); - } - - updates.initialized = true; - - return updates; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -void CommittablePageCacheJob::hint_pages_obsolete( - const std::vector& ref_count_updates) -{ - for (const PageRefCount& prc : ref_count_updates) { - if (prc.ref_count < 0) { - Optional already_pinned = this->job_->get_already_pinned(PageId{prc.page_id}); - if (already_pinned) { - already_pinned->hint_obsolete(); - } - } - } -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status CommittablePageCacheJob::recycle_dead_pages(const JobCommitParams& params, - const DeadPages& dead_pages) -{ - LLFS_VLOG(1) << "commit(PageCacheJob): recycling dead pages (count=" << dead_pages.ids.size() - << ")"; - - BATT_CHECK_NOT_NULLPTR(params.recycler.pointer()); - - BATT_ASSIGN_OK_RESULT( - slot_offset_type recycler_sync_point, - params.recycler.recycle_pages(as_slice(dead_pages.ids), params.recycle_grant, - params.recycle_depth + 1)); - - LLFS_VLOG(1) << "commit(PageCacheJob): waiting for PageRecycler sync point"; - - return params.recycler.await_flush(recycler_sync_point); - // - // IMPORTANT: we must only finalize the job after making sure the list of dead pages is flushed to - // the page recycler's log. -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status CommittablePageCacheJob::drop_deleted_pages(u64 callers) -{ - LLFS_VLOG(1) << "commit(PageCacheJob): dropping deleted pages"; - - const auto& deleted_pages = this->job_->get_deleted_pages(); - - return parallel_drop_pages(as_seq(deleted_pages.begin(), deleted_pages.end()) // - | seq::map([](const auto& kv_pair) -> PageId { - return kv_pair.first; - }) // - | seq::collect_vec(), - this->job_->cache(), this->job_->job_id, callers); -} - } // namespace llfs diff --git a/src/llfs/finalized_page_cache_job.hpp b/src/llfs/finalized_page_cache_job.hpp index 9c919a7..4d670ed 100644 --- a/src/llfs/finalized_page_cache_job.hpp +++ b/src/llfs/finalized_page_cache_job.hpp @@ -10,6 +10,7 @@ #ifndef LLFS_FINALIZED_PAGE_CACHE_JOB_HPP #define LLFS_FINALIZED_PAGE_CACHE_JOB_HPP +#include #include #include #include @@ -28,71 +29,8 @@ namespace llfs { class PageCache; -class PageCacheJob; -class PageArena; -class FinalizedJobTracker; -class FinalizedPageCacheJob; class CommittablePageCacheJob; -std::shared_ptr lock_job(const FinalizedJobTracker* tracker); - -class FinalizedJobTracker : public boost::intrusive_ref_counter -{ - public: - friend class CommittablePageCacheJob; - - using Self = FinalizedJobTracker; - - //+++++++++++-+-+--+----- --- -- - - - - - - friend std::shared_ptr lock_job(const Self*); - - //+++++++++++-+-+--+----- --- -- - - - - - - explicit FinalizedJobTracker(const std::shared_ptr& job) noexcept - : job_{job} - , progress_{PageCacheJobProgress::kPending} - { - } - - //+++++++++++-+-+--+----- --- -- - - - - - - Status await_durable() - { - StatusOr seen = - this->progress_.await_true([](PageCacheJobProgress value) { - return value == PageCacheJobProgress::kDurable || is_terminal_state(value); - }); - - BATT_REQUIRE_OK(seen); - - switch (*seen) { - case PageCacheJobProgress::kPending: - BATT_PANIC() << "Pending is not a terminal state!"; - BATT_UNREACHABLE(); - - case PageCacheJobProgress::kAborted: - return batt::StatusCode::kCancelled; - - case PageCacheJobProgress::kDurable: - return OkStatus(); - } - - return batt::StatusCode::kUnknown; - } - - PageCacheJobProgress get_progress() const - { - return this->progress_.get_value(); - } - - private: - std::weak_ptr job_; - batt::Watch progress_; -}; - -std::ostream& operator<<(std::ostream& out, const FinalizedPageCacheJob& t); - class FinalizedPageCacheJob : public PageLoader { public: @@ -136,116 +74,7 @@ class FinalizedPageCacheJob : public PageLoader boost::intrusive_ptr tracker_; }; -// Write all changes in `job` to durable storage. This is guaranteed to be atomic. -// -Status commit(CommittablePageCacheJob committable_job, const JobCommitParams& params, u64 callers, - slot_offset_type prev_caller_slot = 0, - batt::Watch* durable_caller_slot = nullptr); - -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// Represents a unique PageCacheJob that has been finalized and is ready to be committed to durable -// storage. -// -class CommittablePageCacheJob -{ - public: - //+++++++++++-+-+--+----- --- -- - - - - - - static StatusOr from(std::unique_ptr job, u64 callers); - - friend Status commit(CommittablePageCacheJob committable_job, const JobCommitParams& params, - u64 callers, slot_offset_type prev_caller_slot, - batt::Watch* durable_caller_slot); - - //+++++++++++-+-+--+----- --- -- - - - - - - CommittablePageCacheJob() = default; - - // Moveable but not copyable. - // - CommittablePageCacheJob(const CommittablePageCacheJob&) = delete; - CommittablePageCacheJob& operator=(const CommittablePageCacheJob&) = delete; - - CommittablePageCacheJob(CommittablePageCacheJob&&) = default; - CommittablePageCacheJob& operator=(CommittablePageCacheJob&&) = default; - - // Sets the tracker status to kAborted if not already in a terminal state. - // - ~CommittablePageCacheJob() noexcept; - - // Create a new FinalizedPageCacheJob that points to this underlying job. The returned object can - // be used as the basis for future jobs so that we don't have to wait for all the new pages to be - // written to device. - // - FinalizedPageCacheJob finalized_job() const; - - u64 job_id() const; - - BoxedSeq new_page_ids() const; - - BoxedSeq deleted_page_ids() const; - - BoxedSeq root_set_deltas() const; - - BoxedSeq page_device_ids() const; - - explicit operator bool() const - { - return this->job_ && this->tracker_; - } - - private: - //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - - - struct DeviceUpdateState { - std::vector ref_count_updates; - const PageArena* p_arena = nullptr; - slot_offset_type sync_point = 0; - }; - - struct PageRefCountUpdates { - std::unordered_map per_device; - bool initialized = false; - }; - - struct DeadPages { - std::vector ids; - }; - - //+++++++++++-+-+--+----- --- -- - - - - - - explicit CommittablePageCacheJob(std::unique_ptr finalized_job) noexcept; - - Status commit_impl(const JobCommitParams& params, u64 callers, slot_offset_type prev_caller_slot, - batt::Watch* durable_caller_slot); - - Status write_new_pages(const JobCommitParams& params, u64 callers); - - StatusOr get_page_ref_count_updates(u64 callers) const; - - StatusOr start_ref_count_updates(const JobCommitParams& params, - PageRefCountUpdates& updates, u64 callers); - - Status await_ref_count_updates(const PageRefCountUpdates& updates); - - Status recycle_dead_pages(const JobCommitParams& params, const DeadPages& dead_pages); - - void hint_pages_obsolete(const std::vector& prcs); - - Status drop_deleted_pages(u64 callers); - - //+++++++++++-+-+--+----- --- -- - - - - - - std::shared_ptr job_; - boost::intrusive_ptr tracker_; - PageRefCountUpdates ref_count_updates_; -}; - -// Convenience shortcut for use cases where we do not pipeline job commits. -// -Status commit(std::unique_ptr job, const JobCommitParams& params, u64 callers, - slot_offset_type prev_caller_slot = 0, - batt::Watch* durable_caller_slot = nullptr); +std::ostream& operator<<(std::ostream& out, const FinalizedPageCacheJob& t); } // namespace llfs diff --git a/src/llfs/log_device_snapshot.hpp b/src/llfs/log_device_snapshot.hpp index b21e92d..bdd5890 100644 --- a/src/llfs/log_device_snapshot.hpp +++ b/src/llfs/log_device_snapshot.hpp @@ -110,6 +110,11 @@ class LogDeviceSnapshot : public boost::equality_comparable return ConstBuffer{this->bytes(), this->size()}; } + explicit operator bool() const noexcept + { + return this->byte_storage_ != nullptr; + } + private: usize compute_hash_value() const; diff --git a/src/llfs/mem_fuse.test.cpp b/src/llfs/mem_fuse.test.cpp index ebd14ae..610b518 100644 --- a/src/llfs/mem_fuse.test.cpp +++ b/src/llfs/mem_fuse.test.cpp @@ -165,14 +165,19 @@ class MemFuseTest : public ::testing::Test io.run(); }); + const auto try_umount = [&] { + LLFS_LOG_INFO() << "Attempting to umount " << this->mountpoint_; + int retval = umount2(this->mountpoint_str_.c_str(), MNT_FORCE); + LLFS_LOG_INFO() << BATT_INSPECT(batt::status_from_retval(retval)); + }; + // Create a fresh mount point directory. // for (int retry = 0; retry < 2; ++retry) { std::error_code ec; bool mountpoint_exists = std::filesystem::exists(this->mountpoint_, ec); if (ec && retry == 0) { - LLFS_LOG_INFO() << "Attempting to umount " << this->mountpoint_; - umount(this->mountpoint_str_.c_str()); + try_umount(); continue; } @@ -182,7 +187,13 @@ class MemFuseTest : public ::testing::Test } std::filesystem::create_directories(this->mountpoint_, ec); - ASSERT_FALSE(ec) << "Failed to initialize mountpoint"; + if (ec && retry == 0) { + try_umount(); + continue; + } + ASSERT_FALSE(ec) << "Failed to initialize mountpoint:" + << BATT_INSPECT_STR(this->mountpoint_.string()) << BATT_INSPECT(ec.value()) + << BATT_INSPECT(ec.message()); } // Start FUSE session on a background thread. diff --git a/src/llfs/memory_log_device.cpp b/src/llfs/memory_log_device.cpp index 2261cd9..95d6f60 100644 --- a/src/llfs/memory_log_device.cpp +++ b/src/llfs/memory_log_device.cpp @@ -98,6 +98,15 @@ slot_offset_type MemoryLogStorageDriver::flush_up_to_offset(slot_offset_type off { } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ MemoryLogDevice::MemoryLogDevice(usize size, const LogDeviceSnapshot& snapshot, + LogReadMode mode) noexcept + : BasicRingBufferLogDevice{RingBuffer::TempFile{size}} +{ + this->restore_snapshot(snapshot, mode); +} + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // void MemoryLogDevice::restore_snapshot(const LogDeviceSnapshot& snapshot, LogReadMode mode) diff --git a/src/llfs/memory_log_device.hpp b/src/llfs/memory_log_device.hpp index e3b09a7..798bc53 100644 --- a/src/llfs/memory_log_device.hpp +++ b/src/llfs/memory_log_device.hpp @@ -158,6 +158,20 @@ class MemoryLogDevice : public BasicRingBufferLogDevice page_buffer) + -> StatusOr> { + return {std::make_shared(std::move(page_buffer))}; + }; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*static*/ Status OpaquePageView::register_layout(PageCache& cache) noexcept +{ + return cache.register_page_reader(OpaquePageView::page_layout_id(), __FILE__, __LINE__, + OpaquePageView::page_reader()); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +PageLayoutId OpaquePageView::get_page_layout_id() const /*override*/ +{ + return OpaquePageView::page_layout_id(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - diff --git a/src/llfs/opaque_page_view.hpp b/src/llfs/opaque_page_view.hpp index 422c2eb..42688a9 100644 --- a/src/llfs/opaque_page_view.hpp +++ b/src/llfs/opaque_page_view.hpp @@ -10,6 +10,8 @@ #ifndef LLFS_OPAQUE_PAGE_VIEW_HPP #define LLFS_OPAQUE_PAGE_VIEW_HPP +#include +#include #include namespace llfs { @@ -17,6 +19,14 @@ namespace llfs { class OpaquePageView : public PageView { public: + static const PageLayoutId& page_layout_id() noexcept; + + static PageReader page_reader() noexcept; + + static Status register_layout(PageCache& cache) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + using PageView::PageView; // Get the tag for this page view. diff --git a/src/llfs/pack_as_raw.cpp b/src/llfs/pack_as_raw.cpp index f718ce9..9b81620 100644 --- a/src/llfs/pack_as_raw.cpp +++ b/src/llfs/pack_as_raw.cpp @@ -67,4 +67,11 @@ std::string_view raw_data_from_slot(const SlotParse& slot, const PackedRawData* return std::string_view{slot.body.data() + offset, slot.body.size() - offset}; } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +std::ostream& operator<<(std::ostream& out, const PackAsRawData& t) +{ + return out << "PackAsRawData{.bytes=" << batt::c_str_literal(t.bytes) << ",}"; +} + } // namespace llfs diff --git a/src/llfs/pack_as_raw.hpp b/src/llfs/pack_as_raw.hpp index 9fa07c5..35dba48 100644 --- a/src/llfs/pack_as_raw.hpp +++ b/src/llfs/pack_as_raw.hpp @@ -68,6 +68,8 @@ usize packed_sizeof(const PackAsRawData& pack_as_raw_data); */ PackedRawData* pack_object(const PackAsRawData& pack_as_raw_data, DataPacker* dst); +std::ostream& operator<<(std::ostream& out, const PackAsRawData& t); + //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- /** \brief Wraps a reference or copy to some packable type `T`; causes this type to be packed as * PackedRawData. diff --git a/src/llfs/packed_page_id.hpp b/src/llfs/packed_page_id.hpp index b371fbc..1e77c67 100644 --- a/src/llfs/packed_page_id.hpp +++ b/src/llfs/packed_page_id.hpp @@ -77,6 +77,16 @@ inline constexpr usize packed_sizeof(const PackedPageId&) return sizeof(PackedPageId); } +inline bool operator==(const PackedPageId& l, const PackedPageId& r) +{ + return l.id_val == r.id_val; +} + +inline bool operator!=(const PackedPageId& l, const PackedPageId& r) +{ + return !(l == r); +} + BoxedSeq trace_refs(const PackedArray& packed); BoxedSeq trace_refs(const BoxedSeq& page_ids); diff --git a/src/llfs/packed_variant.hpp b/src/llfs/packed_variant.hpp index 5bf46ed..2585045 100644 --- a/src/llfs/packed_variant.hpp +++ b/src/llfs/packed_variant.hpp @@ -78,6 +78,12 @@ struct PackedVariant { { return batt::static_dispatch(this->which.value(), BATT_FORWARD(visitor)); } + + /** \brief If the variant case is the given type, returns a pointer to the packed T; else returns + * nullptr. + */ + template + const T* as(batt::StaticType) const noexcept; }; BATT_STATIC_ASSERT_EQ(sizeof(PackedVariant<>), 1); @@ -112,6 +118,8 @@ inline constexpr u8 index_of_type_within_packed_variant(batt::StaticType(value); } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// template template void PackedVariant::init(batt::StaticType) @@ -120,6 +128,18 @@ void PackedVariant::init(batt::StaticType) this->init(kWhich); } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +template +const T* PackedVariant::as(batt::StaticType) const noexcept +{ + if (this->which == index_of_type_within_packed_variant, T>()) { + return reinterpret_cast(this + 1); + } + return nullptr; +} + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - template diff --git a/src/llfs/page_allocator.cpp b/src/llfs/page_allocator.cpp index 5e5986a..2a9804b 100644 --- a/src/llfs/page_allocator.cpp +++ b/src/llfs/page_allocator.cpp @@ -181,7 +181,8 @@ void PageAllocator::join() noexcept //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -StatusOr PageAllocator::allocate_page(batt::WaitForResource wait_for_resource) +StatusOr PageAllocator::allocate_page(batt::WaitForResource wait_for_resource, + const batt::CancelToken& cancel_token) { // If some attached users are still recovering, then fail/block (to prevent accidental // re-allocation of a page that belonged to some page job that was partially committed). @@ -209,8 +210,15 @@ StatusOr PageAllocator::allocate_page(batt::WaitForResource wait_for_res } } BATT_DEBUG_INFO("[PageAllocator::allocate_page] waiting for free page"); - auto wait_status = this->state_->await_free_page(); - BATT_REQUIRE_OK(wait_status); + if (cancel_token) { + Status status = cancel_token.await([&](auto&& handler) { + this->state_->async_wait_free_page(BATT_FORWARD(handler)); + }); + BATT_REQUIRE_OK(status) << BATT_INSPECT(cancel_token.debug_info()); + } else { + auto wait_status = this->state_->await_free_page(); + BATT_REQUIRE_OK(wait_status); + } } } diff --git a/src/llfs/page_allocator.hpp b/src/llfs/page_allocator.hpp index 8dee965..e1de350 100644 --- a/src/llfs/page_allocator.hpp +++ b/src/llfs/page_allocator.hpp @@ -23,6 +23,7 @@ #include #include +#include #include #include #include @@ -88,7 +89,8 @@ class PageAllocator // Remove a page from the free pool but don't increment its refcount yet. // - StatusOr allocate_page(batt::WaitForResource wait_for_resource); + StatusOr allocate_page(batt::WaitForResource wait_for_resource, + const batt::CancelToken& cancel_token = batt::CancelToken::none()); // Return the given page to the free pool without updating its refcount. // diff --git a/src/llfs/page_allocator.test.cpp b/src/llfs/page_allocator.test.cpp index 86551c5..6442d1f 100644 --- a/src/llfs/page_allocator.test.cpp +++ b/src/llfs/page_allocator.test.cpp @@ -18,6 +18,8 @@ #include #include +#include +#include #include #include #include @@ -1226,4 +1228,81 @@ TEST(PageAllocatorTest, TooManyAttachments) } } +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +TEST(PageAllocatorTest, CancelAllocate) +{ + constexpr usize kNumRandomSeeds = 100; + constexpr usize kNumPages = 4; + constexpr usize kMaxAttachments = 64; + static const usize kLogSize = llfs::PageAllocator::calculate_log_size(kNumPages, kMaxAttachments); + + const llfs::PageAllocatorRuntimeOptions options{ + .scheduler = batt::Runtime::instance().default_scheduler(), + .name = "TestAllocator", + }; + + const llfs::PageIdFactory id_factory{llfs::PageCount{kNumPages}, /*device_id=*/0}; + + for (usize seed = 0; seed < kNumRandomSeeds; ++seed) { + batt::StatusOr> page_allocator_status = + llfs::PageAllocator::recover(options, id_factory, + *std::make_unique(kLogSize)); + + ASSERT_TRUE(page_allocator_status.ok()) << BATT_INSPECT(page_allocator_status.status()); + + llfs::PageAllocator& page_allocator = **page_allocator_status; + + // Allocate all the pages. + // + std::vector allocated_page_ids; + for (usize i = 0; i < kNumPages; ++i) { + llfs::StatusOr result = + page_allocator.allocate_page(batt::WaitForResource::kFalse); + ASSERT_TRUE(result.ok()) << BATT_INSPECT(result.status()); + + allocated_page_ids.emplace_back(*result); + } + + // Try to allocate more in non-blocking mode; this should immediately fail with + // kResourceExhausted. + // + { + llfs::StatusOr result = + page_allocator.allocate_page(batt::WaitForResource::kFalse); + + EXPECT_EQ(result.status(), batt::StatusCode::kResourceExhausted); + } + + // Now launch a background Task to do a blocking alloc; cancel it in another task and verify + // that the status was `kCancelled`. + // + llfs::StatusOr final_result; + + batt::CancelToken cancel_token; + + batt::FakeExecutionContext ex; + + batt::Task allocate_task{ex.get_executor(), + [&cancel_token, &page_allocator, &final_result] { + final_result = page_allocator.allocate_page( + batt::WaitForResource::kTrue, cancel_token); + }, + "PageAllocatorTest.CancelAllocate.allocate_task"}; + + batt::Task canceller_task{ex.get_executor(), + [&cancel_token] { + cancel_token.cancel(); + }, + "PageAllocatorTest.CancelAllocate.canceller_task"}; + + ex.run_with_random_seed(seed); + + allocate_task.join(); + canceller_task.join(); + + EXPECT_EQ(final_result.status(), batt::StatusCode::kCancelled); + } +} + } // namespace diff --git a/src/llfs/page_allocator_state_no_lock.hpp b/src/llfs/page_allocator_state_no_lock.hpp index 8e7403e..1113270 100644 --- a/src/llfs/page_allocator_state_no_lock.hpp +++ b/src/llfs/page_allocator_state_no_lock.hpp @@ -39,6 +39,9 @@ class PageAllocatorStateNoLock Status await_free_page(); + template + void async_wait_free_page(HandlerFn&& handler); + u64 page_device_capacity() const noexcept; u64 free_pool_size() noexcept; @@ -73,4 +76,6 @@ class PageAllocatorStateNoLock } // namespace llfs +#include + #endif // LLFS_PAGE_ALLOCATOR_PAGE_ALLOCATOR_STATE_NO_LOCK_HPP diff --git a/src/llfs/page_allocator_state_no_lock.ipp b/src/llfs/page_allocator_state_no_lock.ipp new file mode 100644 index 0000000..6d5c2af --- /dev/null +++ b/src/llfs/page_allocator_state_no_lock.ipp @@ -0,0 +1,62 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_PAGE_ALLOCATOR_STATE_NO_LOCK_IPP +#define LLFS_PAGE_ALLOCATOR_STATE_NO_LOCK_IPP + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void PageAllocatorStateNoLock::async_wait_free_page(HandlerArg&& handler) +{ + using HandlerFn = std::decay_t; + + static_assert(std::is_same_v); + + class Wrapper + { + public: + explicit Wrapper(HandlerArg&& handler, PageAllocatorStateNoLock* state) noexcept + : handler_{BATT_FORWARD(handler)} + , state_{state} + { + } + + Wrapper(const Wrapper&) = default; + Wrapper& operator=(const Wrapper&) = default; + + //----- --- -- - - - - + + void operator()(StatusOr result) + { + if (result.ok() && *result == 0) { + PageAllocatorStateNoLock* const local_state = this->state_; + local_state->free_pool_size_.async_wait(/*last_seen=*/0, std::move(*this)); + } else { + HandlerFn local_handler = BATT_FORWARD(this->handler_); + BATT_FORWARD(local_handler)(result.status()); + } + } + + //----- --- -- - - - - + private: + HandlerFn handler_; + PageAllocatorStateNoLock* state_; + }; + + Wrapper wrapper{BATT_FORWARD(handler), this}; + + wrapper(StatusOr{0}); +} + +} //namespace llfs + +#endif // LLFS_PAGE_ALLOCATOR_STATE_NO_LOCK_IPP diff --git a/src/llfs/page_cache.cpp b/src/llfs/page_cache.cpp index d861bb6..4921d0c 100644 --- a/src/llfs/page_cache.cpp +++ b/src/llfs/page_cache.cpp @@ -9,6 +9,7 @@ #include // +#include #include #include #include @@ -37,6 +38,8 @@ Status PageCache::PageDeleterImpl::delete_pages(const Slice if (to_delete.empty()) { return OkStatus(); } + BATT_DEBUG_INFO("delete_pages()" << BATT_INSPECT_RANGE(to_delete) << BATT_INSPECT(recycle_depth)); + const boost::uuids::uuid& caller_uuid = recycler.uuid(); JobCommitParams params{ @@ -94,8 +97,7 @@ PageCache::PageCache(std::vector&& storage_pool, , arenas_by_size_log2_{} , arenas_by_device_id_{} , impl_for_size_log2_{} - , page_readers_{std::make_shared< - batt::Mutex>>()} + , page_readers_{std::make_shared>()} { // Sort the storage pool by page size (MUST be first). // @@ -177,7 +179,39 @@ const PageCacheOptions& PageCache::options() const // bool PageCache::register_page_layout(const PageLayoutId& layout_id, const PageReader& reader) { - return this->page_readers_->lock()->emplace(layout_id, reader).second; + LLFS_LOG_WARNING() << "PageCache::register_page_layout is DEPRECATED; please use " + "PageCache::register_page_reader"; + + return this->page_readers_->lock() + ->emplace(layout_id, + PageReaderFromFile{ + .page_reader = reader, + .file = __FILE__, + .line = __LINE__, + }) + .second; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +batt::Status PageCache::register_page_reader(const PageLayoutId& layout_id, const char* file, + int line, const PageReader& reader) +{ + auto locked = this->page_readers_->lock(); + const auto [iter, was_inserted] = locked->emplace(layout_id, PageReaderFromFile{ + .page_reader = reader, + .file = file, + .line = line, + }); + + if (!was_inserted && (iter->second.file != file || iter->second.line != line)) { + return ::llfs::make_status(StatusCode::kPageReaderConflict); + } + + BATT_CHECK_EQ(iter->second.file, file); + BATT_CHECK_EQ(iter->second.line, line); + + return batt::OkStatus(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -208,19 +242,22 @@ std::unique_ptr PageCache::new_job() //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // StatusOr> PageCache::allocate_page_of_size( - PageSize size, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id) + PageSize size, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id, + const batt::CancelToken& cancel_token) { const PageSizeLog2 size_log2 = log2_ceil(size); BATT_CHECK_EQ(usize{1} << size_log2, size) << "size must be a power of 2"; - return this->allocate_page_of_size_log2( - size_log2, wait_for_resource, callers | Caller::PageCache_allocate_page_of_size, job_id); + return this->allocate_page_of_size_log2(size_log2, wait_for_resource, + callers | Caller::PageCache_allocate_page_of_size, job_id, + std::move(cancel_token)); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // StatusOr> PageCache::allocate_page_of_size_log2( - PageSizeLog2 size_log2, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id) + PageSizeLog2 size_log2, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id, + const batt::CancelToken& cancel_token) { BATT_CHECK_LT(size_log2, kMaxPageSizeLog2); @@ -233,8 +270,12 @@ StatusOr> PageCache::allocate_page_of_size_log2( // for (auto wait_arg : {batt::WaitForResource::kFalse, batt::WaitForResource::kTrue}) { for (const PageArena& arena : arenas) { - StatusOr page_id = arena.allocator().allocate_page(wait_arg); + StatusOr page_id = arena.allocator().allocate_page(wait_arg, cancel_token); if (!page_id.ok()) { + if (page_id.status() == batt::StatusCode::kResourceExhausted) { + const u64 page_size = u64{1} << size_log2; + LLFS_LOG_INFO_FIRST_N(1) << "Failed to allocate page (pool is empty): " << BATT_INSPECT(page_size); + } continue; } @@ -386,6 +427,14 @@ StatusOr PageCache::put_view(std::shared_ptr&& view, { BATT_CHECK_NOT_NULLPTR(view); + if (view->get_page_layout_id() != view->header().layout_id) { + return {::llfs::make_status(StatusCode::kPageHeaderBadLayoutId)}; + } + + if (this->page_readers_->lock()->count(view->get_page_layout_id()) == 0) { + return {::llfs::make_status(StatusCode::kPutViewUnknownLayoutId)}; + } + const page_id_int id_val = view->page_id().int_value(); BATT_CHECK_NE(id_val, kInvalidPageId); @@ -561,10 +610,14 @@ auto PageCache::find_page_in_cache(PageId page_id, const Optional& auto locked = page_readers->lock(); auto iter = locked->find(layout_id); if (iter == locked->end()) { + LLFS_LOG_ERROR() << "Unknown page layout: " + << batt::c_str_literal( + std::string_view{(const char*)&layout_id, sizeof(layout_id)}) + << BATT_INSPECT(page_id); latch->set_value(make_status(StatusCode::kNoReaderForPageViewType)); return; } - reader_for_layout = iter->second; + reader_for_layout = iter->second.page_reader; } // ^^ Release the page_readers mutex ASAP diff --git a/src/llfs/page_cache.hpp b/src/llfs/page_cache.hpp index 4ec7e0f..2553154 100644 --- a/src/llfs/page_cache.hpp +++ b/src/llfs/page_cache.hpp @@ -10,6 +10,8 @@ #ifndef LLFS_PAGE_CACHE_HPP #define LLFS_PAGE_CACHE_HPP +#include +// #include #include #include @@ -36,6 +38,7 @@ #include #include +#include #include #include @@ -95,6 +98,12 @@ class PageCache : public PageLoader public: using CacheImpl = Cache>>; + struct PageReaderFromFile { + PageReader page_reader; + const char* file; + int line; + }; + class PageDeleterImpl : public PageDeleter { public: @@ -129,8 +138,13 @@ class PageCache : public PageLoader const PageCacheOptions& options() const; + /** \brief DEPRECATED - use register_page_reader. + */ bool register_page_layout(const PageLayoutId& layout_id, const PageReader& reader); + batt::Status register_page_reader(const PageLayoutId& layout_id, const char* file, int line, + const PageReader& reader); + void close(); void join(); @@ -138,10 +152,12 @@ class PageCache : public PageLoader std::unique_ptr new_job(); StatusOr> allocate_page_of_size( - PageSize size, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id); + PageSize size, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id, + const batt::CancelToken& cancel_token = None); StatusOr> allocate_page_of_size_log2( - PageSizeLog2 size_log2, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id); + PageSizeLog2 size_log2, batt::WaitForResource wait_for_resource, u64 callers, u64 job_id, + const batt::CancelToken& cancel_token = None); // Returns a page allocated via `allocate_page` to the free pool. This MUST be done before the // page is written to the `PageDevice`. @@ -218,7 +234,9 @@ class PageCache : public PageLoader private: //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- - using PageLayoutReaderMap = std::unordered_map; + + using PageLayoutReaderMap = + std::unordered_map; //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -265,8 +283,7 @@ class PageCache : public PageLoader // A thread-safe shared map from PageLayoutId to PageReader function; layouts must be registered // with the PageCache so that we trace references during page recycling (aka garbage collection). // - std::shared_ptr>> - page_readers_; + std::shared_ptr> page_readers_; //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // TODO [tastolfi 2021-09-08] We need something akin to the PageRecycler/PageAllocator to durably diff --git a/src/llfs/page_cache_job.cpp b/src/llfs/page_cache_job.cpp index c3c4cef..c8765af 100644 --- a/src/llfs/page_cache_job.cpp +++ b/src/llfs/page_cache_job.cpp @@ -102,16 +102,22 @@ bool PageCacheJob::is_page_new_and_pinned(PageId page_id) const //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // StatusOr> PageCacheJob::new_page( - PageSize size, batt::WaitForResource wait_for_resource, u64 callers) + PageSize size, batt::WaitForResource wait_for_resource, const PageLayoutId& layout_id, + u64 callers, const batt::CancelToken& cancel_token) { // TODO [tastolfi 2021-04-07] instead of WaitForResource::kTrue, implement a backoff-and-retry // loop with a cancel token. // StatusOr> buffer = this->cache_->allocate_page_of_size( - size, wait_for_resource, callers | Caller::PageCacheJob_new_page, this->job_id); + size, wait_for_resource, callers | Caller::PageCacheJob_new_page, this->job_id, cancel_token); + BATT_REQUIRE_OK(buffer); const PageId page_id = buffer->get()->page_id(); + { + PackedPageHeader* const header = mutable_page_header(buffer->get()); + header->layout_id = layout_id; + } this->pruned_ = false; this->new_pages_.emplace(page_id, NewPage{batt::make_copy(*buffer)}); @@ -175,7 +181,8 @@ StatusOr PageCacheJob::pin_new(std::shared_ptr&& page_view }); BATT_REQUIRE_OK(pinned_page) << batt::LogLevel::kInfo << "Failed to pin page " << id - << ", reason: " << pinned_page.status(); + << ", reason: " << pinned_page.status() + << BATT_INSPECT(page_view->get_page_layout_id()); // Add to the pinned set. // diff --git a/src/llfs/page_cache_job.hpp b/src/llfs/page_cache_job.hpp index 905070c..80e277c 100644 --- a/src/llfs/page_cache_job.hpp +++ b/src/llfs/page_cache_job.hpp @@ -29,8 +29,6 @@ namespace llfs { -class PageCacheJob; - class PageCacheJob : public PageLoader { public: @@ -128,7 +126,8 @@ class PageCacheJob : public PageLoader // StatusOr> new_page(PageSize size, batt::WaitForResource wait_for_resource, - u64 callers); + const PageLayoutId& layout_id, u64 callers, + const batt::CancelToken& cancel_token); // Inserts a new page into the cache. The passed PageView must have been created using a // PageBuffer returned by `new_page` for this job, or we will panic. diff --git a/src/llfs/page_cache_job_progress.hpp b/src/llfs/page_cache_job_progress.hpp index 0fd9998..0063694 100644 --- a/src/llfs/page_cache_job_progress.hpp +++ b/src/llfs/page_cache_job_progress.hpp @@ -19,7 +19,7 @@ namespace llfs { enum struct PageCacheJobProgress { kPending, kDurable, - kAborted, + kCancelled, }; inline bool is_terminal_state(PageCacheJobProgress t) @@ -29,7 +29,7 @@ inline bool is_terminal_state(PageCacheJobProgress t) return false; case PageCacheJobProgress::kDurable: return true; - case PageCacheJobProgress::kAborted: + case PageCacheJobProgress::kCancelled: return true; } BATT_PANIC() << "bad value for PageCacheJobProgress: " << (int)t; @@ -43,8 +43,8 @@ inline std::ostream& operator<<(std::ostream& out, PageCacheJobProgress t) return out << "Pending"; case PageCacheJobProgress::kDurable: return out << "Durable"; - case PageCacheJobProgress::kAborted: - return out << "Aborted"; + case PageCacheJobProgress::kCancelled: + return out << "Cancelled"; } return out << "Unknown"; } diff --git a/src/llfs/page_graph_node.cpp b/src/llfs/page_graph_node.cpp index 2f980aa..217ecbc 100644 --- a/src/llfs/page_graph_node.cpp +++ b/src/llfs/page_graph_node.cpp @@ -104,9 +104,9 @@ StatusOr PageGraphNodeBuilder::build(PageCacheJob& job) && //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -/*static*/ bool PageGraphNodeView::register_layout(PageCache& cache) +/*static*/ batt::Status PageGraphNodeView::register_layout(PageCache& cache) { - return cache.register_page_layout(PageGraphNodeView::page_layout_id(), + return cache.register_page_reader(PageGraphNodeView::page_layout_id(), __FILE__, __LINE__, PageGraphNodeView::page_reader()); } diff --git a/src/llfs/page_graph_node.hpp b/src/llfs/page_graph_node.hpp index d9e600c..9518863 100644 --- a/src/llfs/page_graph_node.hpp +++ b/src/llfs/page_graph_node.hpp @@ -90,7 +90,7 @@ class PageGraphNodeView : public PageView /** \brief Registers this page layout with the passed cache, so that pages using the layout can be * correctly loaded and parsed by the PageCache. */ - static bool register_layout(PageCache& cache); + static batt::Status register_layout(PageCache& cache); /** \brief Returns a shared instance of PageGraphNodeView for the given page data. * \return error status if the page is ill-formed diff --git a/src/llfs/page_view.hpp b/src/llfs/page_view.hpp index 071215a..38cccbb 100644 --- a/src/llfs/page_view.hpp +++ b/src/llfs/page_view.hpp @@ -45,6 +45,11 @@ class PageView virtual ~PageView() = default; + const PackedPageHeader& header() const + { + return ::llfs::get_page_header(*this->data_); + } + PageId page_id() const noexcept { return this->data_->page_id(); diff --git a/src/llfs/raw_volume_log_data_parser.hpp b/src/llfs/raw_volume_log_data_parser.hpp index da51132..2f573be 100644 --- a/src/llfs/raw_volume_log_data_parser.hpp +++ b/src/llfs/raw_volume_log_data_parser.hpp @@ -56,10 +56,6 @@ class RawVolumeLogDataParser */ Optional current_chunk_slot_range_; - /** \brief Used by the demuxer object inside `parse_chunk` to track prepare/commit event pairs. - */ - VolumePendingJobsMap pending_jobs_; - /** \brief The last value reported by VolumeSlotDemuxer::get_visited_upper_bound (inside * parse_chunk). */ diff --git a/src/llfs/raw_volume_log_data_parser.ipp b/src/llfs/raw_volume_log_data_parser.ipp index dfb53a7..6cd8f21 100644 --- a/src/llfs/raw_volume_log_data_parser.ipp +++ b/src/llfs/raw_volume_log_data_parser.ipp @@ -50,8 +50,7 @@ inline StatusOr RawVolumeLogDataParser::parse_chunk( // Create a demuxer to filter out all Volume Events from the user events. // - VolumeSlotDemuxer demuxer{wrapped_visitor_fn, - this->pending_jobs_}; + VolumeSlotDemuxer demuxer{wrapped_visitor_fn}; // Create a LogDevice::Reader for the current chunk data. // diff --git a/src/llfs/raw_volume_log_data_parser.test.cpp b/src/llfs/raw_volume_log_data_parser.test.cpp index 686bd43..18c4e8a 100644 --- a/src/llfs/raw_volume_log_data_parser.test.cpp +++ b/src/llfs/raw_volume_log_data_parser.test.cpp @@ -21,6 +21,7 @@ namespace { using namespace llfs::int_types; +using namespace llfs::constants; using llfs::MockSlotVisitorFn; @@ -41,6 +42,7 @@ class RawVolumeLogDataParserTest : public ::testing::Test public: void SetUp() override { + this->log_buffer.reserve(1 * kMiB); this->reset_parser(); } @@ -52,10 +54,20 @@ class RawVolumeLogDataParserTest : public ::testing::Test // Packs the event as a VolumeEventVariant slot at the end of `this->log_buffer`. // template - void append_event(T&& event) + llfs::SlotParseWithPayload*> append_event(T&& event) { - const usize slot_body_size = sizeof(llfs::VolumeEventVariant) + packed_sizeof(event); - const usize slot_size = llfs::packed_sizeof_slot(event); + const usize event_size = packed_sizeof(event); + const usize slot_body_size = sizeof(llfs::VolumeEventVariant) + event_size; + const usize slot_size = llfs::packed_sizeof_slot_with_payload_size(event_size); + + LLFS_VLOG(1) << "append_event: " << BATT_INSPECT(batt::name_of()) + << BATT_INSPECT(slot_body_size) << BATT_INSPECT(slot_size) + << BATT_INSPECT(batt::make_printable(event)); + + llfs::SlotRange slot_range{ + .lower_bound = this->log_buffer.size(), + .upper_bound = this->log_buffer.size() + slot_size, + }; this->log_buffer.resize(this->log_buffer.size() + slot_size); @@ -65,12 +77,25 @@ class RawVolumeLogDataParserTest : public ::testing::Test BATT_CHECK_GE(slot_begin, this->log_buffer.data()); { - llfs::MutableBuffer slot_buffer{slot_begin, slot_size}; + const llfs::MutableBuffer slot_buffer{slot_begin, slot_size}; llfs::DataPacker packer{slot_buffer}; BATT_CHECK_NOT_NULLPTR(packer.pack_varint(slot_body_size)); - BATT_CHECK_NOT_NULLPTR(llfs::pack_object( - llfs::pack_as_variant(BATT_FORWARD(event)), &packer)); + + const llfs::VolumeEventVariant* packed_event = llfs::pack_object( + llfs::pack_as_variant(BATT_FORWARD(event)), &packer); + + BATT_CHECK_NOT_NULLPTR(packed_event); + + return llfs::SlotParseWithPayload*>{ + .slot = + llfs::SlotParse{ + .offset = slot_range, + .body = std::string_view{(const char*)slot_buffer.data(), slot_buffer.size()}, + .total_grant_spent = slot_size, + }, + .payload = packed_event->as(batt::StaticType>{}), + }; } } @@ -159,7 +184,7 @@ TEST_F(RawVolumeLogDataParserTest, Test) .upper_bound = second_slot_end, }, .body = user_event_1_packed_variant, - .depends_on_offset = batt::None, + .total_grant_spent = second_slot_end - second_slot_begin, }; { @@ -186,12 +211,15 @@ TEST_F(RawVolumeLogDataParserTest, Test) EXPECT_EQ(third_slot_begin, second_slot_end); - this->append_event(llfs::PrepareJob{ - .new_page_ids = batt::seq::Empty{} | batt::seq::boxed(), - .deleted_page_ids = batt::seq::Empty{} | batt::seq::boxed(), - .page_device_ids = batt::seq::Empty{} | batt::seq::boxed(), - .user_data = llfs::PackableRef{third_slot_user_data}, - }); + llfs::SlotParseWithPayload packed_prepare = + this->append_event(llfs::PrepareJob{ + .new_page_ids = batt::seq::Empty{} | batt::seq::boxed(), + .deleted_page_ids = batt::seq::Empty{} | batt::seq::boxed(), + .page_device_ids = batt::seq::Empty{} | batt::seq::boxed(), + .user_data = llfs::PackableRef{third_slot_user_data}, + }); + + EXPECT_EQ(packed_prepare.slot.offset.lower_bound, third_slot_begin); const usize third_slot_end = this->log_buffer.size(); @@ -217,17 +245,33 @@ TEST_F(RawVolumeLogDataParserTest, Test) EXPECT_EQ(fourth_slot_begin, third_slot_end); - llfs::PackedCommitJob packed_commit_job_event{ - .reserved_ = {0}, - .prepare_slot = third_slot_begin, + llfs::CommitJob commit_job_event{ + .prepare_slot_offset = third_slot_begin, + .packed_prepare = packed_prepare.payload, }; + std::array commit_buffer; + auto& expected_commit = reinterpret_cast((commit_buffer)); + { + expected_commit.prepare_slot_offset = third_slot_begin; + expected_commit.prepare_slot_size = (u32)packed_prepare.slot.offset.size(); + expected_commit.root_page_ids.offset = + sizeof(llfs::PackedPointer>) + + this->user_event_2.size(); + } + std::array)> root_page_ids_buffer; + auto& expected_root_page_ids = + reinterpret_cast&>((root_page_ids_buffer)); + expected_root_page_ids.initialize(0); + const std::string user_event_2_packed_variant = batt::to_string( llfs::index_of_type_within_packed_variant(), - std::string_view{(const char*)&packed_commit_job_event, sizeof(llfs::PackedCommitJob)}); + std::string_view{(const char*)&expected_commit, sizeof(expected_commit)}, + this->user_event_2, + std::string_view{(const char*)&expected_root_page_ids, sizeof(expected_root_page_ids)}); - this->append_event(packed_commit_job_event); + this->append_event(commit_job_event); const usize fourth_slot_end = this->log_buffer.size(); @@ -240,11 +284,8 @@ TEST_F(RawVolumeLogDataParserTest, Test) .upper_bound = fourth_slot_end, }, .body = user_event_2_packed_variant, - .depends_on_offset = - llfs::SlotRange{ - .lower_bound = third_slot_begin, - .upper_bound = third_slot_end, - }, + .total_grant_spent = ((third_slot_end - third_slot_begin) // + + (fourth_slot_end - fourth_slot_begin)), }; { diff --git a/src/llfs/slot_parse.cpp b/src/llfs/slot_parse.cpp index e9718d1..98e7205 100644 --- a/src/llfs/slot_parse.cpp +++ b/src/llfs/slot_parse.cpp @@ -19,7 +19,7 @@ std::ostream& operator<<(std::ostream& out, const SlotParse& t) { return out << "SlotParse{.offset=" << t.offset << ", .slot_size=" << t.offset.size() << ", .body=" << batt::c_str_literal(t.body) << ", .body_size=" << t.body.size() - << ", .depends_on=" << t.depends_on_offset << ",}"; + << ", .total_grant_spent=" << t.total_grant_spent << ",}"; } } // namespace llfs diff --git a/src/llfs/slot_parse.hpp b/src/llfs/slot_parse.hpp index 7e43664..4d736a2 100644 --- a/src/llfs/slot_parse.hpp +++ b/src/llfs/slot_parse.hpp @@ -22,9 +22,28 @@ namespace llfs { //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- // struct SlotParse { + /** \brief The slot offset range of the parsed record. + */ SlotRange offset; + + /** \brief The slot payload data; this excludes only the var-int header portion. + */ std::string_view body; - Optional depends_on_offset; + + /** \brief The total grant size spent to append the logical entity represented by this parse. For + * most records, this is just the number of bytes spanned by the slot (this->offset.size()); for + * commit job events, it includes the `total_grant_spent` of the prepare job slot as well. + */ + u64 total_grant_spent; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns the byte size of the entire parsed slot. + */ + usize size_in_bytes() const noexcept + { + return BATT_CHECKED_CAST(usize, this->offset.size()); + } }; //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -35,7 +54,7 @@ inline bool operator==(const SlotParse& l, const SlotParse& r) { return l.offset == r.offset && // l.body == r.body && // - l.depends_on_offset == r.depends_on_offset; + l.total_grant_spent == r.total_grant_spent; } inline bool operator!=(const SlotParse& l, const SlotParse& r) diff --git a/src/llfs/slot_reader.cpp b/src/llfs/slot_reader.cpp index 4505f99..2b4b6d9 100644 --- a/src/llfs/slot_reader.cpp +++ b/src/llfs/slot_reader.cpp @@ -143,7 +143,7 @@ StatusOr SlotReader::parse_next(batt::WaitForResource wait_for_data) .upper_bound = current_slot + slot_size, }, .body = slot_body, - .depends_on_offset = None, + .total_grant_spent = slot_size, }; } } diff --git a/src/llfs/slot_sequencer.cpp b/src/llfs/slot_sequencer.cpp index cb29706..ce818d2 100644 --- a/src/llfs/slot_sequencer.cpp +++ b/src/llfs/slot_sequencer.cpp @@ -60,6 +60,7 @@ StatusOr SlotSequencer::await_prev() const if (this->prev_ == nullptr) { return SlotRange{0, 0}; } + BATT_DEBUG_INFO(this->debug_info()); return this->prev_->await(); } @@ -71,6 +72,8 @@ StatusOr SlotSequencer::await_current() const if (this->current_ == nullptr) { return {batt::StatusCode::kUnavailable}; } + BATT_DEBUG_INFO(this->debug_info()); + return this->current_->await(); } @@ -95,7 +98,7 @@ bool SlotSequencer::set_current(const SlotRange& slot_range) // bool SlotSequencer::set_error(batt::Status status) { - return this->current_->set_value(std::move(status)); + return this->current_->set_error(std::move(status)); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -113,8 +116,17 @@ SlotSequencer SlotSequencer::get_next() const std::function SlotSequencer::debug_info() const { return [this](std::ostream& out) { - out << "SlotSequencer{.prev_=" << (const void*)this->prev_.get() << ", .current=_" - << (const void*)this->current_.get() << ",}"; + out << "SlotSequencer{.prev_=" << (const void*)this->prev_.get(); + if (this->prev_) { + out << "(ready=" << this->prev_->is_ready() << ", value=" << this->prev_->poll() << ")"; + } + + out << ", .current_=" << (const void*)this->current_.get(); + if (this->current_) { + out << "(ready=" << this->current_->is_ready() << ", value=" << this->current_->poll() << ")"; + } + + out << ",}"; }; } diff --git a/src/llfs/slot_writer.hpp b/src/llfs/slot_writer.hpp index 0b098ae..476d0a5 100644 --- a/src/llfs/slot_writer.hpp +++ b/src/llfs/slot_writer.hpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -188,7 +189,7 @@ class SlotWriter::Append DataPacker packer_; }; -inline usize packed_sizeof_slot_with_payload_size(usize payload_size) +inline constexpr usize packed_sizeof_slot_with_payload_size(usize payload_size) { const usize slot_body_size = sizeof(PackedVariant<>) + payload_size; const usize slot_header_size = packed_sizeof_varint(slot_body_size); @@ -233,34 +234,64 @@ class TypedSlotWriter> : public SlotWriter * mutex; must return the passed slot_range (which is the interval where * `payload` was written) * - * \return The slot offset range where `payload` was appended in the log + * \return The SlotParse and pointer to packed variant case. */ - template - StatusOr append(batt::Grant& caller_grant, T&& payload, - PostCommitFn&& post_commit_fn = {}) + template , + typename PostCommitFn = NullPostCommitFn> + StatusOr> typed_append(batt::Grant& caller_grant, + T&& payload, + PostCommitFn&& post_commit_fn = {}) { const usize slot_body_size = sizeof(PackedVariant) + packed_sizeof(payload); BATT_CHECK_NE(slot_body_size, 0u); - // lock the writer in SlotWriter::prepare + // Lock the writer in SlotWriter::prepare. + // StatusOr op = this->SlotWriter::prepare(caller_grant, slot_body_size); BATT_REQUIRE_OK(op); - // Do allocation for the buffer to write the variant-ID + // Do allocation for the buffer to write the variant-ID. + // PackedVariant* variant_head = op->packer().pack_record(batt::StaticType>{}); if (!variant_head) { return ::llfs::make_status(StatusCode::kFailedToPackSlotVarHead); } - // Get variant-ID ('which') for this entry - variant_head->init(batt::StaticType>{}); + // Get variant-ID ('which') for this entry. + // + variant_head->init(batt::StaticType{}); if (!pack_object(BATT_FORWARD(payload), &(op->packer()))) { return ::llfs::make_status(StatusCode::kFailedToPackSlotVarTail); } - return post_commit_fn(op->commit()); + StatusOr slot_range = post_commit_fn(op->commit()); + BATT_REQUIRE_OK(slot_range); + + auto* slot_body_start = reinterpret_cast(variant_head); + + return SlotParseWithPayload{ + .slot = + SlotParse{ + .offset = *slot_range, + .body = std::string_view{slot_body_start, slot_body_size}, + .total_grant_spent = slot_body_size, + }, + .payload = variant_head->as(batt::StaticType{}), + }; + } + + template + StatusOr append(batt::Grant& caller_grant, T&& payload, + PostCommitFn&& post_commit_fn = {}) + { + StatusOr*>> packed = + this->typed_append(caller_grant, BATT_FORWARD(payload), BATT_FORWARD(post_commit_fn)); + + BATT_REQUIRE_OK(packed); + + return {packed->slot.offset}; } }; diff --git a/src/llfs/status_code.cpp b/src/llfs/status_code.cpp index f0b7c7d..395762b 100644 --- a/src/llfs/status_code.cpp +++ b/src/llfs/status_code.cpp @@ -124,6 +124,13 @@ bool initialize_status_codes() "Failed to attach to PageAllocator: no more available attachments"), // 58, CODE_WITH_MSG_(StatusCode::kPageAllocatorNotAttached, "Client not attached to the PageAllocator"), // 59, + CODE_WITH_MSG_(StatusCode::kPageReaderConflict, + "Conflicting PageReaders registered for the same layout id"), // 60, + CODE_WITH_MSG_( + StatusCode::kPageHeaderBadLayoutId, + "PackedPageHeader::layout_id does not match PageView::get_page_layout_id()"), // 61, + CODE_WITH_MSG_(StatusCode::kPutViewUnknownLayoutId, + "PageCache::put_view failed; page layout id is not registered"), // 62, }); return initialized; } diff --git a/src/llfs/status_code.hpp b/src/llfs/status_code.hpp index d48db90..aa7a1f9 100644 --- a/src/llfs/status_code.hpp +++ b/src/llfs/status_code.hpp @@ -75,6 +75,9 @@ enum struct StatusCode { kStorageFileBadConfigBlockCrc = 57, kOutOfAttachments = 58, kPageAllocatorNotAttached = 59, + kPageReaderConflict = 60, + kPageHeaderBadLayoutId = 61, + kPutViewUnknownLayoutId = 62, }; bool initialize_status_codes(); diff --git a/src/llfs/storage_simulation.cpp b/src/llfs/storage_simulation.cpp index ee84eda..13b7cc3 100644 --- a/src/llfs/storage_simulation.cpp +++ b/src/llfs/storage_simulation.cpp @@ -287,10 +287,14 @@ void StorageSimulation::add_page_arena(PageCount page_count, PageSize page_size) //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -void StorageSimulation::register_page_layout(const PageLayoutId& layout_id, - const PageReader& reader) +void StorageSimulation::register_page_reader(const PageLayoutId& layout_id, const char* file, + int line, const PageReader& reader) { - this->page_layouts_.emplace(layout_id, reader); + this->page_readers_.emplace(layout_id, PageCache::PageReaderFromFile{ + .page_reader = reader, + .file = file, + .line = line, + }); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -322,9 +326,11 @@ const batt::SharedPtr& StorageSimulation::init_cache() noexcept // Register all page layouts for this simulation. // - for (const auto& [layout_id, reader] : this->page_layouts_) { - this->log_event("registering page layout: ", layout_id); - this->cache_->register_page_layout(layout_id, reader); + for (const auto& [layout_id, reader] : this->page_readers_) { + this->log_event("registering page reader: ", layout_id); + BATT_CHECK_OK(this->cache_->register_page_reader(layout_id, reader.file, reader.line, + reader.page_reader)) + << BATT_INSPECT(reader.file) << BATT_INSPECT(reader.line); } } return this->cache_; diff --git a/src/llfs/storage_simulation.hpp b/src/llfs/storage_simulation.hpp index c8cd19f..d26fe90 100644 --- a/src/llfs/storage_simulation.hpp +++ b/src/llfs/storage_simulation.hpp @@ -203,7 +203,8 @@ class StorageSimulation /** \brief Registers the given page format with all future instances of PageCache created by the * simulation. */ - void register_page_layout(const PageLayoutId& layout_id, const PageReader& reader); + void register_page_reader(const PageLayoutId& layout_id, const char* file, int line, + const PageReader& reader); /** \brief Creates/accesses a simulated Volume. * @@ -301,9 +302,9 @@ class StorageSimulation // std::atomic counter_{0}; - // Page layouts that should be registered with PageCache instances on each recovery. + // Page readers that should be registered with PageCache instances on each recovery. // - std::unordered_map page_layouts_; + std::unordered_map page_readers_; }; } //namespace llfs diff --git a/src/llfs/testing/util.cpp b/src/llfs/testing/util.cpp new file mode 100644 index 0000000..988dad0 --- /dev/null +++ b/src/llfs/testing/util.cpp @@ -0,0 +1,57 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +namespace llfs { +namespace testing { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +const std::filesystem::path& get_project_dir() +{ + static const std::filesystem::path cached = []() { + const char* project_dir_env = std::getenv("PROJECT_DIR"); + if (project_dir_env) { + return std::filesystem::path{project_dir_env}; + } + + std::filesystem::path file_path{__FILE__}; + if (std::filesystem::exists(file_path)) { + std::filesystem::path testing_dir_path = file_path.parent_path(); + if (std::filesystem::exists(testing_dir_path)) { + std::filesystem::path src_llfs_dir_path = testing_dir_path.parent_path(); + if (std::filesystem::exists(src_llfs_dir_path)) { + std::filesystem::path src_dir_path = src_llfs_dir_path.parent_path(); + if (std::filesystem::exists(src_dir_path)) { + std::filesystem::path project_dir_path = src_dir_path.parent_path(); + if (std::filesystem::exists(project_dir_path)) { + return project_dir_path; + } + } + } + } + } + + BATT_PANIC() << "Could not find project root; make sure PROJECT_DIR is set!"; + BATT_UNREACHABLE(); + }(); + + return cached; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +std::filesystem::path get_test_data_file_path(std::string_view file_rel_path) +{ + return get_project_dir() / "testdata" / file_rel_path; +} + +} //namespace testing +} //namespace llfs diff --git a/src/llfs/testing/util.hpp b/src/llfs/testing/util.hpp new file mode 100644 index 0000000..1c237f0 --- /dev/null +++ b/src/llfs/testing/util.hpp @@ -0,0 +1,39 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_TESTING_UTIL_HPP +#define LLFS_TESTING_UTIL_HPP + +#include +// +#include + +#include +#include +#include + +namespace llfs { +namespace testing { + +/** \brief Returns the source project root directory. This is the top-level GIT repo dir. + * + * First the environment variable PROJECT_DIR is checked; if this fails, then this function attempts + * to locate this source file using __FILE__ and work backwards from there. + */ +const std::filesystem::path& get_project_dir(); + +/** \brief Returns the full path to the specified test data file, specified as a relative path with + * the testdata/ dir as its base. + */ +std::filesystem::path get_test_data_file_path(std::string_view file_rel_path); + +} //namespace testing +} //namespace llfs + +#endif // LLFS_TESTING_UTIL_HPP diff --git a/src/llfs/trie.test.cpp b/src/llfs/trie.test.cpp index 525c7c2..cb9a6d1 100644 --- a/src/llfs/trie.test.cpp +++ b/src/llfs/trie.test.cpp @@ -5,6 +5,8 @@ #include #include +#include + #include #include #include @@ -42,7 +44,9 @@ using llfs::PackedBPTrie; std::vector load_words() { std::vector words; - std::ifstream ifs{batt::to_string(std::getenv("PROJECT_DIR"), "/testdata/words")}; + std::string word_file_path = llfs::testing::get_test_data_file_path("words"); + std::ifstream ifs{word_file_path}; + BATT_CHECK(ifs.good()) << BATT_INSPECT_STR(word_file_path); std::string word; while (ifs.good()) { ifs >> word; diff --git a/src/llfs/volume.cpp b/src/llfs/volume.cpp index 9b6557b..8ddb627 100644 --- a/src/llfs/volume.cpp +++ b/src/llfs/volume.cpp @@ -10,12 +10,15 @@ // #include +#include +#include #include -#include +#include #include namespace llfs { + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // const VolumeOptions& Volume::options() const @@ -41,7 +44,7 @@ const boost::uuids::uuid& Volume::get_recycler_uuid() const // const boost::uuids::uuid& Volume::get_trimmer_uuid() const { - return this->trimmer_.uuid(); + return this->trimmer_->uuid(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -59,14 +62,7 @@ u64 Volume::calculate_grant_size(const std::string_view& payload) const // u64 Volume::calculate_grant_size(const AppendableJob& appendable) const { - return (packed_sizeof_slot(prepare(appendable)) + - packed_sizeof_slot(batt::StaticType{})) - - // We double the grant size to reserve log space to save the list of pages (and the record - // of a job being committed) in TrimEvent slots, should prepare and commit be split across - // two trimmed regions. - // - * 2; + return appendable.calculate_grant_size(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -100,118 +96,83 @@ u64 Volume::calculate_grant_size(const AppendableJob& appendable) const LLFS_VLOG(1) << "PageRecycler recovered"; - VolumePendingJobsMap pending_jobs; - VolumeRecoveryVisitor visitor{batt::make_copy(slot_visitor_fn), pending_jobs}; - Optional trimmer_visitor; + VolumeMetadata metadata; + + VolumeJobRecoveryVisitor job_visitor; + VolumeMetadataRecoveryVisitor metadata_visitor{metadata}; + VolumeSlotDemuxer user_fn_visitor{batt::make_copy(slot_visitor_fn)}; // Open the log device and scan all slots. // BATT_ASSIGN_OK_RESULT(std::unique_ptr root_log, root_log_factory.open_log_device( [&](LogDevice::Reader& log_reader) -> StatusOr { - trimmer_visitor.emplace(/*trim_pos=*/log_reader.slot_offset()); - TypedSlotReader slot_reader{log_reader}; StatusOr slots_read = slot_reader.run( - batt::WaitForResource::kFalse, - [&visitor, &trimmer_visitor](auto&&... args) -> Status { - BATT_REQUIRE_OK(visitor(args...)); - BATT_REQUIRE_OK((*trimmer_visitor)(args...)); - + batt::WaitForResource::kFalse, [&](auto&&... args) -> Status { + BATT_REQUIRE_OK(job_visitor(args...)); + BATT_REQUIRE_OK(metadata_visitor(args...)); + BATT_REQUIRE_OK(user_fn_visitor(args...)); return batt::OkStatus(); }); - BATT_UNTESTED_COND(!slots_read.ok()); + BATT_REQUIRE_OK(slots_read); return log_reader.slot_offset(); })); - // The amount to allocate to the trimmer to refresh all metadata we append to the log below. + // Create a slot writer to complete recovery. // - usize trimmer_grant_size = 0; - - // Put the main log in a clean state. This means all configuration data must - // be recorded, device attachments created, and pending jobs resolved. - { - TypedSlotWriter slot_writer{*root_log}; - batt::Grant grant = BATT_OK_RESULT_OR_PANIC( - slot_writer.reserve(slot_writer.pool_size(), batt::WaitForResource::kFalse)); + auto slot_writer = std::make_unique>(*root_log); - // If no uuids were found while opening the log, create them now. - // - if (!visitor.ids) { - LLFS_VLOG(1) << "Initializing Volume uuids for the first time"; - - visitor.ids.emplace(SlotWithPayload{ - .slot_range = {0, 1}, - .payload = - { - .main_uuid = volume_options.uuid.value_or(boost::uuids::random_generator{}()), - .recycler_uuid = recycler->uuid(), - .trimmer_uuid = boost::uuids::random_generator{}(), - }, - }); - - StatusOr ids_slot = slot_writer.append(grant, visitor.ids->payload); - - BATT_UNTESTED_COND(!ids_slot.ok()); - BATT_REQUIRE_OK(ids_slot); + // If no uuids were found while opening the log, create them now. + // + if (!metadata.ids) { + LLFS_VLOG(1) << "Initializing Volume uuids for the first time"; + + metadata.ids.emplace(PackedVolumeIds{ + .main_uuid = volume_options.uuid.value_or(boost::uuids::random_generator{}()), + .recycler_uuid = recycler->uuid(), + .trimmer_uuid = boost::uuids::random_generator{}(), + }); + } - Status flush_status = - slot_writer.sync(LogReadMode::kDurable, SlotUpperBoundAt{ids_slot->upper_bound}); + // Make sure all Volume metadata is up-to-date. + // + auto metadata_refresher = + std::make_unique(*slot_writer, batt::make_copy(metadata)); + { + for (const boost::uuids::uuid& uuid : { + metadata.ids->main_uuid, + metadata.ids->recycler_uuid, + metadata.ids->trimmer_uuid, + }) { + for (const PageArena& arena : cache->all_arenas()) { + Optional attachment = + arena.allocator().get_client_attachment_status(uuid); - BATT_UNTESTED_COND(!flush_status.ok()); - BATT_REQUIRE_OK(flush_status); - } - LLFS_VLOG(1) << BATT_INSPECT(visitor.ids->payload); + // If already attached, then nothing to do; continue. + // + if (attachment) { + continue; + } - // Attach the main uuid, recycler uuid, and trimmer uuid to each device in - // the cache storage pool. - // - { - // Loop through all combinations of uuid, device_id. - // - LLFS_VLOG(1) << "Recovered attachments: " << batt::dump_range(visitor.device_attachments); - - for (const auto& uuid : { - visitor.ids->payload.main_uuid, - visitor.ids->payload.recycler_uuid, - visitor.ids->payload.trimmer_uuid, - }) { - for (const PageArena& arena : cache->all_arenas()) { - Optional attachment = - arena.allocator().get_client_attachment_status(uuid); - - // Find the lowest available slot offset for the log associated with `uuid`. - // - const slot_offset_type next_available_slot_offset = [&] { - if (uuid == visitor.ids->payload.recycler_uuid) { - return recycler->slot_upper_bound(LogReadMode::kDurable); - } else { - // Both the volume (main) and trimmer share the same WAL (the main log). - // - return slot_writer.slot_offset(); - } - }(); - - auto attach_event = PackedVolumeAttachEvent{{ - .id = - VolumeAttachmentId{ - .client = uuid, - .device = arena.device().get_id(), - }, - .user_slot_offset = next_available_slot_offset, - }}; - - trimmer_grant_size += packed_sizeof_slot(attach_event); - - // If already attached, then nothing to do; continue. - // - if (attachment || visitor.device_attachments.count(attach_event.id)) { - continue; + // Find the lowest available slot offset for the log associated with `uuid`. + // + const slot_offset_type next_available_slot_offset = [&] { + if (uuid == metadata.ids->recycler_uuid) { + return recycler->slot_upper_bound(LogReadMode::kDurable); + } else { + // Both the volume (main) and trimmer share the same WAL (the main log). + // + return slot_writer->slot_offset(); } + }(); + //----- --- -- - - - - + // Attach to this arena's PageAllocator. + { LLFS_VLOG(1) << "[Volume::recover] attaching client " << uuid << " to device " << arena.device().get_id() << BATT_INSPECT(next_available_slot_offset); @@ -224,29 +185,68 @@ u64 Volume::calculate_grant_size(const AppendableJob& appendable) const Status sync_status = arena.allocator().sync(*sync_slot); BATT_UNTESTED_COND(!sync_status.ok()); - BATT_REQUIRE_OK(sync_status); + } + //----- --- -- - - - - + + // Add the attachment to the metadata refresher; we will append a new slot when we call + // flush below. + // + BATT_ASSIGN_OK_RESULT(batt::Grant attach_grant, + slot_writer->reserve(VolumeMetadataRefresher::kAttachmentGrantSize, + batt::WaitForResource::kFalse)); + + BATT_REQUIRE_OK(metadata_refresher->add_attachment( + VolumeAttachmentId{ + .client = uuid, + .device = arena.device().get_id(), + }, + next_available_slot_offset, attach_grant)); + } + } - StatusOr ids_slot = slot_writer.append(grant, attach_event); + //----- --- -- - - - - + // Flush any non-durable metadata to the log. + // + if (metadata_refresher->needs_flush()) { + batt::Grant initial_metadata_flush_grant = BATT_OK_RESULT_OR_PANIC(slot_writer->reserve( + metadata_refresher->flush_grant_size(), batt::WaitForResource::kFalse)); - BATT_UNTESTED_COND(!ids_slot.ok()); - BATT_REQUIRE_OK(ids_slot); + BATT_REQUIRE_OK(metadata_refresher->update_grant_partial(initial_metadata_flush_grant)); + BATT_REQUIRE_OK(metadata_refresher->flush()); + } - Status flush_status = - slot_writer.sync(LogReadMode::kDurable, SlotUpperBoundAt{ids_slot->upper_bound}); + { + const usize reclaimable_size = metadata_visitor.grant_byte_size_reclaimable_on_trim(); - BATT_UNTESTED_COND(!flush_status.ok()); - BATT_REQUIRE_OK(flush_status); + const usize initial_refresh_grant_size = [&]() -> usize { + if (metadata_refresher->grant_required() < reclaimable_size) { + return 0; } - } - - LLFS_VLOG(1) << "Page devices attached"; + return metadata_refresher->grant_required() - reclaimable_size; + }(); + + LLFS_VLOG(1) << "Reserving grant for metadata refresher;" + << BATT_INSPECT(initial_refresh_grant_size) + << BATT_INSPECT(metadata_refresher->grant_target()) + << BATT_INSPECT(metadata_refresher->grant_size()) + << BATT_INSPECT(metadata_refresher->grant_required()) + << BATT_INSPECT(VolumeMetadata::kVolumeIdsGrantSize) + << BATT_INSPECT(VolumeMetadata::kAttachmentGrantSize) + << BATT_INSPECT(reclaimable_size); + + batt::Grant initial_metadata_refresh_grant = BATT_OK_RESULT_OR_PANIC( + slot_writer->reserve(initial_refresh_grant_size, batt::WaitForResource::kFalse)); + + BATT_REQUIRE_OK(metadata_refresher->update_grant_partial(initial_metadata_refresh_grant)); } + } + { // Resolve any jobs with a PrepareJob slot but no CommitJob or RollbackJob. // LLFS_VLOG(1) << "Resolving pending jobs..."; - Status jobs_resolved = visitor.resolve_pending_jobs( - *cache, *recycler, /*volume_uuid=*/visitor.ids->payload.main_uuid, slot_writer, grant); + Status jobs_resolved = job_visitor.resolve_pending_jobs( + *cache, *recycler, /*volume_uuid=*/metadata.ids->main_uuid, *slot_writer); BATT_REQUIRE_OK(jobs_resolved); @@ -254,10 +254,10 @@ u64 Volume::calculate_grant_size(const AppendableJob& appendable) const // Notify all PageAllocators that we are done with recovery. // - for (const auto& uuid : { - visitor.ids->payload.main_uuid, - visitor.ids->payload.recycler_uuid, - visitor.ids->payload.trimmer_uuid, + for (const boost::uuids::uuid& uuid : { + metadata.ids->main_uuid, + metadata.ids->recycler_uuid, + metadata.ids->trimmer_uuid, }) { for (const PageArena& arena : cache->all_arenas()) { BATT_REQUIRE_OK(arena.allocator().notify_user_recovered(uuid)); @@ -265,20 +265,35 @@ u64 Volume::calculate_grant_size(const AppendableJob& appendable) const } } - // Create the Volume object. + // Recover the VolumeTrimmer state, resolving any pending log trim. // - std::unique_ptr volume{ - new Volume{scheduler, params.options, visitor.ids->payload.main_uuid, std::move(cache), - std::move(params.trim_control), std::move(page_deleter), std::move(root_log), - std::move(recycler), visitor.ids->payload.trimmer_uuid, *trimmer_visitor}}; + BATT_ASSIGN_OK_RESULT(std::unique_ptr trimmer, + VolumeTrimmer::recover( // + metadata.ids->trimmer_uuid, // + batt::to_string(params.options.name, "_Trimmer"), // + params.options.trim_delay_byte_count, // + *root_log, // + *slot_writer, // + VolumeTrimmer::make_default_drop_roots_fn(*cache, *recycler), // + *params.trim_control, // + *metadata_refresher // + )); - { - batt::StatusOr trimmer_grant = - volume->slot_writer_.reserve(trimmer_grant_size, batt::WaitForResource::kFalse); - BATT_REQUIRE_OK(trimmer_grant); - - volume->trimmer_.push_grant(std::move(*trimmer_grant)); - } + // Create the Volume object. + // + std::unique_ptr volume{new Volume{ + scheduler, + params.options, + metadata.ids->main_uuid, + std::move(cache), + std::move(params.trim_control), + std::move(page_deleter), + std::move(root_log), + std::move(recycler), + std::move(slot_writer), + std::move(metadata_refresher), + std::move(trimmer), + }}; volume->start(); @@ -287,15 +302,17 @@ u64 Volume::calculate_grant_size(const AppendableJob& appendable) const //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -/*explicit*/ Volume::Volume(batt::TaskScheduler& task_scheduler, const VolumeOptions& options, - const boost::uuids::uuid& volume_uuid, - batt::SharedPtr&& page_cache, - std::shared_ptr&& trim_control, - std::unique_ptr&& page_deleter, - std::unique_ptr&& root_log, - std::unique_ptr&& recycler, - const boost::uuids::uuid& trimmer_uuid, - const VolumeTrimmer::RecoveryVisitor& trimmer_recovery_visitor) noexcept +/*explicit*/ Volume::Volume(batt::TaskScheduler& task_scheduler, // + const VolumeOptions& options, // + const boost::uuids::uuid& volume_uuid, // + batt::SharedPtr&& page_cache, // + std::shared_ptr&& trim_control, // + std::unique_ptr&& page_deleter, // + std::unique_ptr&& root_log, // + std::unique_ptr&& recycler, // + std::unique_ptr>&& slot_writer, // + std::unique_ptr&& metadata_refresher, // + std::unique_ptr&& trimmer) noexcept : task_scheduler_{task_scheduler} , options_{options} , volume_uuid_{volume_uuid} @@ -306,16 +323,9 @@ u64 Volume::calculate_grant_size(const AppendableJob& appendable) const , trim_lock_{BATT_OK_RESULT_OR_PANIC(this->trim_control_->lock_slots( this->root_log_->slot_range(LogReadMode::kDurable), "Volume::(ctor)"))} , recycler_{std::move(recycler)} - , slot_writer_{*this->root_log_} - , trimmer_{ - trimmer_uuid, - batt::to_string(this->options_.name, "_Trimmer"), - *this->trim_control_, - this->options_.trim_delay_byte_count, - this->root_log_->new_reader(/*slot_lower_bound=*/None, LogReadMode::kDurable), - this->slot_writer_, - VolumeTrimmer::make_default_drop_roots_fn(this->cache(), *this->recycler_, trimmer_uuid), - trimmer_recovery_visitor} + , slot_writer_{std::move(slot_writer)} + , metadata_refresher_{std::move(metadata_refresher)} + , trimmer_{std::move(trimmer)} { } @@ -340,7 +350,7 @@ void Volume::start() this->trimmer_task_.emplace( /*executor=*/this->task_scheduler_.schedule_task(), [this] { - Status result = this->trimmer_.run(); + Status result = this->trimmer_->run(); LLFS_VLOG(1) << "Volume::trimmer_task_ exited with status=" << result; }, "Volume::trimmer_task_"); @@ -351,9 +361,9 @@ void Volume::start() // void Volume::halt() { - this->slot_writer_.halt(); + this->slot_writer_->halt(); this->trim_control_->halt(); - this->trimmer_.halt(); + this->trimmer_->halt(); this->root_log_->close().IgnoreError(); if (this->recycler_) { this->recycler_->halt(); @@ -428,7 +438,7 @@ StatusOr Volume::lock_slots(const SlotRange& slot_range, LogReadMo // StatusOr Volume::reserve(u64 size, batt::WaitForResource wait_for_log_space) { - return this->slot_writer_.reserve(size, wait_for_log_space); + return this->slot_writer_->reserve(size, wait_for_log_space); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -447,124 +457,154 @@ StatusOr Volume::append(const std::string_view& payload, batt::Grant& StatusOr Volume::append(AppendableJob&& appendable, batt::Grant& grant, Optional&& sequencer) { - const auto check_sequencer_is_resolved = batt::finally([&sequencer] { + StatusOr result; + + const auto check_sequencer_is_resolved = batt::finally([&sequencer, &result] { BATT_CHECK_IMPLIES(bool{sequencer}, sequencer->is_resolved()) << "If a SlotSequencer is passed, it must be resolved even on failure " - "paths."; + "paths." + << BATT_INSPECT(result); }); - //+++++++++++-+-+--+----- --- -- - - - - - // Phase 0: Wait for the previous slot in the sequence to be appended to the - // log. - // - if (sequencer) { - BATT_DEBUG_INFO("awaiting previous slot in sequence; " - << BATT_INSPECT(sequencer->has_prev()) << BATT_INSPECT(sequencer->poll_prev()) - << BATT_INSPECT(sequencer->get_current()) << sequencer->debug_info()); - - StatusOr prev_slot = sequencer->await_prev(); - if (!prev_slot.ok()) { - sequencer->set_error(prev_slot.status()); + result = [&]() -> StatusOr { + //+++++++++++-+-+--+----- --- -- - - - - + // Phase 0: Wait for the previous slot in the sequence to be appended to the + // log. + // + if (sequencer) { + BATT_DEBUG_INFO("awaiting previous slot in sequence; " + << BATT_INSPECT(sequencer->has_prev()) << BATT_INSPECT(sequencer->poll_prev()) + << BATT_INSPECT(sequencer->get_current()) << sequencer->debug_info()); + + StatusOr prev_slot = sequencer->await_prev(); + if (!prev_slot.ok()) { + sequencer->set_error(prev_slot.status()); + } + BATT_REQUIRE_OK(prev_slot); + + // We only need to do a speculative sync here, because flushing later slots + // in the log implies that all earlier ones are flushed, and we are going to + // do a durable sync (flush) for our prepare event below. + // + BATT_DEBUG_INFO("awaiting flush of previous slot: " << *prev_slot); + + Status sync_prev = this->slot_writer_->sync(LogReadMode::kSpeculative, + SlotUpperBoundAt{prev_slot->upper_bound}); + if (!sync_prev.ok()) { + sequencer->set_error(sync_prev); + } + BATT_REQUIRE_OK(sync_prev); } - BATT_REQUIRE_OK(prev_slot); - // We only need to do a speculative sync here, because flushing later slots - // in the log implies that all earlier ones are flushed, and we are going to - // do a durable sync (flush) for our prepare event below. + //+++++++++++-+-+--+----- --- -- - - - - + // Phase 1: Write a prepare slot to the write-ahead log and flush it to + // durable storage. // - BATT_DEBUG_INFO("awaiting flush of previous slot: " << *prev_slot); + BATT_DEBUG_INFO("appending PrepareJob slot to the WAL"); - Status sync_prev = this->slot_writer_.sync(LogReadMode::kSpeculative, - SlotUpperBoundAt{prev_slot->upper_bound}); - if (!sync_prev.ok()) { - sequencer->set_error(sync_prev); - } - BATT_REQUIRE_OK(sync_prev); - } + PrepareJob prepared_job = prepare(appendable); + const u64 prepared_job_size = packed_sizeof_slot(prepared_job); - //+++++++++++-+-+--+----- --- -- - - - - - // Phase 1: Write a prepare slot to the write-ahead log and flush it to - // durable storage. - // - BATT_DEBUG_INFO("appending PrepareJob slot to the WAL"); + Optional prev_user_slot; - auto prepared_job = prepare(appendable); - { - BATT_ASSIGN_OK_RESULT(batt::Grant trim_refresh_grant, - grant.spend(packed_sizeof_slot(prepared_job))); - this->trimmer_.push_grant(std::move(trim_refresh_grant)); - } + const auto grant_size_before_prepare = grant.size(); - Optional prev_user_slot; - - StatusOr prepare_slot = LLFS_COLLECT_LATENCY( - this->metrics_.prepare_slot_append_latency, - this->slot_writer_.append( - grant, std::move(prepared_job), [this, &prev_user_slot](StatusOr slot_range) { - if (slot_range.ok()) { - prev_user_slot = this->latest_user_slot_.exchange(slot_range->lower_bound); - } - return slot_range; - })); - - if (sequencer) { - if (!prepare_slot.ok()) { - BATT_CHECK(sequencer->set_error(prepare_slot.status())) - << "each slot within a sequence may only be set once!"; - } else { - BATT_CHECK(sequencer->set_current(*prepare_slot)) - << "each slot within a sequence may only be set once!"; + // Acquired in the post-commit-fn of the prepare slot; prevents the trimmer from having to + // deal with unresolved jobs. + // + Optional trim_lock; + + // Append the prepare! + // + StatusOr> prepare_slot = LLFS_COLLECT_LATENCY( + this->metrics_.prepare_slot_append_latency, + this->slot_writer_->typed_append( + grant, std::move(prepared_job), + [this, &prev_user_slot, &trim_lock](StatusOr slot_range) { + if (slot_range.ok()) { + // Set the prev_user_slot. + // + trim_lock.emplace(BATT_OK_RESULT_OR_PANIC( + this->trim_control_->lock_slots(*slot_range, "Volume::append(job)"))); + + // Acquire a read lock to prevent premature trimming. + // + prev_user_slot = this->latest_user_slot_.exchange(slot_range->lower_bound); + } + return slot_range; + })); + + const auto grant_size_after_prepare = grant.size(); + + BATT_CHECK_EQ(prepared_job_size, grant_size_before_prepare - grant_size_after_prepare); + + if (sequencer) { + if (!prepare_slot.ok()) { + BATT_CHECK(sequencer->set_error(prepare_slot.status())) + << "each slot within a sequence may only be set once!"; + } else { + BATT_CHECK(sequencer->set_current(prepare_slot->slot.offset)) + << "each slot within a sequence may only be set once!"; + } } - } - BATT_REQUIRE_OK(prepare_slot); - BATT_CHECK(prev_user_slot); - BATT_CHECK(slot_at_most(*prev_user_slot, prepare_slot->lower_bound)) - << BATT_INSPECT(prev_user_slot) << BATT_INSPECT(prepare_slot); - BATT_DEBUG_INFO("flushing PrepareJob slot to storage"); + BATT_REQUIRE_OK(prepare_slot); - Status sync_prepare = LLFS_COLLECT_LATENCY( - this->metrics_.prepare_slot_sync_latency, - this->slot_writer_.sync(LogReadMode::kDurable, SlotUpperBoundAt{prepare_slot->upper_bound})); + const slot_offset_type prepare_slot_offset = prepare_slot->slot.offset.lower_bound; - BATT_REQUIRE_OK(sync_prepare); + BATT_CHECK(trim_lock); + BATT_CHECK(prev_user_slot); + BATT_CHECK(slot_at_most(*prev_user_slot, prepare_slot_offset)) + << BATT_INSPECT(prev_user_slot) << BATT_INSPECT(prepare_slot); - //+++++++++++-+-+--+----- --- -- - - - - - // Phase 2a: Commit the job; this writes new pages, updates ref counts, and - // deletes dropped pages. - // - BATT_DEBUG_INFO("committing PageCacheJob"); + BATT_DEBUG_INFO("flushing PrepareJob slot to storage"); - const JobCommitParams params{ - .caller_uuid = &this->get_volume_uuid(), - .caller_slot = prepare_slot->lower_bound, - .recycler = as_ref(*this->recycler_), - .recycle_grant = nullptr, - .recycle_depth = -1, - }; + Status sync_prepare = LLFS_COLLECT_LATENCY( + this->metrics_.prepare_slot_sync_latency, + this->slot_writer_->sync(LogReadMode::kDurable, + SlotUpperBoundAt{prepare_slot->slot.offset.upper_bound})); - Status commit_job_result = commit(std::move(appendable.job), params, Caller::Unknown, - *prev_user_slot, &this->durable_user_slot_); + BATT_REQUIRE_OK(sync_prepare); - BATT_REQUIRE_OK(commit_job_result); + //+++++++++++-+-+--+----- --- -- - - - - + // Phase 2a: Commit the job; this writes new pages, updates ref counts, and + // deletes dropped pages. + // + BATT_DEBUG_INFO("committing PageCacheJob"); - //+++++++++++-+-+--+----- --- -- - - - - - // Phase 2b: Write the commit slot. - // - BATT_DEBUG_INFO("writing commit slot"); + const JobCommitParams params{ + .caller_uuid = &this->get_volume_uuid(), + .caller_slot = prepare_slot_offset, + .recycler = as_ref(*this->recycler_), + .recycle_grant = nullptr, + .recycle_depth = -1, + }; + + Status commit_job_result = commit(std::move(appendable.job), params, Caller::Unknown, + *prev_user_slot, &this->durable_user_slot_); + + BATT_REQUIRE_OK(commit_job_result); - StatusOr commit_slot = - this->slot_writer_.append(grant, PackedCommitJob{ - .reserved_ = {}, - .prepare_slot = prepare_slot->lower_bound, - }); + //+++++++++++-+-+--+----- --- -- - - - - + // Phase 2b: Write the commit slot. + // + BATT_DEBUG_INFO("writing commit slot"); + + StatusOr commit_slot = + this->slot_writer_->append(grant, CommitJob{ + .prepare_slot_offset = prepare_slot_offset, + .packed_prepare = prepare_slot->payload, + }); - BATT_REQUIRE_OK(commit_slot); + BATT_REQUIRE_OK(commit_slot); + + return SlotRange{ + .lower_bound = prepare_slot->slot.offset.lower_bound, + .upper_bound = commit_slot->upper_bound, + }; + }(); - return SlotRange{ - .lower_bound = prepare_slot->lower_bound, - .upper_bound = commit_slot->upper_bound, - }; + return result; } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -620,14 +660,14 @@ struct PrepareJob_must_be_passed_to_Volume_append_by_move* Volume::append(const // Status Volume::await_trim(slot_offset_type slot_lower_bound) { - return this->slot_writer_.await_trim(slot_lower_bound); + return this->slot_writer_->await_trim(slot_lower_bound); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // slot_offset_type Volume::get_trim_pos() const noexcept { - return this->slot_writer_.get_trim_pos(); + return this->slot_writer_->get_trim_pos(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - diff --git a/src/llfs/volume.hpp b/src/llfs/volume.hpp index f4ca06e..a277ff6 100644 --- a/src/llfs/volume.hpp +++ b/src/llfs/volume.hpp @@ -44,6 +44,8 @@ struct VolumeRecoverParams { std::shared_ptr trim_control; }; +class VolumeJobRecoveryVisitor; + //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- // class Volume @@ -196,6 +198,34 @@ class Volume // u64 root_log_capacity() const; + /** \brief The current total number of log bytes available to reserve. + * + * This number goes down when Volume::reserve is called, and goes up with either the Grant objects + * returned by reserve are spent/destructed, or when the log is trimmed. + */ + u64 available_to_reserve() const noexcept + { + return this->slot_writer_->pool_size(); + } + + /** \brief The amount of reserved log space currently held by the VolumeTrimmer for its own + * internal use. + * + * This is exposed for debugging/tuning purposes only (e.g., to detect possible leaks) + */ + u64 trimmer_grant_size() const noexcept + { + return this->trimmer_->grant_pool_size(); + } + + /** \brief The number of successful log trims that have completed since this Volume object was + * created. + */ + u64 trim_count() const noexcept + { + return this->trimmer_->trim_count(); + } + // Returns the current valid slot offset range for the root log at the specified durability level. // SlotRange root_log_slot_range(LogReadMode mode) const; @@ -222,14 +252,19 @@ class Volume return *(this->root_log_); } + //+++++++++++-+-+--+----- --- -- - - - - private: - explicit Volume(batt::TaskScheduler& task_scheduler, const VolumeOptions& options, - const boost::uuids::uuid& volume_uuid, batt::SharedPtr&& page_cache, - std::shared_ptr&& trim_control, - std::unique_ptr&& page_deleter, - std::unique_ptr&& root_log, std::unique_ptr&& recycler, - const boost::uuids::uuid& trimmer_uuid, - const VolumeTrimmer::RecoveryVisitor& trimmer_recovery_visitor) noexcept; + explicit Volume(batt::TaskScheduler& task_scheduler, // + const VolumeOptions& options, // + const boost::uuids::uuid& volume_uuid, // + batt::SharedPtr&& page_cache, // + std::shared_ptr&& trim_control, // + std::unique_ptr&& page_deleter, // + std::unique_ptr&& root_log, // + std::unique_ptr&& recycler, // + std::unique_ptr>&& slot_writer, // + std::unique_ptr&& metadata_refresher, // + std::unique_ptr&& trimmer) noexcept; // Launch background tasks associated with this Volume. // @@ -279,11 +314,16 @@ class Volume // Appends new slots to the root log. // - TypedSlotWriter slot_writer_; + std::unique_ptr> slot_writer_; + + // Used by the VolumeTrimmer to refresh Volume metadata (ids and attachments) when the log is + // trimmed. + // + std::unique_ptr metadata_refresher_; // Refreshes volume config slots and trims the root log. // - VolumeTrimmer trimmer_; + std::unique_ptr trimmer_; // Task that runs `trimmer_` continuously in the background. // diff --git a/src/llfs/volume.ipp b/src/llfs/volume.ipp index 22cf67d..457623d 100644 --- a/src/llfs/volume.ipp +++ b/src/llfs/volume.ipp @@ -38,7 +38,7 @@ StatusOr Volume::append(const T& payload, batt::Grant& grant) { llfs::PackObjectAsRawData packed_obj_as_raw{payload}; - return this->slot_writer_.append(grant, packed_obj_as_raw); + return this->slot_writer_->append(grant, packed_obj_as_raw); } } // namespace llfs diff --git a/src/llfs/volume.test.cpp b/src/llfs/volume.test.cpp index d121cbc..bed3f28 100644 --- a/src/llfs/volume.test.cpp +++ b/src/llfs/volume.test.cpp @@ -150,7 +150,7 @@ class VolumeTest : public ::testing::Test }, // BATT_FORWARD(slot_visitor_fn)); - BATT_CHECK(test_volume_recovered.ok()); + BATT_CHECK(test_volume_recovered.ok()) << BATT_INSPECT(test_volume_recovered.status()); return std::move(*test_volume_recovered); } @@ -200,8 +200,11 @@ class VolumeTest : public ::testing::Test llfs::StatusOr make_opaque_page(llfs::PageCacheJob& job) { - llfs::StatusOr> page_allocated = - job.new_page(llfs::PageSize{256}, batt::WaitForResource::kFalse, llfs::Caller::Unknown); + BATT_REQUIRE_OK(llfs::OpaquePageView::register_layout(job.cache())); + + llfs::StatusOr> page_allocated = job.new_page( + llfs::PageSize{256}, batt::WaitForResource::kFalse, llfs::OpaquePageView::page_layout_id(), + llfs::Caller::Unknown, /*cancel_token=*/llfs::None); BATT_REQUIRE_OK(page_allocated); @@ -1077,8 +1080,6 @@ TEST_F(VolumeTest, PageJobs) [&](const llfs::SlotParse& slot, std::string_view user_data) -> llfs::Status { job_i += 1; - EXPECT_TRUE(slot.depends_on_offset); - LLFS_VLOG(1) << "Visiting slot: " << BATT_INSPECT(slot) << " user_data=" << batt::c_str_literal(user_data); @@ -1501,7 +1502,7 @@ TEST_F(VolumeSimTest, ConcurrentAppendJobs) // sim.add_page_arena(this->pages_per_device, llfs::PageSize{1 * kKiB}); - sim.register_page_layout(llfs::PageGraphNodeView::page_layout_id(), + sim.register_page_reader(llfs::PageGraphNodeView::page_layout_id(), __FILE__, __LINE__, llfs::PageGraphNodeView::page_reader()); const auto main_task_fn = [&] { @@ -1648,7 +1649,7 @@ void VolumeSimTest::run_recovery_sim(u32 seed) sim.add_page_arena(this->pages_per_device, llfs::PageSize{2 * kKiB}); sim.add_page_arena(this->pages_per_device, llfs::PageSize{4 * kKiB}); - sim.register_page_layout(llfs::PageGraphNodeView::page_layout_id(), + sim.register_page_reader(llfs::PageGraphNodeView::page_layout_id(), __FILE__, __LINE__, llfs::PageGraphNodeView::page_reader()); const auto main_task_fn = [&] { @@ -1911,8 +1912,9 @@ batt::StatusOr VolumeSimTest::build_page_with_refs_to( llfs::PageCacheJob& job, llfs::StorageSimulation& /*sim*/) { batt::StatusOr page_builder = - llfs::PageGraphNodeBuilder::from_new_page( - job.new_page(page_size, batt::WaitForResource::kFalse, /*callers=*/0)); + llfs::PageGraphNodeBuilder::from_new_page(job.new_page( + page_size, batt::WaitForResource::kFalse, llfs::PageGraphNodeView::page_layout_id(), + /*callers=*/0, /*cancel_token=*/llfs::None)); BATT_REQUIRE_OK(page_builder); diff --git a/src/llfs/volume_event_visitor.hpp b/src/llfs/volume_event_visitor.hpp index 8ff5723..58f06e2 100644 --- a/src/llfs/volume_event_visitor.hpp +++ b/src/llfs/volume_event_visitor.hpp @@ -26,6 +26,10 @@ template class VolumeEventVisitor { public: + class NullImpl; + + //+++++++++++-+-+--+----- --- -- - - - - + static VolumeEventVisitor& null_impl(); VolumeEventVisitor(const VolumeEventVisitor&) = delete; @@ -46,7 +50,7 @@ class VolumeEventVisitor LLFS_VOLUME_EVENT_HANDLER_DECL(Ref, on_raw_data) LLFS_VOLUME_EVENT_HANDLER_DECL(Ref, on_prepare_job) - LLFS_VOLUME_EVENT_HANDLER_DECL(PackedCommitJob, on_commit_job) + LLFS_VOLUME_EVENT_HANDLER_DECL(Ref, on_commit_job) LLFS_VOLUME_EVENT_HANDLER_DECL(PackedRollbackJob, on_rollback_job) LLFS_VOLUME_EVENT_HANDLER_DECL(PackedVolumeAttachEvent, on_volume_attach) LLFS_VOLUME_EVENT_HANDLER_DECL(PackedVolumeDetachEvent, on_volume_detach) @@ -61,65 +65,68 @@ class VolumeEventVisitor VolumeEventVisitor() = default; }; -//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- template -/*static*/ VolumeEventVisitor& VolumeEventVisitor::null_impl() +class VolumeEventVisitor::NullImpl : public VolumeEventVisitor { - class NullImpl : public VolumeEventVisitor + public: + R on_raw_data(const SlotParse&, const Ref&) override { - public: - R on_raw_data(const SlotParse&, const Ref&) override - { - return batt::make_default(); - } - - R on_prepare_job(const SlotParse&, const Ref&) override - { - return batt::make_default(); - } - - R on_commit_job(const SlotParse&, const PackedCommitJob&) override - { - return batt::make_default(); - } - - R on_rollback_job(const SlotParse&, const PackedRollbackJob&) override - { - return batt::make_default(); - } - - R on_volume_attach(const SlotParse&, const PackedVolumeAttachEvent&) override - { - return batt::make_default(); - } - - R on_volume_detach(const SlotParse&, const PackedVolumeDetachEvent&) override - { - return batt::make_default(); - } - - R on_volume_ids(const SlotParse&, const PackedVolumeIds&) override - { - return batt::make_default(); - } - - R on_volume_recovered(const SlotParse&, const PackedVolumeRecovered&) override - { - return batt::make_default(); - } - - R on_volume_format_upgrade(const SlotParse&, const PackedVolumeFormatUpgrade&) override - { - return batt::make_default(); - } - - R on_volume_trim(const SlotParse&, const VolumeTrimEvent&) override - { - return batt::make_default(); - } - }; + return batt::make_default(); + } + + R on_prepare_job(const SlotParse&, const Ref&) override + { + return batt::make_default(); + } + R on_commit_job(const SlotParse&, const Ref&) override + { + return batt::make_default(); + } + + R on_rollback_job(const SlotParse&, const PackedRollbackJob&) override + { + return batt::make_default(); + } + + R on_volume_attach(const SlotParse&, const PackedVolumeAttachEvent&) override + { + return batt::make_default(); + } + + R on_volume_detach(const SlotParse&, const PackedVolumeDetachEvent&) override + { + return batt::make_default(); + } + + R on_volume_ids(const SlotParse&, const PackedVolumeIds&) override + { + return batt::make_default(); + } + + R on_volume_recovered(const SlotParse&, const PackedVolumeRecovered&) override + { + return batt::make_default(); + } + + R on_volume_format_upgrade(const SlotParse&, const PackedVolumeFormatUpgrade&) override + { + return batt::make_default(); + } + + R on_volume_trim(const SlotParse&, const VolumeTrimEvent&) override + { + return batt::make_default(); + } +}; + +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ + +template +/*static*/ VolumeEventVisitor& VolumeEventVisitor::null_impl() +{ // It's OK that this is non-const, since it has no state. // static NullImpl impl_; diff --git a/src/llfs/volume_events.cpp b/src/llfs/volume_events.cpp index 0b38d8c..7c28e20 100644 --- a/src/llfs/volume_events.cpp +++ b/src/llfs/volume_events.cpp @@ -15,6 +15,24 @@ namespace llfs { +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +std::ostream& operator<<(std::ostream& out, const PackedVolumeIds& t) +{ + return out << "PackedVolumeIds{.main_uuid=" << t.main_uuid + << ", .recycler_uuid=" << t.recycler_uuid << ", .trimmer_uuid=" << t.trimmer_uuid + << ",}"; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +bool operator==(const VolumeAttachmentId& l, const VolumeAttachmentId& r) +{ + return l.client == r.client // + && l.device == r.device // + ; +} + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // std::ostream& operator<<(std::ostream& out, const VolumeAttachmentId& id) @@ -27,18 +45,41 @@ std::ostream& operator<<(std::ostream& out, const VolumeAttachmentId& id) usize packed_sizeof(const PrepareJob& obj) { return sizeof(PackedPrepareJob) + // + packed_sizeof(obj.user_data) + // + packed_array_size(trace_refs(obj.user_data) | seq::count()) + // packed_array_size(batt::make_copy(obj.new_page_ids) | seq::count()) + // packed_array_size(batt::make_copy(obj.deleted_page_ids) | seq::count()) + // packed_array_size(batt::make_copy(obj.page_device_ids) | - seq::count()) + // - packed_array_size(trace_refs(obj.user_data) | seq::count()) + // - packed_sizeof(obj.user_data); + seq::count()); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // PackedPrepareJob* pack_object_to(const PrepareJob& obj, PackedPrepareJob* packed, DataPacker* dst) { + const usize user_data_size = packed_sizeof(obj.user_data); + { + Optional user_data_buffer = dst->reserve_front(user_data_size); + if (!user_data_buffer) { + return nullptr; + } + DataPacker user_data_packer{*user_data_buffer}; + PackedRawData* packed_user_data = pack_object(obj.user_data, &user_data_packer); + if (!packed_user_data) { + return nullptr; + } + BATT_CHECK_EQ((const void*)packed_user_data, (const void*)(packed + 1)); + } + + // IMPORTANT: root_page_ids must be first (after user data)! + { + PackedArray* packed_root_page_ids = pack_object(trace_refs(obj.user_data), dst); + if (!packed_root_page_ids) { + return nullptr; + } + packed->root_page_ids.reset(packed_root_page_ids, dst); + } + //----- --- -- - - - - { PackedArray* packed_new_page_ids = pack_object(obj.new_page_ids, dst); if (!packed_new_page_ids) { @@ -64,22 +105,6 @@ PackedPrepareJob* pack_object_to(const PrepareJob& obj, PackedPrepareJob* packed packed->page_device_ids.reset(packed_page_device_ids, dst); } //----- --- -- - - - - - { - PackedArray* packed_root_page_ids = pack_object(trace_refs(obj.user_data), dst); - if (!packed_root_page_ids) { - return nullptr; - } - packed->root_page_ids.reset(packed_root_page_ids, dst); - } - //----- --- -- - - - - - { - PackedRawData* packed_user_data = pack_object(obj.user_data, dst); - if (!packed_user_data) { - return nullptr; - } - packed->user_data.reset(packed_user_data, dst); - } - //----- --- -- - - - - return packed; } @@ -92,6 +117,20 @@ StatusOr> unpack_object(const PackedPrepareJob& pack return as_cref(packed); } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status validate_packed_value(const PackedPrepareJob& packed, const void* buffer_data, + usize buffer_size) +{ + BATT_REQUIRE_OK(validate_packed_struct(packed, buffer_data, buffer_size)); + BATT_REQUIRE_OK(validate_packed_value(*packed.root_page_ids, buffer_data, buffer_size)); + BATT_REQUIRE_OK(validate_packed_value(*packed.new_page_ids, buffer_data, buffer_size)); + BATT_REQUIRE_OK(validate_packed_value(*packed.deleted_page_ids, buffer_data, buffer_size)); + BATT_REQUIRE_OK(validate_packed_value(*packed.page_device_ids, buffer_data, buffer_size)); + + return OkStatus(); +} + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // usize packed_sizeof(const PackedPrepareJob& obj) @@ -101,220 +140,133 @@ usize packed_sizeof(const PackedPrepareJob& obj) packed_sizeof(*obj.deleted_page_ids) + // packed_sizeof(*obj.root_page_ids) + // packed_sizeof(*obj.page_device_ids) + // - packed_sizeof(*obj.user_data); + obj.user_data().size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize packed_sizeof_commit(const PrepareJob& obj) +{ + return sizeof(PackedCommitJob) + packed_sizeof(obj.user_data) + + packed_array_size(trace_refs(obj.user_data) | seq::count()); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -usize packed_sizeof(const TrimmedPrepareJob& object) +usize packed_sizeof_commit_slot(const PrepareJob& obj) { - return sizeof(PackedTrimmedPrepareJob) + - sizeof(PackedPageId) * (batt::make_copy(object.page_ids) | batt::seq::count()); + return packed_sizeof_slot_with_payload_size(packed_sizeof_commit(obj)); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize packed_sizeof(const CommitJob& obj) +{ + BATT_CHECK_NOT_NULLPTR(obj.packed_prepare); + BATT_CHECK_NOT_NULLPTR(obj.packed_prepare->root_page_ids.get()); + + return sizeof(PackedCommitJob) + packed_sizeof(*obj.packed_prepare->root_page_ids) + + obj.packed_prepare->user_data().size(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -usize packed_sizeof(const PackedTrimmedPrepareJob& packed) +usize packed_sizeof(const PackedCommitJob& obj) { - return sizeof(PackedTrimmedPrepareJob) + sizeof(PackedPageId) * packed.page_ids.size(); + return sizeof(PackedCommitJob) + packed_sizeof(*obj.root_page_ids) + obj.user_data().size(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -PackedTrimmedPrepareJob* pack_object_to(const TrimmedPrepareJob& object, - PackedTrimmedPrepareJob* packed, DataPacker* dst) +PackedCommitJob* pack_object_to(const CommitJob& obj, PackedCommitJob* packed, DataPacker* dst) { - packed->prepare_slot = object.prepare_slot; - packed->page_ids.initialize(0u); - - bool error = false; - usize count = 0; - batt::make_copy(object.page_ids) // - | batt::seq::for_each([&count, &error, dst](PageId page_id) { - if (pack_object(page_id, dst) == nullptr) { - error = true; - return batt::seq::LoopControl::kBreak; - } - ++count; - return batt::seq::LoopControl::kContinue; - }); - - if (error) { + packed->prepare_slot_offset = obj.prepare_slot_offset; + packed->prepare_slot_size = packed_sizeof_slot(*obj.packed_prepare); + + // Byte-wise copy the user data from the prepare job directly after the PackedCommitJob struct. + // + std::string_view user_data = obj.packed_prepare->user_data(); + Optional packed_user_data = + dst->pack_raw_data(user_data.data(), user_data.size()); + if (!packed_user_data) { + return nullptr; + } + BATT_CHECK_EQ((const void*)(packed + 1), (const void*)packed_user_data->data()) + << "User data must come right after the PackedCommitJob!"; + + // Place the root_page_ids PackedArray after the user data; this allows us to use the offset + // stored in the pointer (PackedCommitJob::root_page_ids) to derive the size of user data. + // + PackedArray* packed_root_page_ids = dst->pack_record>(); + if (!packed_root_page_ids) { return nullptr; } - packed->page_ids.item_count = count; + // Initialize the array and copy any page ids. + // + const usize id_count = obj.packed_prepare->root_page_ids->size(); + if (id_count != 0) { + if (!dst->pack_raw_data(obj.packed_prepare->root_page_ids->data(), + sizeof(PackedPageId) * id_count)) { + return nullptr; + } + } + packed_root_page_ids->initialize(id_count); + packed->root_page_ids.reset(packed_root_page_ids, dst); return packed; } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// TODO [tastolfi 2023-08-22] DEPRECATE (replace with unpack_cast/validate) // -StatusOr unpack_object(const PackedTrimmedPrepareJob& packed, - DataReader* /*src*/) +StatusOr> unpack_object(const PackedCommitJob& packed, DataReader*) { - TrimmedPrepareJob object; - - object.prepare_slot = packed.prepare_slot; - object.page_ids = as_seq(packed.page_ids) // - | batt::seq::map([](const PackedPageId& page_id) { - return page_id.unpack(); - }) // - | batt::seq::boxed(); - - return object; + return as_cref(packed); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -Status validate_packed_value(const PackedTrimmedPrepareJob& packed, const void* buffer_data, +Status validate_packed_value(const PackedCommitJob& packed, const void* buffer_data, usize buffer_size) { BATT_REQUIRE_OK(validate_packed_struct(packed, buffer_data, buffer_size)); - BATT_REQUIRE_OK( - validate_packed_byte_range(&packed, packed_sizeof(packed), buffer_data, buffer_size)); + BATT_REQUIRE_OK(validate_packed_value(*packed.root_page_ids, buffer_data, buffer_size)); - return batt::OkStatus(); + return OkStatus(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -usize packed_sizeof(const VolumeTrimEvent& object) +usize packed_sizeof(const VolumeTrimEvent&) { - const usize n_committed = [&] { - if (object.committed_jobs) { - return batt::make_copy(object.committed_jobs) | batt::seq::count(); - } else { - return usize{0}; - } - }(); - - return sizeof(PackedVolumeTrimEvent) // - + n_committed * sizeof(PackedSlotOffset) // - + ((n_committed > 0) ? sizeof(PackedArray) : 0u) // - + (batt::make_copy(object.trimmed_prepare_jobs) // - | batt::seq::map([](const TrimmedPrepareJob& pending) { - return packed_sizeof(pending) + sizeof(PackedPointer); - }) // - | batt::seq::sum()); + return sizeof(PackedVolumeTrimEvent); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -usize packed_sizeof(const PackedVolumeTrimEvent& packed) +usize packed_sizeof(const PackedVolumeTrimEvent&) { - return sizeof(PackedVolumeTrimEvent) + - sizeof(PackedPointer) * packed.trimmed_prepare_jobs.size() + - ([&]() -> usize { - if (!packed.committed_jobs) { - return 0; - } else { - return packed.committed_jobs->size() * sizeof(PackedSlotOffset); - } - }()) + - (as_seq(packed.trimmed_prepare_jobs) // - | batt::seq::map([](const PackedPointer& p_job) { - return packed_sizeof(*p_job); - }) // - | batt::seq::sum()); + return sizeof(PackedVolumeTrimEvent); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // PackedVolumeTrimEvent* pack_object_to(const VolumeTrimEvent& object, PackedVolumeTrimEvent* packed, - DataPacker* dst) + DataPacker*) { packed->old_trim_pos = object.old_trim_pos; packed->new_trim_pos = object.new_trim_pos; - packed->committed_jobs.offset = 0; - packed->trimmed_prepare_jobs.initialize(0u); - - const usize pending_job_count = batt::make_copy(object.trimmed_prepare_jobs) | batt::seq::count(); - - for (usize i = 0; i < pending_job_count; ++i) { - if (dst->pack_record>() == nullptr) { - return nullptr; - } - } - - { - bool error = false; - batt::make_copy(object.trimmed_prepare_jobs) // - | batt::seq::for_each([&error, packed, dst](const TrimmedPrepareJob& job) { - PackedTrimmedPrepareJob* const packed_job = pack_object(job, dst); - if (packed_job == nullptr) { - error = true; - return batt::seq::LoopControl::kBreak; - } - packed->trimmed_prepare_jobs.item_count += 1; - packed->trimmed_prepare_jobs[packed->trimmed_prepare_jobs.size() - 1].reset(packed_job, - dst); - return batt::seq::LoopControl::kContinue; - }); - - if (error) { - return nullptr; - } - } - - BATT_CHECK_EQ(pending_job_count, packed->trimmed_prepare_jobs.size()); - - if ((batt::make_copy(object.committed_jobs) | batt::seq::take_n(1) | batt::seq::count()) > 0) { - PackedArray* const committed_jobs = - dst->pack_record(batt::StaticType>{}); - if (committed_jobs == nullptr) { - return nullptr; - } - committed_jobs->initialize(0u); - packed->committed_jobs.reset(committed_jobs, dst); - - bool error = false; - batt::make_copy(object.committed_jobs) // - | batt::seq::for_each([&error, committed_jobs, dst](slot_offset_type prepare_slot) { - if (!dst->pack_u64(prepare_slot)) { - error = true; - return batt::seq::LoopControl::kBreak; - } - committed_jobs->item_count += 1; - return batt::seq::LoopControl::kContinue; - }); - if (error) { - return nullptr; - } - } - return packed; } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -StatusOr unpack_object(const PackedVolumeTrimEvent& packed, DataReader* src) +StatusOr unpack_object(const PackedVolumeTrimEvent& packed, DataReader*) { return VolumeTrimEvent{ .old_trim_pos = packed.old_trim_pos, .new_trim_pos = packed.new_trim_pos, - - .committed_jobs = [&]() -> batt::BoxedSeq { - if (packed.committed_jobs) { - return as_seq(*packed.committed_jobs) | - batt::seq::map([](const PackedSlotOffset& prepare_slot) -> slot_offset_type { - return prepare_slot.value(); - }) | - batt::seq::boxed(); - } else { - return batt::seq::Empty{} | batt::seq::boxed(); - } - }(), - - .trimmed_prepare_jobs = - as_seq(packed.trimmed_prepare_jobs) // - | batt::seq::map( - [src](const PackedPointer& p_job) -> TrimmedPrepareJob { - StatusOr job = unpack_object(*p_job, src); - BATT_CHECK_OK(job); - return std::move(*job); - }) // - | batt::seq::boxed(), }; } @@ -324,10 +276,18 @@ Status validate_packed_value(const PackedVolumeTrimEvent& packed, const void* bu usize buffer_size) { BATT_REQUIRE_OK(validate_packed_struct(packed, buffer_data, buffer_size)); - BATT_REQUIRE_OK(validate_packed_value(packed.committed_jobs, buffer_data, buffer_size)); - BATT_REQUIRE_OK(validate_packed_value(packed.trimmed_prepare_jobs, buffer_data, buffer_size)); return batt::OkStatus(); } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +std::ostream& operator<<(std::ostream& out, const VolumeTrimEvent& t) +{ + return out << "VolumeTrimEvent" // + << "{.old_trim_pos=" << t.old_trim_pos // + << ", .new_trim_pos=" << t.new_trim_pos // + << ",}"; +} + } // namespace llfs diff --git a/src/llfs/volume_events.hpp b/src/llfs/volume_events.hpp index b23d459..347a600 100644 --- a/src/llfs/volume_events.hpp +++ b/src/llfs/volume_events.hpp @@ -28,6 +28,18 @@ namespace llfs { +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +struct PackedVolumeIds { + boost::uuids::uuid main_uuid; + boost::uuids::uuid recycler_uuid; + boost::uuids::uuid trimmer_uuid; +}; + +LLFS_SIMPLE_PACKED_TYPE(PackedVolumeIds); + +std::ostream& operator<<(std::ostream& out, const PackedVolumeIds& t); + //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- template @@ -51,12 +63,7 @@ struct VolumeAttachmentId { }; }; -inline bool operator==(const VolumeAttachmentId& l, const VolumeAttachmentId& r) -{ - return l.client == r.client // - && l.device == r.device // - ; -} +bool operator==(const VolumeAttachmentId& l, const VolumeAttachmentId& r); std::ostream& operator<<(std::ostream& out, const VolumeAttachmentId& id); @@ -86,7 +93,7 @@ struct PackedVolumeAttachEvent : PackedVolumeAttachmentEvent new_page_ids; + BoxedSeq deleted_page_ids; + BoxedSeq page_device_ids; + PackableRef user_data; +}; + +usize packed_sizeof(const PrepareJob& obj); + +/** \brief Calculates and returns the size (in bytes) of a PackedCommitJob for the passed + * PrepareJob. NOTE: this is *just* the size of the PackedCommitJob itself, not including + * slot/variant headers. + */ +usize packed_sizeof_commit(const PrepareJob& obj); + +/** \brief Calculates and returns the size (in bytes) of a full PackedCommitJob slot for the passed + * PrepareJob. + */ +usize packed_sizeof_commit_slot(const PrepareJob& obj); + +inline std::ostream& operator<<(std::ostream& out, const PrepareJob& t) { - return out << "PackedVolumeIds{.main_uuid=" << t.main_uuid - << ", .recycler_uuid=" << t.recycler_uuid << ", .trimmer_uuid=" << t.trimmer_uuid - << ",}"; + return out << "PrepareJob{.new_page_ids=[" + << (batt::make_copy(t.new_page_ids) | batt::seq::count()) << "], .deleted_page_ids=[" + << (batt::make_copy(t.deleted_page_ids) | batt::seq::count()) + << "], .page_device_ids=[" << (batt::make_copy(t.page_device_ids) | batt::seq::count()) + << ", .user_data=...,}"; } -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// Packed representation of PrepareJob. // -struct PackedTrimmedPrepareJob { - PackedSlotOffset prepare_slot; - PackedArray page_ids; -}; +// +------------------+----------------+--------------------------------------------+-- - +// | PackedPrepareJob | (user_data...) | PackedArray root_page_ids... | ... +// +------------------+----------------+--------------------------------------------+-- - +// +struct PackedPrepareJob { + PackedPrepareJob(const PackedPrepareJob&) = delete; + PackedPrepareJob& operator=(const PackedPrepareJob&) = delete; -BATT_STATIC_ASSERT_EQ(sizeof(PackedTrimmedPrepareJob), 16); + PackedPointer> root_page_ids; + PackedPointer> new_page_ids; + PackedPointer> deleted_page_ids; + PackedPointer> page_device_ids; -struct TrimmedPrepareJob { - slot_offset_type prepare_slot; - batt::BoxedSeq page_ids; -}; + //+++++++++++-+-+--+----- --- -- - - - - -LLFS_DEFINE_PACKED_TYPE_FOR(TrimmedPrepareJob, PackedTrimmedPrepareJob); + std::string_view user_data() const noexcept + { + const char* p_begin = reinterpret_cast(this + 1); + const char* p_end = reinterpret_cast(this->root_page_ids.get()); -usize packed_sizeof(const TrimmedPrepareJob& object); + BATT_CHECK_LE((const void*)p_begin, (const void*)p_end); -usize packed_sizeof(const PackedTrimmedPrepareJob& packed); + return std::string_view{p_begin, usize(p_end - p_begin)}; + } +}; -PackedTrimmedPrepareJob* pack_object_to(const TrimmedPrepareJob& object, - PackedTrimmedPrepareJob* packed, DataPacker* dst); +LLFS_DEFINE_PACKED_TYPE_FOR(PrepareJob, PackedPrepareJob); -StatusOr unpack_object(const PackedTrimmedPrepareJob& packed, DataReader* src); +usize packed_sizeof(const PackedPrepareJob& obj); + +PackedPrepareJob* pack_object_to(const PrepareJob& obj, PackedPrepareJob* packed, DataPacker* dst); + +StatusOr> unpack_object(const PackedPrepareJob& packed, DataReader*); -Status validate_packed_value(const PackedTrimmedPrepareJob& packed, const void* buffer_data, +Status validate_packed_value(const PackedPrepareJob& packed, const void* buffer_data, usize buffer_size); //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -/** \brief Written and flushed to the Volume WAL before trimming a segment of the log. This allows - * correct recovery in the case where a trim operation that needs to make page ref_count updates is - * interrupted by shutdown. - * - * Only one pending PackedVolumeTrim event may be present in the WAL at a given time. A trim event - * is considered "pending" when new_trim_position is ahead of the actual log trim position, and it - * is considered resolved when the actual log trim position catches up. - */ -struct PackedVolumeTrimEvent { - PackedSlotOffset old_trim_pos; - PackedSlotOffset new_trim_pos; +// +struct CommitJob { + slot_offset_type prepare_slot_offset; + const PackedPrepareJob* packed_prepare; +}; - /** \brief The prepare slot offsets of any jobs that were resolved in this trim (i.e., the - * CommitJob slot for the corresponding prepare slot was found in the trimmed region). - */ - PackedPointer> committed_jobs; +usize packed_sizeof(const CommitJob& obj); - /** \brief The pending PrepareJob slots from this trimmed region or a previous one. - */ - PackedArray> trimmed_prepare_jobs; -}; +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +// +------------------+----------------+-------------------------------------------+ +// | PackedCommitJob | (user_data...) | PackedArray root_page_ids... | +// +------------------+----------------+-------------------------------------------+ +// +struct PackedCommitJob { + PackedCommitJob(const PackedCommitJob&) = delete; + PackedCommitJob& operator=(const PackedCommitJob&) = delete; -BATT_STATIC_ASSERT_EQ(sizeof(PackedVolumeTrimEvent), 28); + PackedSlotOffset prepare_slot_offset; + little_u32 prepare_slot_size; + PackedPointer> root_page_ids; -struct VolumeTrimEvent { - slot_offset_type old_trim_pos; - slot_offset_type new_trim_pos; - batt::BoxedSeq committed_jobs; - batt::BoxedSeq trimmed_prepare_jobs; + //+++++++++++-+-+--+----- --- -- - - - - + + std::string_view user_data() const noexcept + { + const char* p_begin = reinterpret_cast(this + 1); + const char* p_end = reinterpret_cast(this->root_page_ids.get()); + + BATT_CHECK_LE((const void*)p_begin, (const void*)p_end); + + return std::string_view{p_begin, usize(p_end - p_begin)}; + } }; -LLFS_DEFINE_PACKED_TYPE_FOR(VolumeTrimEvent, PackedVolumeTrimEvent); +LLFS_DEFINE_PACKED_TYPE_FOR(CommitJob, PackedCommitJob); -usize packed_sizeof(const VolumeTrimEvent& object); +BATT_STATIC_ASSERT_EQ(sizeof(PackedCommitJob), 16); -usize packed_sizeof(const PackedVolumeTrimEvent& packed); +usize packed_sizeof(const PackedCommitJob& obj); -PackedVolumeTrimEvent* pack_object_to(const VolumeTrimEvent& object, PackedVolumeTrimEvent* packed, - DataPacker* dst); +PackedCommitJob* pack_object_to(const CommitJob& obj, PackedCommitJob* packed, DataPacker* dst); -StatusOr unpack_object(const PackedVolumeTrimEvent& packed, DataReader* src); +StatusOr> unpack_object(const PackedCommitJob& packed, DataReader*); -Status validate_packed_value(const PackedVolumeTrimEvent& packed, const void* buffer_data, +Status validate_packed_value(const PackedCommitJob& packed, const void* buffer_data, usize buffer_size); //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- // -struct PackedVolumeRecovered { +struct PackedRollbackJob { + PackedSlotOffset prepare_slot; }; -LLFS_SIMPLE_PACKED_TYPE(PackedVolumeRecovered); +LLFS_SIMPLE_PACKED_TYPE(PackedRollbackJob); //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- // @@ -211,57 +257,41 @@ struct PackedVolumeFormatUpgrade { LLFS_SIMPLE_PACKED_TYPE(PackedVolumeFormatUpgrade); //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// The "prepare" phase slot written to a log when transactionally appending a PageCacheJob with user -// data (T). -// -struct PrepareJob { - BoxedSeq new_page_ids; - BoxedSeq deleted_page_ids; - BoxedSeq page_device_ids; - PackableRef user_data; +/** \brief Written and flushed to the Volume WAL before trimming a segment of the log. This allows + * correct recovery in the case where a trim operation that needs to make page ref_count updates is + * interrupted by shutdown. + * + * Only one pending PackedVolumeTrim event may be present in the WAL at a given time. A trim event + * is considered "pending" when new_trim_position is ahead of the actual log trim position, and it + * is considered resolved when the actual log trim position catches up. + */ +struct PackedVolumeTrimEvent { + PackedSlotOffset old_trim_pos; + PackedSlotOffset new_trim_pos; }; -usize packed_sizeof(const PrepareJob& obj); - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// Packed representation of PrepareJob. -// -struct PackedPrepareJob { - PackedPrepareJob(const PackedPrepareJob&) = delete; - PackedPrepareJob& operator=(const PackedPrepareJob&) = delete; +BATT_STATIC_ASSERT_EQ(sizeof(PackedVolumeTrimEvent), 16); - u8 reserved_[sizeof(PackedVolumeTrimEvent) + sizeof(PackedPointer)]; - PackedPointer> new_page_ids; - PackedPointer> deleted_page_ids; - PackedPointer> root_page_ids; - PackedPointer> page_device_ids; - PackedPointer user_data; +struct VolumeTrimEvent { + slot_offset_type old_trim_pos; + slot_offset_type new_trim_pos; }; -LLFS_DEFINE_PACKED_TYPE_FOR(PrepareJob, PackedPrepareJob); - -usize packed_sizeof(const PackedPrepareJob& obj); +LLFS_DEFINE_PACKED_TYPE_FOR(VolumeTrimEvent, PackedVolumeTrimEvent); -PackedPrepareJob* pack_object_to(const PrepareJob& obj, PackedPrepareJob* packed, DataPacker* dst); +usize packed_sizeof(const VolumeTrimEvent& object); -StatusOr> unpack_object(const PackedPrepareJob& packed, DataReader*); +usize packed_sizeof(const PackedVolumeTrimEvent& packed); -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// -struct PackedCommitJob { - u8 reserved_[sizeof(PackedVolumeTrimEvent) + sizeof(PackedArray)]; - PackedSlotOffset prepare_slot; -}; +PackedVolumeTrimEvent* pack_object_to(const VolumeTrimEvent& object, PackedVolumeTrimEvent* packed, + DataPacker* dst); -LLFS_SIMPLE_PACKED_TYPE(PackedCommitJob); +StatusOr unpack_object(const PackedVolumeTrimEvent& packed, DataReader* src); -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// -struct PackedRollbackJob { - PackedSlotOffset prepare_slot; -}; +Status validate_packed_value(const PackedVolumeTrimEvent& packed, const void* buffer_data, + usize buffer_size); -LLFS_SIMPLE_PACKED_TYPE(PackedRollbackJob); +std::ostream& operator<<(std::ostream& out, const VolumeTrimEvent& t); } // namespace llfs diff --git a/src/llfs/volume_events.test.cpp b/src/llfs/volume_events.test.cpp index eccd117..d45d30e 100644 --- a/src/llfs/volume_events.test.cpp +++ b/src/llfs/volume_events.test.cpp @@ -15,226 +15,314 @@ #include +#include + namespace { using namespace llfs::int_types; -class VolumeEventsTest : public llfs::testing::PackedTypeTestFixture +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +/** \brief Fake user data, for testing only. + */ +struct FakeUserData { + std::string str; +}; + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize packed_sizeof(const FakeUserData& obj) { - public: - llfs::TrimmedPrepareJob make_trimmed_prepare_job(llfs::slot_offset_type prepare_slot, - usize n_pages) - { - llfs::TrimmedPrepareJob object; - object.prepare_slot = prepare_slot; - object.page_ids = batt::as_seq(this->page_ids) | batt::seq::take_n(n_pages) | - batt::seq::decayed() | batt::seq::boxed(); + return obj.str.size(); +} - return object; +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +llfs::PackedRawData* pack_object(const FakeUserData& obj, llfs::DataPacker* dst) +{ + llfs::Optional packed = dst->pack_raw_data(obj.str.data(), obj.str.size()); + if (!packed) { + return nullptr; } - - std::vector page_ids{ - llfs::PageId{1}, llfs::PageId{2}, llfs::PageId{3}, llfs::PageId{4}, - llfs::PageId{5}, llfs::PageId{6}, llfs::PageId{7}, llfs::PageId{8}, - llfs::PageId{9}, llfs::PageId{10}, llfs::PageId{11}, llfs::PageId{12}, - }; -}; + return (llfs::PackedRawData*)packed->data(); +} //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// Test plan: -// - do all of: -// 1. serialize using pack_object -// 2. unpack via unpack_object -// 1. success -// 2. failure - buffer truncated -// 3. unpack via unpack_cast -// - for each of: -// a. page_ids empty -// b. 1 page_ids -// c. >1 page_ids // -TEST_F(VolumeEventsTest, TrimmedPrepareJobPackUnpackNoPages) +llfs::BoxedSeq trace_refs(const FakeUserData& obj) { - llfs::TrimmedPrepareJob object; - object.prepare_slot = 1996; - object.page_ids = batt::seq::Empty{} | batt::seq::boxed(); - - ASSERT_NE(this->pack_into_buffer(object), nullptr); + const llfs::PackedPageId* packed_ids = (const llfs::PackedPageId*)obj.str.data(); + usize count = obj.str.size() / sizeof(llfs::PackedPageId); + + return batt::as_seq(boost::irange(usize{0}, count)) // + | batt::seq::map([packed_ids](usize i) -> llfs::PageId { + return packed_ids[i].unpack(); + }) // + | batt::seq::boxed(); +} +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +class VolumeJobEventsTest : public ::testing::Test +{ + public: + template + void pack_test_object(const T& obj, std::vector& storage) { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer()); + usize size = packed_sizeof(obj); - ASSERT_TRUE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed->prepare_slot, object.prepare_slot); - EXPECT_EQ(packed->page_ids.size(), 0u); + storage.clear(); + storage.resize(size); - batt::StatusOr unpacked = this->unpack_from_buffer(*packed); + llfs::MutableBuffer buffer{storage.data(), storage.size()}; + llfs::DataPacker packer{buffer}; - ASSERT_TRUE(unpacked.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(unpacked->prepare_slot, object.prepare_slot); - EXPECT_EQ((batt::make_copy(unpacked->page_ids) | batt::seq::collect_vec()), - (batt::make_copy(object.page_ids) | batt::seq::collect_vec())); + ASSERT_TRUE(pack_object(obj, &packer)); + EXPECT_EQ(packer.space(), 0u) + << "packed_sizeof did not report the actual size used by pack_object!"; } + + void init_user_data(usize user_data_size) noexcept { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer(1)); + this->user_data_.str.clear(); + this->root_refs_.clear(); + + if (user_data_size < sizeof(llfs::PackedPageId)) { + this->user_data_.str = std::string(user_data_size, 'a'); + + } else { + this->root_refs_.resize(user_data_size / sizeof(llfs::PackedPageId)); + for (usize i = 0; i < this->root_refs_.size(); ++i) { + this->root_refs_[i] = llfs::PackedPageId{ + .id_val = i, + }; + } + this->user_data_.str = std::string_view{(const char*)this->root_refs_.data(), + this->root_refs_.size() * sizeof(llfs::PackedPageId)}; + } + EXPECT_EQ(this->user_data_.str.size(), user_data_size); + } - ASSERT_FALSE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed.status(), llfs::StatusCode::kUnpackCastStructOver); + void init_new_page_ids(usize new_page_count) noexcept + { + this->new_page_ids_.clear(); + for (usize i = 0; i < new_page_count; ++i) { + this->new_page_ids_.emplace_back(llfs::PageId{i}); + } } -} -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -TEST_F(VolumeEventsTest, TrimmedPrepareJobPackUnpackSinglePage) -{ - llfs::TrimmedPrepareJob object; - object.prepare_slot = 2001; - object.page_ids = batt::seq::single_item(llfs::PageId{0x2468}) | batt::seq::boxed(); + void init_deleted_page_ids(usize deleted_page_count) noexcept + { + this->deleted_page_ids_.clear(); + for (usize i = 0; i < deleted_page_count; ++i) { + this->deleted_page_ids_.emplace_back(llfs::PageId{i + 1000}); + } + } - ASSERT_NE(this->pack_into_buffer(object), nullptr); + void init_page_device_ids(usize page_device_count) noexcept + { + this->page_device_ids_.clear(); + for (usize i = 0; i < page_device_count; ++i) { + this->page_device_ids_.emplace_back(i); + } + } + void pack_prepare_job(std::vector& storage) noexcept { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer()); + this->pack_test_object( + llfs::PrepareJob{ + .new_page_ids = (batt::as_seq(this->new_page_ids_) // + | batt::seq::decayed() // + | batt::seq::boxed()), + + .deleted_page_ids = (batt::as_seq(this->deleted_page_ids_) | // + batt::seq::decayed() | // + batt::seq::boxed()), + + .page_device_ids = (batt::as_seq(this->page_device_ids_) | // + batt::seq::decayed() | // + batt::seq::boxed()), + + .user_data = llfs::PackableRef{this->user_data_}, + }, + storage); + } - ASSERT_TRUE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed->prepare_slot, object.prepare_slot); - EXPECT_EQ(packed->page_ids.size(), 1u); + void verify_page_ids(const llfs::PackedArray& actual_ids, + const std::vector& expected_ids, const char* file, int line) + { + EXPECT_EQ((as_seq(actual_ids) // + | batt::seq::map([](const llfs::PackedPageId& id) { + return id.unpack(); + }) // + | batt::seq::collect_vec()), + expected_ids) + << BATT_INSPECT(file) << BATT_INSPECT(line); + } - batt::StatusOr unpacked = this->unpack_from_buffer(*packed); + void verify_page_ids(const llfs::PackedArray& actual_ids, + const std::vector& expected_ids, const char* file, + int line) + { + EXPECT_EQ((as_seq(actual_ids) // + | batt::seq::decayed() // + | batt::seq::collect_vec()), + expected_ids) + << BATT_INSPECT(file) << BATT_INSPECT(line); + } + + void verify_prepare_job(const llfs::PackedPrepareJob& unpacked) noexcept + { + ASSERT_TRUE(unpacked.root_page_ids); + ASSERT_TRUE(unpacked.new_page_ids); + ASSERT_TRUE(unpacked.deleted_page_ids); + ASSERT_TRUE(unpacked.page_device_ids); + + EXPECT_THAT(unpacked.user_data(), ::testing::StrEq(this->user_data_.str)); - ASSERT_TRUE(unpacked.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(unpacked->prepare_slot, object.prepare_slot); - EXPECT_EQ((batt::make_copy(unpacked->page_ids) | batt::seq::collect_vec()), - (batt::make_copy(object.page_ids) | batt::seq::collect_vec())); + this->verify_page_ids( // + *unpacked.root_page_ids, this->root_refs_, __FILE__, __LINE__); + + this->verify_page_ids( // + *unpacked.new_page_ids, this->new_page_ids_, __FILE__, __LINE__); + + this->verify_page_ids( // + *unpacked.deleted_page_ids, this->deleted_page_ids_, __FILE__, __LINE__); + + EXPECT_EQ((as_seq(*unpacked.page_device_ids) // + | batt::seq::map([](const auto& id) { + return id.value(); + }) // + | batt::seq::collect_vec()), + this->page_device_ids_); } + + void verify_commit_job(const llfs::PackedCommitJob& unpacked) noexcept { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer(1)); + ASSERT_TRUE(unpacked.root_page_ids); + + EXPECT_THAT(unpacked.user_data(), ::testing::StrEq(this->user_data_.str)); - ASSERT_FALSE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed.status(), llfs::StatusCode::kUnpackCastStructOver); + this->verify_page_ids( // + *unpacked.root_page_ids, this->root_refs_, __FILE__, __LINE__); } -} -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - + //+++++++++++-+-+--+----- --- -- - - - - + + FakeUserData user_data_; + std::vector root_refs_; + std::vector new_page_ids_; + std::vector deleted_page_ids_; + std::vector page_device_ids_; +}; + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// PrepareJob Test Plan +// -------------------- +// 1. Minimal event, pack/unpack +// - zero size user data +// - no root page ids, new page ids, deleted page ids, or page device ids +// 2. Test all combinations of zero size, size=1, and size > 1 for each field. // -TEST_F(VolumeEventsTest, TrimmedPrepareJobPackUnpackMultiPage) +TEST_F(VolumeJobEventsTest, PrepareJobTest) { - llfs::TrimmedPrepareJob object = this->make_trimmed_prepare_job(40000, 12); + for (usize user_data_size : {usize{0}, usize{1}, usize{sizeof(llfs::PackedPageId) * 1}, + usize{sizeof(llfs::PackedPageId) * 17}}) { + for (usize new_page_count : {0, 1, 101}) { + for (usize deleted_page_count : {0, 1, 102}) { + for (usize page_device_count : {0, 1, 7}) { + std::vector storage; - ASSERT_NE(this->pack_into_buffer(object), nullptr); + this->init_user_data(user_data_size); + this->init_new_page_ids(new_page_count); + this->init_deleted_page_ids(deleted_page_count); + this->init_page_device_ids(page_device_count); - { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer()); + ASSERT_NO_FATAL_FAILURE(this->pack_prepare_job(storage)); - ASSERT_TRUE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed->prepare_slot, object.prepare_slot); - EXPECT_EQ(packed->page_ids.size(), 12u); + llfs::StatusOr unpacked = + llfs::unpack_cast(storage, batt::StaticType{}); - batt::StatusOr unpacked = this->unpack_from_buffer(*packed); + ASSERT_TRUE(unpacked.ok()) << BATT_INSPECT(unpacked.status()); + EXPECT_EQ(llfs::packed_sizeof(*unpacked), storage.size()); - ASSERT_TRUE(unpacked.ok()) << BATT_INSPECT(unpacked.status()); - EXPECT_EQ(unpacked->prepare_slot, object.prepare_slot); - EXPECT_EQ((batt::make_copy(unpacked->page_ids) | batt::seq::collect_vec()), - (batt::make_copy(object.page_ids) | batt::seq::collect_vec())); - } - { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer(1)); + llfs::StatusOr> legacy_unpacked = + llfs::unpack_object(*unpacked, nullptr); + + ASSERT_TRUE(legacy_unpacked.ok()) << BATT_INSPECT(legacy_unpacked.status()); + EXPECT_EQ(legacy_unpacked->pointer(), std::addressof(*unpacked)); - ASSERT_FALSE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed.status(), llfs::StatusCode::kUnpackCastStructOver); + this->verify_prepare_job(*unpacked); + } + } + } } } -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// CommitJob Test Plan +// -------------------- +// Repeat the PrepareJob cases, and follow up with a CommitJob; verify user_data and root_page_ids. // -TEST_F(VolumeEventsTest, TrimEventPackUnpack) +TEST_F(VolumeJobEventsTest, CommitJobTest) { - usize prev_packed_size = 0; + for (usize user_data_size : {usize{0}, usize{1}, usize{sizeof(llfs::PackedPageId) * 1}, + usize{sizeof(llfs::PackedPageId) * 17}}) { + for (usize new_page_count : {0, 1, 101}) { + for (usize deleted_page_count : {0, 1, 102}) { + for (usize page_device_count : {0, 1, 7}) { + std::vector prepare_storage; - for (usize n_trimmed_jobs = 0; n_trimmed_jobs < 10; ++n_trimmed_jobs) { - std::vector jobs; - for (usize i = 0; i < n_trimmed_jobs; ++i) { - jobs.emplace_back(this->make_trimmed_prepare_job(/*prepare_slot=*/i, /*n_pages=*/i)); - } + llfs::slot_offset_type fake_prepare_slot = 0xf00dabcde8765; - std::vector commits; - for (usize i = 1; i < n_trimmed_jobs; ++i) { - commits.emplace_back(/*prepare_slot=*/i - 1); - } + this->init_user_data(user_data_size); + this->init_new_page_ids(new_page_count); + this->init_deleted_page_ids(deleted_page_count); + this->init_page_device_ids(page_device_count); - llfs::VolumeTrimEvent trim_event; - trim_event.old_trim_pos = n_trimmed_jobs; - trim_event.new_trim_pos = (n_trimmed_jobs + 1) * 1917; - - trim_event.committed_jobs = // - batt::as_seq(commits) // - | batt::seq::decayed() // - | batt::seq::boxed(); - - trim_event.trimmed_prepare_jobs = // - batt::as_seq(jobs) // - | batt::seq::decayed() // - | batt::seq::boxed(); - - EXPECT_GT(llfs::packed_sizeof(trim_event), prev_packed_size); - prev_packed_size = llfs::packed_sizeof(trim_event); - - ASSERT_NE(this->pack_into_buffer(trim_event), nullptr); - - { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer()); - - ASSERT_TRUE(packed.ok()) << BATT_INSPECT(packed.status()); - EXPECT_EQ(packed->old_trim_pos, trim_event.old_trim_pos); - EXPECT_EQ(packed->new_trim_pos, trim_event.new_trim_pos); - if (n_trimmed_jobs > 1) { - ASSERT_TRUE(packed->committed_jobs); - ASSERT_EQ(packed->committed_jobs->size(), n_trimmed_jobs - 1); - } else { - EXPECT_FALSE(packed->committed_jobs); - } - ASSERT_EQ(packed->trimmed_prepare_jobs.size(), n_trimmed_jobs); + ASSERT_NO_FATAL_FAILURE(this->pack_prepare_job(prepare_storage)); - for (usize i = 0; i < n_trimmed_jobs; ++i) { - const auto& job = packed->trimmed_prepare_jobs[i]; + llfs::StatusOr unpacked_prepare = + llfs::unpack_cast(prepare_storage, batt::StaticType{}); - ASSERT_TRUE(job); - EXPECT_EQ(job->prepare_slot, i); - EXPECT_EQ(job->page_ids.size(), i); + ASSERT_TRUE(unpacked_prepare.ok()) << BATT_INSPECT(unpacked_prepare.status()); - for (usize j = 0; j < i; ++j) { - EXPECT_EQ(this->page_ids[j], job->page_ids[j].unpack()) - << BATT_INSPECT(i) << BATT_INSPECT(j); - } - } + this->verify_prepare_job(*unpacked_prepare); - batt::StatusOr unpacked = this->unpack_from_buffer(*packed); - - ASSERT_TRUE(unpacked.ok()) << BATT_INSPECT(unpacked.status()); - EXPECT_EQ(unpacked->old_trim_pos, trim_event.old_trim_pos); - EXPECT_EQ(unpacked->new_trim_pos, trim_event.new_trim_pos); - EXPECT_EQ((batt::make_copy(unpacked->committed_jobs) | - batt::seq::map([](llfs::PackedSlotOffset offset) -> llfs::slot_offset_type { - return offset.value(); - }) | - batt::seq::collect_vec()), - commits); - EXPECT_EQ((batt::make_copy(unpacked->trimmed_prepare_jobs) | batt::seq::collect_vec()).size(), - n_trimmed_jobs); - } + std::vector commit_storage; + + ASSERT_NO_FATAL_FAILURE(this->pack_test_object( + llfs::CommitJob{ + .prepare_slot_offset = fake_prepare_slot, + .packed_prepare = std::addressof(*unpacked_prepare), + }, + commit_storage)); + + llfs::StatusOr unpacked_commit = + llfs::unpack_cast(commit_storage, batt::StaticType{}); + + ASSERT_TRUE(unpacked_commit.ok()) << BATT_INSPECT(unpacked_commit.status()); + EXPECT_EQ(llfs::packed_sizeof(*unpacked_commit), commit_storage.size()); + EXPECT_EQ(unpacked_commit->prepare_slot_offset, fake_prepare_slot); + EXPECT_EQ(unpacked_commit->prepare_slot_size, + llfs::packed_sizeof_slot_with_payload_size(prepare_storage.size())); + + llfs::StatusOr> legacy_unpacked = + llfs::unpack_object(*unpacked_commit, nullptr); - { - batt::StatusOr packed = - llfs::unpack_cast(this->const_buffer(1)); + ASSERT_TRUE(legacy_unpacked.ok()) << BATT_INSPECT(legacy_unpacked.status()); + EXPECT_EQ(legacy_unpacked->pointer(), std::addressof(*unpacked_commit)); - EXPECT_EQ(packed.status(), llfs::StatusCode::kUnpackCastStructOver); + this->verify_commit_job(*unpacked_commit); + + // Simulate a trim of the prepare slot and re-verify the commit (which should be + // independent). + // + std::memset(prepare_storage.data(), 0xab, prepare_storage.size()); + prepare_storage.clear(); + + this->verify_commit_job(*unpacked_commit); + } + } } } } diff --git a/src/llfs/volume_recovery_visitor.cpp b/src/llfs/volume_job_recovery_visitor.cpp similarity index 74% rename from src/llfs/volume_recovery_visitor.cpp rename to src/llfs/volume_job_recovery_visitor.cpp index 05171da..be89f6b 100644 --- a/src/llfs/volume_recovery_visitor.cpp +++ b/src/llfs/volume_job_recovery_visitor.cpp @@ -6,7 +6,7 @@ // //+++++++++++-+-+--+----- --- -- - - - - -#include +#include // #include @@ -15,50 +15,51 @@ namespace llfs { //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -/*explicit*/ VolumeRecoveryVisitor::VolumeRecoveryVisitor( - VolumeReader::SlotVisitorFn&& slot_recovery_fn, VolumePendingJobsMap& pending_jobs) noexcept - : VolumeSlotDemuxer{std::move(slot_recovery_fn), pending_jobs} +/*explicit*/ VolumeJobRecoveryVisitor::VolumeJobRecoveryVisitor() noexcept { } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -StatusOr VolumeRecoveryVisitor::on_volume_attach(const SlotParse& /*slot*/, - const PackedVolumeAttachEvent& attach) +Status VolumeJobRecoveryVisitor::on_prepare_job( + const SlotParse& slot, const Ref& prepare) /*override*/ { - this->device_attachments.emplace(attach.id); - return None; + const auto [iter, inserted] = this->pending_jobs_.emplace( + slot.offset.lower_bound, SlotParseWithPayload>{ + slot, + prepare, + }); + + BATT_CHECK(inserted) << "Duplicate prepare job!" << BATT_INSPECT(slot); + + return OkStatus(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -StatusOr VolumeRecoveryVisitor::on_volume_detach(const SlotParse& /*slot*/, - const PackedVolumeDetachEvent& detach) +Status VolumeJobRecoveryVisitor::on_commit_job( + const SlotParse& /*slot*/, const Ref& commit) /*override*/ { - this->device_attachments.erase(detach.id); - return None; + this->pending_jobs_.erase(commit.get().prepare_slot_offset); + + return OkStatus(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -StatusOr VolumeRecoveryVisitor::on_volume_ids(const SlotParse& slot, - const PackedVolumeIds& ids) +Status VolumeJobRecoveryVisitor::on_rollback_job(const SlotParse& /*slot*/, + const PackedRollbackJob& rollback) /*override*/ { - this->ids = SlotWithPayload{ - .slot_range = slot.offset, - .payload = ids, - }; - return None; -} + this->pending_jobs_.erase(rollback.prepare_slot); -// TODO[tastolfi 2022-11-18] on_volume_trim; + return OkStatus(); +} //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -Status VolumeRecoveryVisitor::resolve_pending_jobs(PageCache& cache, PageRecycler& recycler, - const boost::uuids::uuid& volume_uuid, - TypedSlotWriter& slot_writer, - batt::Grant& grant) +Status VolumeJobRecoveryVisitor::resolve_pending_jobs( + PageCache& cache, PageRecycler& recycler, const boost::uuids::uuid& volume_uuid, + TypedSlotWriter& slot_writer) { Optional slot_upper_bound; @@ -115,11 +116,15 @@ Status VolumeRecoveryVisitor::resolve_pending_jobs(PageCache& cache, PageRecycle BATT_REQUIRE_OK(drop_status); - StatusOr rollback_slot = - slot_writer.append(grant, PackedRollbackJob{ - .prepare_slot = prepare_slot, - }); + const auto rollback_event = PackedRollbackJob{ + .prepare_slot = prepare_slot, + }; + + BATT_ASSIGN_OK_RESULT( + batt::Grant grant, + slot_writer.reserve(packed_sizeof_slot(rollback_event), batt::WaitForResource::kFalse)); + StatusOr rollback_slot = slot_writer.append(grant, rollback_event); BATT_REQUIRE_OK(rollback_slot); clamp_min_slot(&slot_upper_bound, rollback_slot->upper_bound); @@ -165,13 +170,17 @@ Status VolumeRecoveryVisitor::resolve_pending_jobs(PageCache& cache, PageRecycle BATT_REQUIRE_OK(commit_status); + const auto commit_event = CommitJob{ + .prepare_slot_offset = prepare_slot, + .packed_prepare = packed_prepare_job.pointer(), + }; + + BATT_ASSIGN_OK_RESULT(batt::Grant grant, slot_writer.reserve(packed_sizeof_slot(commit_event), + batt::WaitForResource::kFalse)); + // Now we can write the commit slot. // - StatusOr commit_slot = slot_writer.append(grant, PackedCommitJob{ - .reserved_ = {}, - .prepare_slot = prepare_slot, - }); - + StatusOr commit_slot = slot_writer.append(grant, commit_event); BATT_REQUIRE_OK(commit_slot); clamp_min_slot(&slot_upper_bound, commit_slot->upper_bound); diff --git a/src/llfs/volume_job_recovery_visitor.hpp b/src/llfs/volume_job_recovery_visitor.hpp new file mode 100644 index 0000000..c477d29 --- /dev/null +++ b/src/llfs/volume_job_recovery_visitor.hpp @@ -0,0 +1,66 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_JOB_RECOVERY_VISITOR_HPP +#define LLFS_VOLUME_JOB_RECOVERY_VISITOR_HPP + +#include +// +#include +#include +#include +#include +#include + +#include +#include + +namespace llfs { + +class VolumeJobRecoveryVisitor : public VolumeEventVisitor::NullImpl +{ + public: + explicit VolumeJobRecoveryVisitor() noexcept; + + //----- --- -- - - - - + + void reset_pending_jobs() + { + this->pending_jobs_.clear(); + } + + const VolumePendingJobsMap& get_pending_jobs() const noexcept + { + return this->pending_jobs_; + } + + Status resolve_pending_jobs(PageCache& cache, PageRecycler& recycler, + const boost::uuids::uuid& volume_uuid, + TypedSlotWriter& slot_writer); + + //----- --- -- - - - - + + Status on_prepare_job(const SlotParse&, const Ref&) override; + + Status on_commit_job(const SlotParse&, const Ref&) override; + + Status on_rollback_job(const SlotParse&, const PackedRollbackJob&) override; + + //+++++++++++-+-+--+----- --- -- - - - - + private: + // The pending jobs for this volume. + // + VolumePendingJobsMap pending_jobs_; +}; + +} // namespace llfs + +#endif // LLFS_VOLUME_JOB_RECOVERY_VISITOR_HPP + +#include diff --git a/src/llfs/volume_recovery_visitor.ipp b/src/llfs/volume_job_recovery_visitor.ipp similarity index 92% rename from src/llfs/volume_recovery_visitor.ipp rename to src/llfs/volume_job_recovery_visitor.ipp index 3544ac6..812382c 100644 --- a/src/llfs/volume_recovery_visitor.ipp +++ b/src/llfs/volume_job_recovery_visitor.ipp @@ -12,7 +12,7 @@ #include #include -#include +#include namespace llfs { diff --git a/src/llfs/volume_metadata.hpp b/src/llfs/volume_metadata.hpp new file mode 100644 index 0000000..c8319e6 --- /dev/null +++ b/src/llfs/volume_metadata.hpp @@ -0,0 +1,68 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_METADATA_HPP +#define LLFS_VOLUME_METADATA_HPP + +#include +// +#include +#include +#include + +#include + +namespace llfs { + +/** \brief Volume metadata that is written to the root log and refreshed on trim. + */ +struct VolumeMetadata { + //----- --- -- - - - - + + /** \brief Stores the slot offset of the most recent attach event appended to record an attachment + * associated with this volume. + */ + struct AttachInfo { + Optional last_refresh; + PackedVolumeAttachEvent event; + }; + + //+++++++++++-+-+--+----- --- -- - - - - + + static const usize kVolumeIdsGrantSize = + packed_sizeof_slot_with_payload_size(sizeof(PackedVolumeIds)); + + static const usize kAttachmentGrantSize = + packed_sizeof_slot_with_payload_size(sizeof(PackedVolumeAttachEvent)); + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief The uuids for this Volume. + */ + Optional ids; + + /** \brief The slot offset of the most recent refresh of `ids`. + */ + Optional ids_last_refresh; + + /** \brief The current set of attachments, with most recent slot (if present). + */ + std::unordered_map attachments; + + //+++++++++++-+-+--+----- --- -- - - - - + + u64 grant_target() const noexcept + { + return kVolumeIdsGrantSize + kAttachmentGrantSize * this->attachments.size(); + } +}; + +} //namespace llfs + +#endif // LLFS_VOLUME_METADATA_HPP diff --git a/src/llfs/volume_metadata_recovery_visitor.cpp b/src/llfs/volume_metadata_recovery_visitor.cpp new file mode 100644 index 0000000..7b4f6f3 --- /dev/null +++ b/src/llfs/volume_metadata_recovery_visitor.cpp @@ -0,0 +1,72 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ VolumeMetadataRecoveryVisitor::VolumeMetadataRecoveryVisitor( + VolumeMetadata& metadata) noexcept + : metadata_{metadata} +{ +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRecoveryVisitor::on_volume_ids(const SlotParse& slot, + const PackedVolumeIds& ids) /*override*/ +{ + this->metadata_.ids = ids; + if (this->metadata_.ids_last_refresh) { + this->ids_duplicated_ = true; + } + this->metadata_.ids_last_refresh = slot.offset.lower_bound; + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRecoveryVisitor::on_volume_attach( + const SlotParse& slot, const PackedVolumeAttachEvent& attach) /*override*/ +{ + VolumeMetadata::AttachInfo& attach_info = this->metadata_.attachments[attach.id]; + + if (attach_info.last_refresh) { + this->attachment_duplicated_.emplace(attach.id); + } + + attach_info = VolumeMetadata::AttachInfo{ + .last_refresh = slot.offset.lower_bound, + .event = attach, + }; + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRecoveryVisitor::on_volume_detach( + const SlotParse&, const PackedVolumeDetachEvent& detach) /*override*/ +{ + this->metadata_.attachments.erase(detach.id); + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize VolumeMetadataRecoveryVisitor::grant_byte_size_reclaimable_on_trim() const noexcept +{ + return (this->ids_duplicated_ ? VolumeMetadata::kVolumeIdsGrantSize : 0) + + (this->attachment_duplicated_.size() * VolumeMetadata::kAttachmentGrantSize); +} + +} //namespace llfs diff --git a/src/llfs/volume_metadata_recovery_visitor.hpp b/src/llfs/volume_metadata_recovery_visitor.hpp new file mode 100644 index 0000000..807fba2 --- /dev/null +++ b/src/llfs/volume_metadata_recovery_visitor.hpp @@ -0,0 +1,47 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_METADATA_RECOVERY_VISITOR_HPP +#define LLFS_VOLUME_METADATA_RECOVERY_VISITOR_HPP + +#include +// +#include +#include +#include + +namespace llfs { + +class VolumeMetadataRecoveryVisitor : public VolumeEventVisitor::NullImpl +{ + public: + explicit VolumeMetadataRecoveryVisitor(VolumeMetadata& metadata) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + // VolumeEventVisitor methods. + // + Status on_volume_attach(const SlotParse& slot, const PackedVolumeAttachEvent& attach) override; + + Status on_volume_detach(const SlotParse& slot, const PackedVolumeDetachEvent& detach) override; + + Status on_volume_ids(const SlotParse& slot, const PackedVolumeIds&) override; + // + //+++++++++++-+-+--+----- --- -- - - - - + + usize grant_byte_size_reclaimable_on_trim() const noexcept; + + private: + VolumeMetadata& metadata_; + bool ids_duplicated_ = false; + std::unordered_set attachment_duplicated_; +}; + +} //namespace llfs + +#endif // LLFS_VOLUME_METADATA_RECOVERY_VISITOR_HPP diff --git a/src/llfs/volume_metadata_refresher.cpp b/src/llfs/volume_metadata_refresher.cpp new file mode 100644 index 0000000..fe94855 --- /dev/null +++ b/src/llfs/volume_metadata_refresher.cpp @@ -0,0 +1,391 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +namespace llfs { + +/*explicit*/ VolumeMetadataRefresher::VolumeMetadataRefresher( + TypedSlotWriter& slot_writer, VolumeMetadata&& recovered) noexcept + : state_{slot_writer, std::move(recovered)} +{ +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Optional VolumeMetadataRefresher::ids_last_refresh_slot() const noexcept +{ + return this->state_.lock()->ids_last_refresh_slot(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Optional VolumeMetadataRefresher::attachment_last_refresh_slot( + const VolumeAttachmentId& attach_id) const noexcept +{ + return this->state_.lock()->attachment_last_refresh_slot(attach_id); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::grant_target() const noexcept +{ + return this->state_.lock()->grant_target(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::grant_size() const noexcept +{ + return this->state_.lock()->grant_size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::grant_required() const noexcept +{ + return this->state_.lock()->grant_required(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::update_grant_partial(batt::Grant& pool) noexcept +{ + return this->state_.lock()->update_grant_partial(pool); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::update_grant(batt::Grant& pool) noexcept +{ + return this->state_.lock()->update_grant(pool); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::add_attachment(VolumeAttachmentId id, + slot_offset_type user_slot_offset, + batt::Grant& grant) noexcept +{ + return this->state_.lock()->add_attachment(id, user_slot_offset, grant); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::remove_attachment(VolumeAttachmentId id) noexcept +{ + return this->state_.lock()->remove_attachment(id); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize VolumeMetadataRefresher::attachment_count() const noexcept +{ + return this->state_.lock()->attachment_count(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::invalidate(slot_offset_type slot_offset) noexcept +{ + return this->state_.lock()->invalidate(slot_offset); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +bool VolumeMetadataRefresher::needs_flush() const noexcept +{ + return this->state_.lock()->needs_flush(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::flush_grant_size() const noexcept +{ + return this->state_.lock()->flush_grant_size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr VolumeMetadataRefresher::flush() noexcept +{ + return this->state_.lock()->flush(); +} + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// class VolumeMetadataRefresher::State + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ VolumeMetadataRefresher::State::State(TypedSlotWriter& slot_writer, + VolumeMetadata&& metadata) noexcept + : slot_writer_{slot_writer} + , metadata_{std::move(metadata)} + , grant_{BATT_OK_RESULT_OR_PANIC(this->slot_writer_.reserve(0, batt::WaitForResource::kFalse))} +{ + BATT_CHECK(this->metadata_.ids); + + if (!this->metadata_.ids_last_refresh) { + this->ids_need_refresh_ = true; + } + + for (const auto& [attach_id, attach_info] : this->metadata_.attachments) { + if (!attach_info.last_refresh) { + this->attachments_needing_refresh_.emplace_back(attach_id); + } + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Optional VolumeMetadataRefresher::State::ids_last_refresh_slot() const noexcept +{ + return this->metadata_.ids_last_refresh; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Optional VolumeMetadataRefresher::State::attachment_last_refresh_slot( + const VolumeAttachmentId& attach_id) const noexcept +{ + auto iter = this->metadata_.attachments.find(attach_id); + if (iter == this->metadata_.attachments.end()) { + return None; + } + return iter->second.last_refresh; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::State::grant_target() const noexcept +{ + return this->metadata_.grant_target(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::State::grant_size() const noexcept +{ + return this->grant_.size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::State::grant_required() const noexcept +{ + const u64 target = this->grant_target(); + + return target - std::min(target, this->grant_size()); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::State::update_grant_partial(batt::Grant& pool) noexcept +{ + for (;;) { + StatusOr spent = pool.spend(std::min(pool.size(), this->grant_required()), + batt::WaitForResource::kFalse); + + if (spent.status() == batt::StatusCode::kGrantUnavailable) { + // retry... + continue; + } + if (spent.ok()) { + this->grant_.subsume(std::move(*spent)); + return OkStatus(); + } + return spent.status(); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::State::update_grant(batt::Grant& pool) noexcept +{ + StatusOr spent = pool.spend(this->grant_required(), batt::WaitForResource::kFalse); + BATT_REQUIRE_OK(spent); + + this->grant_.subsume(std::move(*spent)); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::State::add_attachment(VolumeAttachmentId attach_id, + slot_offset_type user_slot_offset, + batt::Grant& grant) noexcept +{ + if (this->metadata_.attachments.count(attach_id) != 0) { + return OkStatus(); + } + + BATT_ASSIGN_OK_RESULT(batt::Grant attachment_grant, + grant.spend(kAttachmentGrantSize, batt::WaitForResource::kFalse)); + + this->metadata_.attachments.emplace(attach_id, VolumeMetadata::AttachInfo{ + .last_refresh = None, + .event = PackedVolumeAttachEvent{{ + .id = attach_id, + .user_slot_offset = user_slot_offset, + }}, + }); + + this->attachments_needing_refresh_.emplace_back(attach_id); + + BATT_CHECK_OK(this->update_grant_partial(attachment_grant)); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::State::remove_attachment(VolumeAttachmentId attach_id) noexcept +{ + if (this->metadata_.attachments.count(attach_id) == 0) { + return OkStatus(); + } + + this->metadata_.attachments.erase(attach_id); + this->attachments_needing_refresh_.emplace_back(attach_id); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize VolumeMetadataRefresher::State::attachment_count() const noexcept +{ + return this->metadata_.attachments.size(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeMetadataRefresher::State::invalidate(slot_offset_type slot_offset) noexcept +{ + if (this->metadata_.ids_last_refresh && + slot_less_than(*this->metadata_.ids_last_refresh, slot_offset)) { + this->ids_need_refresh_ = true; + } + + for (const auto& [attach_id, attach_info] : this->metadata_.attachments) { + if (!attach_info.last_refresh) { + continue; + } + const slot_offset_type last_refresh_slot = *attach_info.last_refresh; + if (slot_less_than(last_refresh_slot, slot_offset)) { + this->attachments_needing_refresh_.emplace_back(attach_id); + } + } + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +bool VolumeMetadataRefresher::State::needs_flush() const noexcept +{ + return this->ids_need_refresh_ || !this->attachments_needing_refresh_.empty(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +u64 VolumeMetadataRefresher::State::flush_grant_size() const noexcept +{ + return (this->ids_need_refresh_ ? kVolumeIdsGrantSize : 0u) + + (this->attachments_needing_refresh_.size() * kAttachmentGrantSize); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr VolumeMetadataRefresher::State::flush() noexcept +{ + Optional lower_bound; + Optional upper_bound; + + const auto update_written_slot_range = [&](const SlotRange& slot_range) { + // Update the written slot range. + // + if (!lower_bound) { + lower_bound.emplace(slot_range.lower_bound); + } + clamp_min_slot(&upper_bound, slot_range.upper_bound); + }; + + //----- --- -- - - - - + // + if (this->ids_need_refresh_) { + LLFS_VLOG(1) << "Refreshing volume ids: " << this->metadata_.ids; + + StatusOr ids_refresh_slot = + this->slot_writer_.append(this->grant_, *this->metadata_.ids); + + BATT_REQUIRE_OK(ids_refresh_slot); + + // Mark ids as up-to-date. + // + this->ids_need_refresh_ = false; + this->metadata_.ids_last_refresh = ids_refresh_slot->lower_bound; + + update_written_slot_range(*ids_refresh_slot); + } + + //----- --- -- - - - - + // Refresh any attachments that are out-of-date. + // + usize attach_refresh_count = 0; + + // In any case, erase those attachments which we succeeded in updating. + // + auto on_scope_exit = batt::finally([&] { + this->attachments_needing_refresh_.erase( + this->attachments_needing_refresh_.begin(), + std::next(this->attachments_needing_refresh_.begin(), attach_refresh_count)); + }); + + for (const auto& attach_id : this->attachments_needing_refresh_) { + LLFS_VLOG(1) << "Refreshing attachment " << attach_id; + + auto iter = this->metadata_.attachments.find(attach_id); + const bool is_attached = iter != this->metadata_.attachments.end(); + + StatusOr slot_range = [&] { + if (is_attached) { + StatusOr new_slot_range = + this->slot_writer_.append(this->grant_, iter->second.event); + + if (new_slot_range.ok()) { + iter->second.last_refresh = new_slot_range->lower_bound; + } + return new_slot_range; + } + return this->slot_writer_.append( // + this->grant_, PackedVolumeDetachEvent{{ + .id = attach_id, + .user_slot_offset = this->slot_writer_.slot_offset(), + }}); + }(); + + BATT_REQUIRE_OK(slot_range); + + ++attach_refresh_count; + update_written_slot_range(*slot_range); + } + + if (!lower_bound) { + lower_bound = this->slot_writer_.slot_offset(); + } + if (!upper_bound) { + upper_bound = lower_bound; + } + + return {SlotRange{*lower_bound, *upper_bound}}; +} + +} //namespace llfs diff --git a/src/llfs/volume_metadata_refresher.hpp b/src/llfs/volume_metadata_refresher.hpp new file mode 100644 index 0000000..a944607 --- /dev/null +++ b/src/llfs/volume_metadata_refresher.hpp @@ -0,0 +1,170 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_METADATA_REFRESHER_HPP +#define LLFS_VOLUME_METADATA_REFRESHER_HPP + +#include +// +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include + +namespace llfs { + +/** \brief Tracks when the Volume metadata was last refreshed, and writes update slots to the log. + */ +class VolumeMetadataRefresher +{ + public: + static const usize kVolumeIdsGrantSize = VolumeMetadata::kVolumeIdsGrantSize; + static const usize kAttachmentGrantSize = VolumeMetadata::kAttachmentGrantSize; + + //+++++++++++-+-+--+----- --- -- - - - - + + explicit VolumeMetadataRefresher(TypedSlotWriter& slot_writer, + VolumeMetadata&& recovered) noexcept; + + VolumeMetadataRefresher(const VolumeMetadataRefresher&) = delete; + VolumeMetadataRefresher& operator=(const VolumeMetadataRefresher&) = delete; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns the most recent slot at which the volume ids were refreshed, or None if they + * are not currently in the log. + */ + Optional ids_last_refresh_slot() const noexcept; + + /** \brief Returns the most recent slot at which the given attachment was refreshed, or None if it + * is not currently in the log. + */ + Optional attachment_last_refresh_slot( + const VolumeAttachmentId& attach_id) const noexcept; + + /** \brief Returns the size that the refresh grant *should*, ideally, be. + */ + u64 grant_target() const noexcept; + + /** \brief Returns the actual refresh grant size. + */ + u64 grant_size() const noexcept; + + /** \brief Returns the number of grant bytes needed to reach `grant_target()`; this is always + * `this->grant_target() - this->grant_size()`. + */ + u64 grant_required() const noexcept; + + /** \brief Spends some of `pool`, if necessary, to bring the refresh grant up to where it should + * be. Differs from this->update_grant(pool) in that this function does not fail if it can only + * grab _some_ grant from `pool`. + */ + Status update_grant_partial(batt::Grant& pool) noexcept; + + /** \brief Spends some of `pool`, if necessary, to bring the refresh grant up to where it should + * be. + */ + Status update_grant(batt::Grant& pool) noexcept; + + /** \brief Adds a new attachment; does not append anything to the log (see flush). + */ + Status add_attachment(VolumeAttachmentId id, slot_offset_type user_slot_offset, + batt::Grant& grant) noexcept; + + /** \brief Removes an attachment; does not append anything to the log (see flush). + */ + Status remove_attachment(VolumeAttachmentId id) noexcept; + + /** \brief Returns the number of attachments being tracked by this object. + */ + usize attachment_count() const noexcept; + + /** \brief Marks anything that was last refreshed before the specified log offset as needing to be + * refreshed. + */ + Status invalidate(slot_offset_type slot_offset) noexcept; + + /** \brief Returns true iff metadata needs to be refreshed to the log. + */ + bool needs_flush() const noexcept; + + /** \brief The size of grant needed to call flush() successfully. + */ + u64 flush_grant_size() const noexcept; + + /** \brief Writes all pending updates to the log, returning the slot range of the update(s). + */ + StatusOr flush() noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + private: + class State + { + public: + explicit State(TypedSlotWriter& slot_writer, + VolumeMetadata&& recovered) noexcept; + + Optional ids_last_refresh_slot() const noexcept; + + Optional attachment_last_refresh_slot( + const VolumeAttachmentId& attach_id) const noexcept; + + u64 grant_target() const noexcept; + + u64 grant_size() const noexcept; + + u64 grant_required() const noexcept; + + Status update_grant_partial(batt::Grant& pool) noexcept; + + Status update_grant(batt::Grant& pool) noexcept; + + Status add_attachment(VolumeAttachmentId id, slot_offset_type user_slot_offset, + batt::Grant& grant) noexcept; + + Status remove_attachment(VolumeAttachmentId id) noexcept; + + usize attachment_count() const noexcept; + + Status invalidate(slot_offset_type slot_offset) noexcept; + + bool needs_flush() const noexcept; + + u64 flush_grant_size() const noexcept; + + StatusOr flush() noexcept; + + private: + TypedSlotWriter& slot_writer_; + + VolumeMetadata metadata_; + + bool ids_need_refresh_ = false; + + std::vector attachments_needing_refresh_; + + batt::Grant grant_; + }; + + //+++++++++++-+-+--+----- --- -- - - - - + + batt::Mutex state_; +}; + +} //namespace llfs + +#endif // LLFS_VOLUME_METADATA_REFRESHER_HPP diff --git a/src/llfs/volume_reader.cpp b/src/llfs/volume_reader.cpp index fd718fa..94a6178 100644 --- a/src/llfs/volume_reader.cpp +++ b/src/llfs/volume_reader.cpp @@ -29,7 +29,6 @@ namespace { mode)} , slot_reader_{*this->log_reader_} , paused_{true} - , pending_jobs_{} , trim_lock_update_lower_bound_{this->log_reader_->slot_offset() + this->volume_.options().trim_lock_update_interval} { diff --git a/src/llfs/volume_reader.ipp b/src/llfs/volume_reader.ipp index 06a4aa8..7da9d8e 100644 --- a/src/llfs/volume_reader.ipp +++ b/src/llfs/volume_reader.ipp @@ -39,7 +39,6 @@ class VolumeReader::Impl std::unique_ptr log_reader_; TypedSlotReader slot_reader_; bool paused_; - VolumePendingJobsMap pending_jobs_; slot_offset_type trim_lock_update_lower_bound_; }; @@ -68,8 +67,7 @@ inline StatusOr VolumeReader::visit_next(batt::WaitForResource wait_for_c return visitor_fn(slot, user_data); }; - VolumeSlotDemuxer demuxer{wrapped_visitor_fn, - this->impl_->pending_jobs_}; + VolumeSlotDemuxer demuxer{wrapped_visitor_fn}; StatusOr visited = this->impl_->slot_reader_.run(wait_for_commit, [&demuxer](auto&&... args) -> Status { @@ -101,8 +99,7 @@ inline StatusOr VolumeReader::consume_slots(batt::WaitForResource wait_fo return visitor_fn(BATT_FORWARD(args)...); }; - VolumeSlotDemuxer demuxer{wrapped_visitor_fn, - this->impl_->pending_jobs_}; + VolumeSlotDemuxer demuxer{wrapped_visitor_fn}; // SlotReader::run will block for as long as it can unpack and visit new slots from the log. // diff --git a/src/llfs/volume_recovery_visitor.hpp b/src/llfs/volume_recovery_visitor.hpp deleted file mode 100644 index 861f7f0..0000000 --- a/src/llfs/volume_recovery_visitor.hpp +++ /dev/null @@ -1,56 +0,0 @@ -//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ -// -// Part of the LLFS Project, under Apache License v2.0. -// See https://www.apache.org/licenses/LICENSE-2.0 for license information. -// SPDX short identifier: Apache-2.0 -// -//+++++++++++-+-+--+----- --- -- - - - - - -#pragma once -#ifndef LLFS_VOLUME_RECOVERY_VISITOR_HPP -#define LLFS_VOLUME_RECOVERY_VISITOR_HPP - -#include -#include -#include -#include - -#include -#include - -namespace llfs { - -class VolumeRecoveryVisitor : public VolumeSlotDemuxer -{ - public: - explicit VolumeRecoveryVisitor(VolumeReader::SlotVisitorFn&& slot_recovery_fn, - VolumePendingJobsMap& pending_jobs) noexcept; - - Status resolve_pending_jobs(PageCache& cache, PageRecycler& recycler, - const boost::uuids::uuid& volume_uuid, - TypedSlotWriter& slot_writer, batt::Grant& grant); - - StatusOr on_volume_attach(const SlotParse&, const PackedVolumeAttachEvent&) override; - - StatusOr on_volume_detach(const SlotParse&, const PackedVolumeDetachEvent&) override; - - StatusOr on_volume_ids(const SlotParse&, const PackedVolumeIds&) override; - - //+++++++++++-+-+--+----- --- -- - - - - - // public data - //+++++++++++-+-+--+----- --- -- - - - - - - // The device attachments for this volume. - // - std::unordered_set device_attachments; - - // The uuids for the volume. - // - Optional> ids; -}; - -} // namespace llfs - -#endif // LLFS_VOLUME_RECOVERY_VISITOR_HPP - -#include diff --git a/src/llfs/volume_slot_demuxer.hpp b/src/llfs/volume_slot_demuxer.hpp index ce9ede6..919f0bd 100644 --- a/src/llfs/volume_slot_demuxer.hpp +++ b/src/llfs/volume_slot_demuxer.hpp @@ -13,7 +13,6 @@ #include #include #include -#include #include namespace llfs { @@ -33,21 +32,11 @@ class VolumeSlotDemuxer : public VolumeEventVisitor> public: template explicit VolumeSlotDemuxer( - FnArg&& slot_visitor_fn, VolumePendingJobsMap& pending_jobs, + FnArg&& slot_visitor_fn, VolumeEventVisitor& base = VolumeEventVisitor::null_impl()) noexcept; //+++++++++++-+-+--+----- --- -- - - - - - void reset_pending_jobs() - { - this->pending_jobs_.clear(); - } - - const VolumePendingJobsMap& get_pending_jobs() const noexcept - { - return this->pending_jobs_; - } - // Tracks the pending jobs to make sure that no prepare slots are trimmed before the commit slot. // Optional get_safe_trim_pos() const; @@ -67,7 +56,7 @@ class VolumeSlotDemuxer : public VolumeEventVisitor> StatusOr on_prepare_job(const SlotParse&, const Ref&) override; - StatusOr on_commit_job(const SlotParse&, const PackedCommitJob&) override; + StatusOr on_commit_job(const SlotParse&, const Ref&) override; StatusOr on_rollback_job(const SlotParse&, const PackedRollbackJob&) override; @@ -100,10 +89,6 @@ class VolumeSlotDemuxer : public VolumeEventVisitor> // VolumeEventVisitor& base_; - // The pending jobs for this volume. - // - VolumePendingJobsMap& pending_jobs_; - // The upper bound offset of the last visited slot. // Optional visited_upper_bound_; diff --git a/src/llfs/volume_slot_demuxer.ipp b/src/llfs/volume_slot_demuxer.ipp index ba42b3e..3853c00 100644 --- a/src/llfs/volume_slot_demuxer.ipp +++ b/src/llfs/volume_slot_demuxer.ipp @@ -21,11 +21,9 @@ namespace llfs { template template /*explicit*/ VolumeSlotDemuxer::VolumeSlotDemuxer(FnArg&& slot_visitor_fn, - VolumePendingJobsMap& pending_jobs, VolumeEventVisitor& base) noexcept : visitor_fn_{BATT_FORWARD(slot_visitor_fn)} , base_{base} - , pending_jobs_{pending_jobs} , visited_upper_bound_{None} { initialize_status_codes(); @@ -36,14 +34,7 @@ template template Optional VolumeSlotDemuxer::get_safe_trim_pos() const { - if (this->pending_jobs_.empty()) { - return this->visited_upper_bound_; - } - const slot_offset_type oldest_pending_job_offset = this->pending_jobs_.begin()->first; - if (!this->visited_upper_bound_) { - return oldest_pending_job_offset; - } - return slot_min(oldest_pending_job_offset, *this->visited_upper_bound_); + return this->visited_upper_bound_; } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -75,24 +66,14 @@ StatusOr VolumeSlotDemuxer::on_prepare_job( LLFS_VLOG(1) << "on_prepare_job(" << BATT_INSPECT(slot) << ")"; - const auto [iter, inserted] = this->pending_jobs_.emplace( - slot.offset.lower_bound, SlotParseWithPayload>{ - slot, - prepare, - }); - - if (!inserted) { - return ::llfs::make_status(StatusCode::kDuplicatePrepareJob); - } - return this->base_.on_prepare_job(slot, prepare); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // template -StatusOr VolumeSlotDemuxer::on_commit_job(const SlotParse& slot, - const PackedCommitJob& commit) /*override*/ +StatusOr VolumeSlotDemuxer::on_commit_job( + const SlotParse& slot, const Ref& commit) /*override*/ { auto on_scope_exit = batt::finally([&] { this->mark_slot_visited(slot); @@ -100,25 +81,22 @@ StatusOr VolumeSlotDemuxer::on_commit_job(const SlotParse& slot, LLFS_VLOG(1) << "on_commit_job(" << BATT_INSPECT(slot) << ")"; - auto iter = this->pending_jobs_.find(commit.prepare_slot); - if (iter != this->pending_jobs_.end()) { - const SlotParse& commit_slot = slot; - const SlotParseWithPayload>& prepare_slot_with_payload = - iter->second; - const PackedPrepareJob* prepare_slot = prepare_slot_with_payload.payload.pointer(); - std::string_view user_data = raw_data_from_slot(prepare_slot_with_payload.slot, // - prepare_slot->user_data.get()); - this->pending_jobs_.erase(iter); - - // The user_slot must reference the job prepare slot so that data isn't trimmed too soon by user - // code. - // - SlotParse user_slot = commit_slot; - user_slot.depends_on_offset = prepare_slot_with_payload.slot.offset; - - Status status = this->visitor_fn_(user_slot, user_data); - BATT_REQUIRE_OK(status); - } + const SlotParse& commit_slot = slot; + const usize commit_slot_size = slot.size_in_bytes(); + std::string_view user_data = commit.get().user_data(); + + // The user_slot must reference the job prepare slot so that data isn't trimmed too soon by user + // code. + // + const auto user_slot = SlotParse{ + .offset = commit_slot.offset, + .body = commit_slot.body, + .total_grant_spent = commit.get().prepare_slot_size + commit_slot_size, + }; + + Status status = this->visitor_fn_(user_slot, user_data); + BATT_REQUIRE_OK(status); + return this->base_.on_commit_job(slot, commit); } @@ -134,8 +112,6 @@ StatusOr VolumeSlotDemuxer::on_rollback_job( LLFS_VLOG(1) << "on_rollback_job(" << BATT_INSPECT(slot) << ")"; - this->pending_jobs_.erase(rollback.prepare_slot); - return this->base_.on_rollback_job(slot, rollback); } @@ -196,8 +172,6 @@ StatusOr VolumeSlotDemuxer::on_volume_recovered( LLFS_VLOG(1) << "on_volume_recovered(" << BATT_INSPECT(slot) << ")"; - this->pending_jobs_.clear(); - return this->base_.on_volume_recovered(slot, recovered); } diff --git a/src/llfs/volume_trimmed_region_info.hpp b/src/llfs/volume_trimmed_region_info.hpp new file mode 100644 index 0000000..42ac880 --- /dev/null +++ b/src/llfs/volume_trimmed_region_info.hpp @@ -0,0 +1,49 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_TRIMMED_REGION_INFO_HPP +#define LLFS_VOLUME_TRIMMED_REGION_INFO_HPP + +#include +// +#include +#include + +#include + +namespace llfs { + +struct VolumeTrimmedRegionInfo { + /** \brief The slot offset range of the trimmed region. + */ + SlotRange slot_range; + + /** \brief The root page references in this trimmed region. + */ + std::vector obsolete_roots; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns whether a PackedVolumeTrimEvent needs to be written before trimming this + * region. + * + * PackedVolumeTrimEvent is only necesary when we are going to be using a PageCacheJob to drop + * root refs within the trimmed region. In this case, we must make sure that all page ref count + * updates are repeatable in case the trim is interrupted and needs to be completed during + * recovery. + */ + bool requires_trim_event_slot() const + { + return !this->obsolete_roots.empty(); + } +}; + +} //namespace llfs + +#endif // LLFS_VOLUME_TRIMMED_REGION_INFO_HPP diff --git a/src/llfs/volume_trimmed_region_visitor.cpp b/src/llfs/volume_trimmed_region_visitor.cpp new file mode 100644 index 0000000..964ad36 --- /dev/null +++ b/src/llfs/volume_trimmed_region_visitor.cpp @@ -0,0 +1,164 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr read_trimmed_region( + TypedSlotReader& slot_reader, VolumeMetadataRefresher& metadata_refresher, + HaveTrimEventGrant have_trim_event_grant, slot_offset_type trim_upper_bound) +{ + VolumeTrimmedRegionInfo trimmed_region; + trimmed_region.slot_range.lower_bound = slot_reader.next_slot_offset(); + trimmed_region.slot_range.upper_bound = trimmed_region.slot_range.lower_bound; + + VolumeTrimmedRegionVisitor visit_slot{trimmed_region, metadata_refresher, have_trim_event_grant}; + + bool reached_end = false; + + StatusOr read_status = slot_reader.run( + batt::WaitForResource::kTrue, [&](const SlotParse& slot, const auto& payload) -> Status { + const SlotRange& slot_range = slot.offset; + const u64 slot_size = slot.size_in_bytes(); + + const bool starts_before_trim_pos = // + slot_less_than(slot_range.lower_bound, trim_upper_bound); + + const bool ends_after_trim_pos = // + slot_less_than(trim_upper_bound, slot_range.upper_bound); + + const bool will_visit = starts_before_trim_pos && !ends_after_trim_pos; + + LLFS_VLOG(1) << "read slot: " << BATT_INSPECT(slot_range) << BATT_INSPECT(slot_size) + << BATT_INSPECT(starts_before_trim_pos) << BATT_INSPECT(ends_after_trim_pos) + << BATT_INSPECT(will_visit); + + if (!will_visit) { + reached_end = true; + return ::llfs::make_status(StatusCode::kBreakSlotReaderLoop); + } + + BATT_REQUIRE_OK(visit_slot(slot, payload)); + + trimmed_region.slot_range.upper_bound = + slot_max(trimmed_region.slot_range.upper_bound, slot_range.upper_bound); + + return OkStatus(); + }); + + if (!read_status.ok() && !(read_status.status() == StatusCode::kBreakSlotReaderLoop && + (reached_end || visit_slot.must_stop_and_trim()))) { + if (read_status.status() == StatusCode::kBreakSlotReaderLoop && + visit_slot.must_stop_and_trim()) { + BATT_CHECK_NE(trimmed_region.slot_range.lower_bound, trimmed_region.slot_range.upper_bound) + << "VolumeTrimmedRegionVisitor says must_stop_and_trim() == true, but the size of the " + "region to trim is 0! We are deadlocked."; + } + BATT_REQUIRE_OK(read_status); + } + + return {std::move(trimmed_region)}; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ VolumeTrimmedRegionVisitor::VolumeTrimmedRegionVisitor( + VolumeTrimmedRegionInfo& trimmed_region, VolumeMetadataRefresher& metadata_refresher, + HaveTrimEventGrant have_trim_event_grant) noexcept + : trimmed_region_{trimmed_region} + , metadata_refresher_{metadata_refresher} + , have_trim_event_grant_{have_trim_event_grant} +{ +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmedRegionVisitor::on_commit_job( + const SlotParse& slot, const Ref& commit_ref) /*override*/ +{ + const PackedCommitJob& commit = commit_ref.get(); + + LLFS_VLOG(1) << "VolumeTrimmedRegionVisitor::on_commit_job(slot=" << slot.offset << ");" + << BATT_INSPECT(slot.size_in_bytes()); + + if (commit.root_page_ids) { + if (!this->have_trim_event_grant_) { + this->must_stop_and_trim_ = true; + return ::llfs::make_status(StatusCode::kBreakSlotReaderLoop); + } + for (const PackedPageId& packed_page_id : *commit.root_page_ids) { + this->trimmed_region_.obsolete_roots.emplace_back(packed_page_id.unpack()); + } + } + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmedRegionVisitor::on_volume_attach( + const SlotParse& slot, const PackedVolumeAttachEvent& attach) /*override*/ +{ + this->update_metadata_flush_grant_needed( // + slot, this->metadata_refresher_.attachment_last_refresh_slot(attach.id), // + VolumeMetadata::kAttachmentGrantSize); + + return this->check_for_must_stop(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmedRegionVisitor::on_volume_detach( + const SlotParse& slot, const PackedVolumeDetachEvent& detach) /*override*/ +{ + this->update_metadata_flush_grant_needed( // + slot, this->metadata_refresher_.attachment_last_refresh_slot(detach.id), // + VolumeMetadata::kAttachmentGrantSize); + + return this->check_for_must_stop(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmedRegionVisitor::on_volume_ids(const SlotParse& slot, + const PackedVolumeIds&) /*override*/ +{ + this->update_metadata_flush_grant_needed( // + slot, this->metadata_refresher_.ids_last_refresh_slot(), // + VolumeMetadata::kVolumeIdsGrantSize); + + return this->check_for_must_stop(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmedRegionVisitor::update_metadata_flush_grant_needed( + const SlotParse& slot, const Optional last_refresh_slot, + usize slot_grant_size) noexcept +{ + if (last_refresh_slot && !slot_less_than(slot.offset.lower_bound, *last_refresh_slot)) { + this->metadata_flush_grant_needed_ += slot_grant_size; + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmedRegionVisitor::check_for_must_stop() noexcept +{ + if (this->metadata_flush_grant_needed_ > this->metadata_refresher_.grant_size()) { + this->must_stop_and_trim_ = true; + return ::llfs::make_status(StatusCode::kBreakSlotReaderLoop); + } + return OkStatus(); +} + +} //namespace llfs diff --git a/src/llfs/volume_trimmed_region_visitor.hpp b/src/llfs/volume_trimmed_region_visitor.hpp new file mode 100644 index 0000000..6d71b3c --- /dev/null +++ b/src/llfs/volume_trimmed_region_visitor.hpp @@ -0,0 +1,104 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_TRIMMED_REGION_VISITOR_HPP +#define LLFS_VOLUME_TRIMMED_REGION_VISITOR_HPP + +#include +// +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace llfs { + +BATT_STRONG_TYPEDEF(bool, HaveTrimEventGrant); + +/** \brief Reads slots from the passed reader, up to the given slot upper bound, collecting the + * information needed to trim the log. + */ +StatusOr read_trimmed_region( + TypedSlotReader& slot_reader, VolumeMetadataRefresher& metadata_refresher, + HaveTrimEventGrant have_trim_event_grant, slot_offset_type trim_upper_bound); + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +/** \brief Scans regions to be trimmed, saving information in a VolumeTrimmedRegionInfo struct. + */ +class VolumeTrimmedRegionVisitor : public VolumeEventVisitor::NullImpl +{ + public: + using Super = VolumeEventVisitor::NullImpl; + + //+++++++++++-+-+--+----- --- -- - - - - + + explicit VolumeTrimmedRegionVisitor(VolumeTrimmedRegionInfo& trimmed_region, + VolumeMetadataRefresher& metadata_refresher, + HaveTrimEventGrant have_trim_event_grant) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + // VolumeEventVisitor methods. + // + template + Status operator()(const SlotParse& slot, const T& event) + { + if (this->must_stop_and_trim()) { + return ::llfs::make_status(StatusCode::kBreakSlotReaderLoop); + } + return this->Super::operator()(slot, event); + } + + Status on_commit_job(const SlotParse&, const Ref&) override; + + Status on_volume_attach(const SlotParse& slot, const PackedVolumeAttachEvent& attach) override; + + Status on_volume_detach(const SlotParse& slot, const PackedVolumeDetachEvent& detach) override; + + Status on_volume_ids(const SlotParse& slot, const PackedVolumeIds&) override; + // + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns true iff this visitor stopped the scan early because we need to trim in order + * to replentish the metadata refresh grant (with the trimmed space) before going further. + */ + bool must_stop_and_trim() const noexcept + { + return this->must_stop_and_trim_; + } + + //+++++++++++-+-+--+----- --- -- - - - - + private: + void update_metadata_flush_grant_needed(const SlotParse& slot, + const Optional last_refresh_slot, + usize slot_grant_size) noexcept; + + Status check_for_must_stop() noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + + VolumeTrimmedRegionInfo& trimmed_region_; + + VolumeMetadataRefresher& metadata_refresher_; + + HaveTrimEventGrant have_trim_event_grant_; + + u64 metadata_flush_grant_needed_ = 0; + + bool must_stop_and_trim_ = false; +}; + +} //namespace llfs + +#endif // LLFS_VOLUME_TRIMMED_REGION_VISITOR_HPP diff --git a/src/llfs/volume_trimmer.cpp b/src/llfs/volume_trimmer.cpp index 5506a8e..b485144 100644 --- a/src/llfs/volume_trimmer.cpp +++ b/src/llfs/volume_trimmer.cpp @@ -11,25 +11,35 @@ #include #include +#include +#include #include namespace llfs { +namespace { + +const usize kTrimEventGrantSize = + packed_sizeof_slot_with_payload_size(sizeof(PackedVolumeTrimEvent)); + +} //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -/*static*/ VolumeDropRootsFn VolumeTrimmer::make_default_drop_roots_fn( - PageCache& cache, PageRecycler& recycler, const boost::uuids::uuid& trimmer_uuid) +/*static*/ VolumeDropRootsFn VolumeTrimmer::make_default_drop_roots_fn(PageCache& cache, + PageRecycler& recycler) { - return [&cache, &recycler, trimmer_uuid](slot_offset_type slot_offset, - Slice roots_to_trim) -> Status { + return [&cache, &recycler](const boost::uuids::uuid& trimmer_uuid, slot_offset_type slot_offset, + Slice roots_to_trim) -> Status { // Create a job for the root set ref count updates. // std::unique_ptr job = cache.new_job(); - LLFS_VLOG(1) << "Dropping PageId roots from the log..." << BATT_INSPECT(slot_offset); + LLFS_VLOG(1) << "Dropping PageId roots from the log..." << BATT_INSPECT(slot_offset) + << BATT_INSPECT(roots_to_trim.size()); + for (PageId page_id : roots_to_trim) { - LLFS_VLOG(1) << " -- " << page_id; + LLFS_VLOG(2) << " -- " << page_id; job->delete_root(page_id); } @@ -50,13 +60,104 @@ namespace llfs { //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -/*explicit*/ VolumeTrimmer::VolumeTrimmer(const boost::uuids::uuid& trimmer_uuid, - std::string&& name, SlotLockManager& trim_control, - TrimDelayByteCount trim_delay, - std::unique_ptr&& log_reader, - TypedSlotWriter& slot_writer, - VolumeDropRootsFn&& drop_roots, - const RecoveryVisitor& recovery_visitor) noexcept +/*static*/ StatusOr> VolumeTrimmer::recover( + const boost::uuids::uuid& trimmer_uuid, // + std::string&& name, // + TrimDelayByteCount trim_delay, // + LogDevice& volume_root_log, // + TypedSlotWriter& slot_writer, // + VolumeDropRootsFn&& drop_roots, // + SlotLockManager& trim_control, // + VolumeMetadataRefresher& metadata_refresher) +{ + // Recover VolumeTrimmer state. It is important that we do this only once all jobs have been + // resolved and metadata (such as attachments) has been appended. + // + std::unique_ptr log_reader = + volume_root_log.new_reader(/*slot_lower_bound=*/None, LogReadMode::kDurable); + + VolumeTrimmerRecoveryVisitor trimmer_visitor{/*trim_pos=*/log_reader->slot_offset()}; + { + TypedSlotReader slot_reader{*log_reader}; + StatusOr slots_read = slot_reader.run(batt::WaitForResource::kFalse, + [&trimmer_visitor](auto&&... args) -> Status { + BATT_REQUIRE_OK(trimmer_visitor(args...)); + return batt::OkStatus(); + }); + BATT_REQUIRE_OK(slots_read); + }; + + Optional trim_event_info = trimmer_visitor.get_trim_event_info(); + if (!trim_event_info) { + log_reader = volume_root_log.new_reader( + /*slot_lower_bound=*/None, LogReadMode::kDurable); + + } else { + LLFS_VLOG(1) << "[VolumeTrimmer::recover] resolving partial trim:" + << BATT_INSPECT(trim_event_info); + + // A pending VolumeTrimEventInfo record was recovered from the log. Scan the trimmed region and + // complete the trim before creating the VolumeTrimmer object. + // + log_reader = volume_root_log.new_reader( + /*slot_lower_bound=*/trim_event_info->trimmed_region_slot_range.lower_bound, + LogReadMode::kDurable); + + const slot_offset_type trim_upper_bound = + trim_event_info->trimmed_region_slot_range.upper_bound; + + TypedSlotReader slot_reader{*log_reader}; + + // have_trim_event_grant arg is set to true because, evidently, we had a trim event grant when + // this was written! + // + StatusOr trimmed_region_info = read_trimmed_region( + slot_reader, metadata_refresher, HaveTrimEventGrant{true}, trim_upper_bound); + + BATT_REQUIRE_OK(trimmed_region_info); + + //----- --- -- - - - - + // Metadata is refreshed *after* writing the trim event, so it has already been done! + //----- --- -- - - - - + + // Sanity check; the scanned slot range should be the same as it was when the trim event + // record was written. + // + BATT_CHECK_EQ(trim_event_info->trimmed_region_slot_range, trimmed_region_info->slot_range); + + // Normally the grant passed to `trim_volume_log` would subsume any portion of the trimmed log + // region necessary to pay for the next trim event, but since we're going to allocate a new + // grant inside VolumeTrimmer::VolumeTrimmer anyhow, this is just a formality. + // + // We preallocate the grant here because the trim even was already written, prior to recovery. + // + batt::Grant grant = + BATT_OK_RESULT_OR_PANIC(slot_writer.reserve(0, batt::WaitForResource::kFalse)); + + // Finish the partially completed trim. + // + BATT_REQUIRE_OK(trim_volume_log(trimmer_uuid, slot_writer, grant, std::move(*trim_event_info), + std::move(*trimmed_region_info), metadata_refresher, + drop_roots)); + } + + // The trim state is now clean! Create and return the VolumeTrimmer. + // + return std::make_unique(trimmer_uuid, std::move(name), trim_control, trim_delay, + std::move(log_reader), slot_writer, std::move(drop_roots), + metadata_refresher); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ VolumeTrimmer::VolumeTrimmer(const boost::uuids::uuid& trimmer_uuid, // + std::string&& name, // + SlotLockManager& trim_control, // + TrimDelayByteCount trim_delay, // + std::unique_ptr&& log_reader, // + TypedSlotWriter& slot_writer, // + VolumeDropRootsFn&& drop_roots_fn, // + VolumeMetadataRefresher& metadata_refresher) noexcept : trimmer_uuid_{trimmer_uuid} , name_{std::move(name)} , trim_control_{trim_control} @@ -64,16 +165,21 @@ namespace llfs { , log_reader_{std::move(log_reader)} , slot_reader_{*this->log_reader_} , slot_writer_{slot_writer} - , drop_roots_{std::move(drop_roots)} - , trimmer_grant_{BATT_OK_RESULT_OR_PANIC(this->slot_writer_.reserve( - (packed_sizeof_slot_with_payload_size(sizeof(PackedVolumeIds)) + - packed_sizeof_slot_with_payload_size(sizeof(PackedVolumeTrimEvent)) * 2 + - recovery_visitor.get_trimmer_grant_size()), - batt::WaitForResource::kFalse))} - , pending_jobs_{recovery_visitor.get_pending_jobs()} - , refresh_info_{recovery_visitor.get_refresh_info()} - , latest_trim_event_{recovery_visitor.get_trim_event_info()} + , drop_roots_{std::move(drop_roots_fn)} + , trimmer_grant_{BATT_OK_RESULT_OR_PANIC( + this->slot_writer_.reserve(0, batt::WaitForResource::kFalse))} + , metadata_refresher_{metadata_refresher} { + StatusOr init_grant = this->slot_writer_.reserve( + std::min(this->slot_writer_.pool_size(), kTrimEventGrantSize), + batt::WaitForResource::kFalse); + + BATT_CHECK_OK(init_grant) << BATT_INSPECT(kTrimEventGrantSize) + << BATT_INSPECT(this->slot_writer_.log_size()) + << BATT_INSPECT(this->slot_writer_.log_capacity()) + << BATT_INSPECT(this->slot_writer_.pool_size()); + + this->trimmer_grant_.subsume(std::move(*init_grant)); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -93,16 +199,6 @@ void VolumeTrimmer::halt() this->trimmer_grant_.revoke(); } -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -void VolumeTrimmer::push_grant(batt::Grant&& grant) noexcept -{ - LLFS_VLOG(2) << "trimmer grant: " << this->trimmer_grant_.size() << " -> " - << (this->trimmer_grant_.size() + grant.size()); - - this->trimmer_grant_.subsume(std::move(grant)); -} - //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // Status VolumeTrimmer::run() @@ -113,15 +209,6 @@ Status VolumeTrimmer::run() slot_offset_type least_upper_bound = trim_lower_bound + 1; usize bytes_trimmed = 0; - // If we found a pending trim event during recovery, use its trim upper bound first. - // - if (this->latest_trim_event_) { - BATT_CHECK_EQ(trim_lower_bound, - this->latest_trim_event_->trimmed_region_slot_range.lower_bound); - - least_upper_bound = this->latest_trim_event_->trimmed_region_slot_range.upper_bound; - } - // Main trimmer loop. // for (usize loop_counter = 0;; loop_counter += 1) { @@ -139,12 +226,6 @@ Status VolumeTrimmer::run() BATT_REQUIRE_OK(trim_upper_bound); - // If we are recovering a previously initiated trim, then limit the trim upper bound. - // - if (this->latest_trim_event_) { - *trim_upper_bound = this->latest_trim_event_->trimmed_region_slot_range.upper_bound; - } - // The next time we wait for a new trim target, it should be at least one past the previously // observed offset. // @@ -155,52 +236,87 @@ Status VolumeTrimmer::run() << BATT_INSPECT(this->trimmer_grant_.size()); //+++++++++++-+-+--+----- --- -- - - - - - // Scan the trimmed region if necessary. + // Scan the trimmed region. // - if (!this->trimmed_region_info_) { - StatusOr info = - read_trimmed_region(this->slot_reader_, *trim_upper_bound, this->pending_jobs_); - BATT_REQUIRE_OK(info); - this->trimmed_region_info_.emplace(std::move(*info)); - } - BATT_CHECK(this->trimmed_region_info_); + LLFS_VLOG(1) << "[VolumeTrimmer] read_trimmed_region"; + // + StatusOr trimmed_region_info = read_trimmed_region( + this->slot_reader_, this->metadata_refresher_, + HaveTrimEventGrant{this->trimmer_grant_.size() >= kTrimEventGrantSize}, *trim_upper_bound); + + BATT_REQUIRE_OK(trimmed_region_info); - if (this->trimmed_region_info_->slot_range.empty()) { + if (trimmed_region_info->slot_range.empty()) { LLFS_VLOG(1) << "Trimmed region too small; waiting for more to be trimmed," << BATT_INSPECT(least_upper_bound); - this->trimmed_region_info_ = batt::None; continue; } - //+++++++++++-+-+--+----- --- -- - - - - - // Make sure all Volume metadata has been refreshed. - // - BATT_REQUIRE_OK(refresh_volume_metadata(this->slot_writer_, this->trimmer_grant_, - this->refresh_info_, *this->trimmed_region_info_)); + Optional trim_event_info; - //+++++++++++-+-+--+----- --- -- - - - - - // Write a TrimEvent to the log if necessary. - // - if (this->trimmed_region_info_->requires_trim_event_slot() && !this->latest_trim_event_) { - BATT_ASSIGN_OK_RESULT(this->latest_trim_event_, - write_trim_event(this->slot_writer_, this->trimmer_grant_, - *this->trimmed_region_info_, this->pending_jobs_)); - BATT_CHECK(this->latest_trim_event_); + { + Optional sync_point; + + // Invalidate the new trim pos with the metadata refresher, since it might make the difference + // between writing a trim event and not. + // + LLFS_VLOG(1) << "[VolumeTrimmer] metadata_refresher.invalidate(" << *trim_upper_bound << ")" + << BATT_INSPECT(this->metadata_refresher_.grant_target()) + << BATT_INSPECT(this->metadata_refresher_.grant_required()) + << BATT_INSPECT(this->metadata_refresher_.grant_size()); + + BATT_REQUIRE_OK(this->metadata_refresher_.invalidate(*trim_upper_bound)); + + //+++++++++++-+-+--+----- --- -- - - - - + // Make sure all Volume metadata has been refreshed. + // + if (this->metadata_refresher_.needs_flush()) { + StatusOr metadata_slots = this->metadata_refresher_.flush(); + BATT_REQUIRE_OK(metadata_slots); + + clamp_min_slot(&sync_point, metadata_slots->upper_bound); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Write a TrimEvent to the log if necessary. + // IMPORTANT: this must come BEFORE refreshing metadata so that we don't refresh metadata, + // crash, then forget there was a trim; this could exhaust available grant! + // + if (trimmed_region_info->requires_trim_event_slot()) { + LLFS_VLOG(1) << "[VolumeTrimmer] write_trim_event"; + // + BATT_ASSIGN_OK_RESULT( + trim_event_info, + write_trim_event(this->slot_writer_, this->trimmer_grant_, *trimmed_region_info)); + + clamp_min_slot(&sync_point, trim_event_info->trim_event_slot.upper_bound); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Sync if necessary. + // + if (sync_point) { + LLFS_VLOG(1) << "Flushing trim event," << BATT_INSPECT(*sync_point) << ";" + << BATT_INSPECT(trimmed_region_info->slot_range); + // + BATT_REQUIRE_OK( + this->slot_writer_.sync(LogReadMode::kDurable, SlotUpperBoundAt{*sync_point})); + } } //+++++++++++-+-+--+----- --- -- - - - - // Trim the log. // - const slot_offset_type new_trim_target = this->trimmed_region_info_->slot_range.upper_bound; + const slot_offset_type new_trim_target = trimmed_region_info->slot_range.upper_bound; BATT_DEBUG_INFO("VolumeTrimmer -> trim_volume_log;" << BATT_INSPECT(this->name_)); + LLFS_VLOG(1) << "[VolumeTrimmer] trim_volume_log"; Status trim_result = trim_volume_log( - this->slot_writer_, this->trimmer_grant_, std::move(this->latest_trim_event_), - std::move(*this->trimmed_region_info_), this->drop_roots_, this->pending_jobs_); + this->trimmer_uuid_, this->slot_writer_, this->trimmer_grant_, std::move(trim_event_info), + std::move(*trimmed_region_info), this->metadata_refresher_, this->drop_roots_); - this->latest_trim_event_ = batt::None; - this->trimmed_region_info_ = batt::None; + this->trim_count_.fetch_add(1); BATT_REQUIRE_OK(trim_result); @@ -229,312 +345,33 @@ StatusOr VolumeTrimmer::await_trim_target(slot_offset_type min return new_trim_target; } - -//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -StatusOr read_trimmed_region( - TypedSlotReader& slot_reader, slot_offset_type trim_upper_bound, - VolumePendingJobsUMap& prior_pending_jobs) -{ - StatusOr result = VolumeTrimmedRegionInfo{}; - - result->slot_range.lower_bound = slot_reader.next_slot_offset(); - result->slot_range.upper_bound = result->slot_range.lower_bound; - - StatusOr read_status = slot_reader.run( - batt::WaitForResource::kTrue, - [trim_upper_bound, &result, &prior_pending_jobs](const SlotParse& slot, - const auto& payload) -> Status { - const SlotRange& slot_range = slot.offset; - - LLFS_VLOG(2) << "read slot: " << BATT_INSPECT(slot_range) - << BATT_INSPECT(!slot_less_than(slot_range.lower_bound, trim_upper_bound)) - << BATT_INSPECT(slot_less_than(trim_upper_bound, slot_range.upper_bound)); - - if (!slot_less_than(slot_range.lower_bound, trim_upper_bound) || - slot_less_than(trim_upper_bound, slot_range.upper_bound)) { - return ::llfs::make_status(StatusCode::kBreakSlotReaderLoop); - } - - LLFS_VLOG(2) << "visiting slot..."; - - result->slot_range.upper_bound = - slot_max(result->slot_range.upper_bound, slot_range.upper_bound); - - return batt::make_case_of_visitor( - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse& slot, const Ref& prepare) { - std::vector root_page_ids = - as_seq(*prepare.get().root_page_ids) | - seq::map([](const PackedPageId& packed) -> PageId { - return packed.as_page_id(); - }) | - seq::collect_vec(); - - LLFS_VLOG(1) << "visit_slot(" << BATT_INSPECT(slot.offset) - << ", PrepareJob) root_page_ids=" << batt::dump_range(root_page_ids); - - result->grant_size_to_release += packed_sizeof_slot(prepare.get()); - - const auto& [iter, inserted] = - result->pending_jobs.emplace(slot.offset.lower_bound, std::move(root_page_ids)); - if (!inserted) { - BATT_UNTESTED_LINE(); - LLFS_LOG_WARNING() - << "duplicate prepare job found at " << BATT_INSPECT(slot.offset); - } - - return OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse& slot, const PackedCommitJob& commit) { - LLFS_VLOG(2) << "visit_slot(" << BATT_INSPECT(slot.offset) << ", CommitJob)"; - - //----- --- -- - - - - - const auto extract_pending_job = [&](VolumePendingJobsUMap& from) -> bool { - auto iter = from.find(commit.prepare_slot); - if (iter == from.end()) { - return false; - } - - result->obsolete_roots.insert(result->obsolete_roots.end(), // - iter->second.begin(), iter->second.end()); - - from.erase(iter); - - return true; - }; - //----- --- -- - - - - - - result->grant_size_to_release += packed_sizeof_slot(commit); - - // Check the pending PrepareJob slots from before this trim. - // - if (extract_pending_job(prior_pending_jobs)) { - // Sanity check: if this commit's prepare slot is in _prior_ pending jobs, then the - // prepare slot offset should be before the old trim pos (otherwise we would be - // finding it in trimmed_pending_jobs_). - // - BATT_CHECK(slot_less_than(commit.prepare_slot, result->slot_range.lower_bound)) - << BATT_INSPECT(commit.prepare_slot) << BATT_INSPECT(result->slot_range); - - result->resolved_jobs.emplace_back(commit.prepare_slot); - - return batt::OkStatus(); - } - - // Check the current PrepareJob slots for a match. - // - if (extract_pending_job(result->pending_jobs)) { - return batt::OkStatus(); - } - - LLFS_LOG_WARNING() << "commit slot found for missing prepare: " - << BATT_INSPECT(commit.prepare_slot) << BATT_INSPECT(slot.offset); - - return batt::OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse&, const PackedRollbackJob& rollback) { - // The job has been resolved (rolled back); remove it from the maps. - // - LLFS_VLOG(1) << "Rolling back pending job;" << BATT_INSPECT(rollback.prepare_slot); - if (prior_pending_jobs.erase(rollback.prepare_slot) == 1u) { - result->resolved_jobs.emplace_back(rollback.prepare_slot); - } - result->pending_jobs.erase(rollback.prepare_slot); - - return batt::OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse& slot, const PackedVolumeIds& ids) { - LLFS_VLOG(1) << "Found ids to refresh: " << ids << BATT_INSPECT(slot.offset); - result->ids_to_refresh = ids; - return batt::OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse&, const PackedVolumeAttachEvent& attach) { - const auto& [iter, inserted] = - result->attachments_to_refresh.emplace(attach.id, attach); - if (!inserted) { - BATT_CHECK_EQ(iter->second.user_slot_offset, attach.user_slot_offset); - } - return batt::OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse&, const PackedVolumeDetachEvent& detach) { - result->attachments_to_refresh.erase(detach.id); - return batt::OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [&](const SlotParse&, const VolumeTrimEvent& trim_event) { - batt::make_copy(trim_event.trimmed_prepare_jobs) // - | batt::seq::for_each([&](const TrimmedPrepareJob& job) { - // If the pending job from this past trim event is *still* pending as of the - // start of the current trim job, then we transfer it to the - // pending jobs map and treat it like it was discovered in this - // trim. - // - auto iter = prior_pending_jobs.find(job.prepare_slot); - if (iter != prior_pending_jobs.end()) { - result->pending_jobs.emplace(iter->first, std::move(iter->second)); - prior_pending_jobs.erase(iter); - } - }); - - return batt::OkStatus(); - }, - - //+++++++++++-+-+--+----- --- -- - - - - - // - [](const SlotParse&, const auto& /*payload*/) { - return batt::OkStatus(); - } - //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - - )(slot, payload); - }); - - LLFS_VLOG(1) << "read_trimmed_region: done visiting slots," << BATT_INSPECT(read_status); - - if (!read_status.ok() && - read_status.status() != ::llfs::make_status(StatusCode::kBreakSlotReaderLoop)) { - BATT_REQUIRE_OK(read_status); - } - - return result; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status refresh_volume_metadata(TypedSlotWriter& slot_writer, batt::Grant& grant, - VolumeMetadataRefreshInfo& refresh_info, - VolumeTrimmedRegionInfo& trimmed_region) -{ - LLFS_VLOG(1) << "Refreshing Volume metadata"; - - Optional sync_point; - - // If the trimmed log segment contained a PackedVolumeIds slot, then refresh it before trimming. - // - if (trimmed_region.ids_to_refresh) { - LLFS_VLOG(1) << "Refreshing volume ids: " << *trimmed_region.ids_to_refresh; - - trimmed_region.grant_size_to_reserve += packed_sizeof_slot(*trimmed_region.ids_to_refresh); - - const slot_offset_type last_refresh_slot = - refresh_info.most_recent_ids_slot.value_or(trimmed_region.slot_range.lower_bound); - - if (slot_less_than(last_refresh_slot, trimmed_region.slot_range.upper_bound)) { - StatusOr id_refresh_slot = - slot_writer.append(grant, *trimmed_region.ids_to_refresh); - - BATT_REQUIRE_OK(id_refresh_slot); - - refresh_info.most_recent_ids_slot = id_refresh_slot->lower_bound; - clamp_min_slot(&sync_point, id_refresh_slot->upper_bound); - } - - LLFS_VLOG(1) << " -- " << BATT_INSPECT(sync_point); - } - - // Refresh any attachments that will be lost in this trim. - // - for (const auto& [attach_id, event] : trimmed_region.attachments_to_refresh) { - trimmed_region.grant_size_to_reserve += packed_sizeof_slot(event); - - // Skip this attachment if we know it has been refreshed at a higher slot. - // - { - auto iter = refresh_info.most_recent_attach_slot.find(attach_id); - if (iter != refresh_info.most_recent_attach_slot.find(attach_id)) { - if (!slot_less_than(iter->second, trimmed_region.slot_range.upper_bound)) { - continue; - } - } - } - - LLFS_VLOG(1) << "Refreshing attachment " << attach_id; - - StatusOr slot_range = slot_writer.append(grant, event); - BATT_REQUIRE_OK(slot_range); - - // Update the latest refresh slot for this attachment. - // - refresh_info.most_recent_attach_slot.emplace(attach_id, slot_range->lower_bound); - - clamp_min_slot(&sync_point, slot_range->upper_bound); - - LLFS_VLOG(1) << " -- " << BATT_INSPECT(sync_point); - } - - // Make sure all refreshed slots are flushed before returning. - // - if (sync_point) { - Status flush_status = [&] { - BATT_DEBUG_INFO("[VolumeTrimmer] SlotWriter::sync(SlotUpperBoundAt{" << *sync_point << "})"); - return slot_writer.sync(LogReadMode::kDurable, SlotUpperBoundAt{*sync_point}); - }(); - - BATT_REQUIRE_OK(flush_status); - } - - return batt::OkStatus(); -} +// class VolumeTrimmer +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // StatusOr write_trim_event(TypedSlotWriter& slot_writer, batt::Grant& grant, - VolumeTrimmedRegionInfo& trimmed_region, - VolumePendingJobsUMap& /*prior_pending_jobs*/) + const VolumeTrimmedRegionInfo& trimmed_region) { - // TODO [tastolfi 2023-05-22] use or remove `prior_pending_jobs` arg - - VolumeTrimEvent event; + VolumeTrimEvent trim_event; - event.old_trim_pos = trimmed_region.slot_range.lower_bound; - event.new_trim_pos = trimmed_region.slot_range.upper_bound; + trim_event.old_trim_pos = trimmed_region.slot_range.lower_bound; + trim_event.new_trim_pos = trimmed_region.slot_range.upper_bound; - event.committed_jobs = batt::as_seq(trimmed_region.resolved_jobs) // - | batt::seq::decayed() // - | batt::seq::boxed(); + const u64 trim_slot_size = packed_sizeof_slot(trim_event); - event.trimmed_prepare_jobs = - batt::as_seq(trimmed_region.pending_jobs.begin(), trimmed_region.pending_jobs.end()) // - | batt::seq::map([](const std::pair>& kvp) { - return TrimmedPrepareJob{ - .prepare_slot = kvp.first, - .page_ids = batt::as_seq(kvp.second) // - | batt::seq::decayed() // - | batt::seq::boxed(), - }; - }) // - | batt::seq::boxed(); + StatusOr event_grant = grant.spend(trim_slot_size, batt::WaitForResource::kFalse); + BATT_REQUIRE_OK(event_grant) << BATT_INSPECT(trim_slot_size) << BATT_INSPECT(grant.size()); - trimmed_region.grant_size_to_reserve += packed_sizeof_slot(event); - - StatusOr result = slot_writer.append(grant, std::move(event)); + const u64 event_grant_size_before = event_grant->size(); + // + StatusOr result = slot_writer.append(*event_grant, std::move(trim_event)); BATT_REQUIRE_OK(result); - - LLFS_VLOG(1) << "Flushing trim event at slot_range=" << *result - << BATT_INSPECT(trimmed_region.slot_range); - - BATT_REQUIRE_OK(slot_writer.sync(LogReadMode::kDurable, SlotUpperBoundAt{result->upper_bound})); + // + const u64 event_grant_size_after = event_grant->size(); + BATT_CHECK_EQ(event_grant_size_before - event_grant_size_after, trim_slot_size); return VolumeTrimEventInfo{ .trim_event_slot = *result, @@ -544,27 +381,33 @@ StatusOr write_trim_event(TypedSlotWriter& slot_writer, batt::Grant& grant, +Status trim_volume_log(const boost::uuids::uuid& trimmer_uuid, + TypedSlotWriter& slot_writer, batt::Grant& grant, Optional&& trim_event, VolumeTrimmedRegionInfo&& trimmed_region, - const VolumeDropRootsFn& drop_roots, - VolumePendingJobsUMap& prior_pending_jobs) + VolumeMetadataRefresher& metadata_refresher, + const VolumeDropRootsFn& drop_roots) { - BATT_CHECK_IMPLIES(!trim_event, trimmed_region.obsolete_roots.empty()); + LLFS_VLOG(1) << "trim_volume_log()" << BATT_INSPECT(trimmed_region.slot_range); + + auto on_scope_exit = batt::finally([&] { + BATT_CHECK_LE(grant.size(), kTrimEventGrantSize); + }); // If we found some obsolete jobs in the newly trimmed log segment, then collect up all root set // ref counts to release. // if (!trimmed_region.obsolete_roots.empty()) { + BATT_CHECK(trim_event); + std::vector roots_to_trim; std::swap(roots_to_trim, trimmed_region.obsolete_roots); - - LLFS_VLOG(1) << "trim_volume_log()" << BATT_INSPECT(trimmed_region.slot_range); - - BATT_REQUIRE_OK(drop_roots(trim_event->trim_event_slot.lower_bound, as_slice(roots_to_trim))); + BATT_REQUIRE_OK( + drop_roots(trimmer_uuid, trim_event->trim_event_slot.lower_bound, as_slice(roots_to_trim))); } - // ** IMPORTANT ** It is only safe to trim the log after `commit(PageCacheJob, ...)` returns; - // otherwise we will lose information about which root set page references to remove! + // ** IMPORTANT ** It is only safe to trim the log after `commit(PageCacheJob, ...)` (which is + // called inside `drop_roots`) returns; otherwise we will lose information about which root set + // page references to remove! StatusOr trimmed_space = [&slot_writer, &trimmed_region] { BATT_DEBUG_INFO("[VolumeTrimmer] SlotWriter::trim_and_reserve(" @@ -573,195 +416,27 @@ Status trim_volume_log(TypedSlotWriter& slot_writer, batt::G }(); BATT_REQUIRE_OK(trimmed_space); - // Balance the books. + // Take some of the trimmed space and retain it to cover the trim event that was just written. // - { - // Because grant_size_to_reserve and grant_size_to_release are counted against each other, we - // can cancel out the smaller of the two from both. - // - const usize common_size = std::min(trimmed_region.grant_size_to_reserve, // - trimmed_region.grant_size_to_release); - - trimmed_region.grant_size_to_reserve -= common_size; - trimmed_region.grant_size_to_release -= common_size; - } - - if (trimmed_region.grant_size_to_reserve > 0) { - usize reserve_size = 0; - std::swap(reserve_size, trimmed_region.grant_size_to_reserve); - StatusOr reserved = - trimmed_space->spend(reserve_size, batt::WaitForResource::kFalse); - BATT_REQUIRE_OK(reserved) << BATT_INSPECT(reserve_size) << BATT_INSPECT(trimmed_space->size()); + if (trim_event && grant.size() < kTrimEventGrantSize) { + StatusOr trim_event_grant = + trimmed_space->spend(kTrimEventGrantSize - grant.size(), batt::WaitForResource::kFalse); - grant.subsume(std::move(*reserved)); - } + BATT_REQUIRE_OK(trim_event_grant) + << BATT_INSPECT(slot_writer.log_size()) << BATT_INSPECT(slot_writer.log_capacity()) + << BATT_INSPECT(slot_writer.pool_size()) << BATT_INSPECT(grant.size()); - if (trimmed_region.grant_size_to_release > 0) { - usize release_size = 0; - std::swap(release_size, trimmed_region.grant_size_to_release); - StatusOr released = grant.spend(release_size, batt::WaitForResource::kFalse); - BATT_REQUIRE_OK(released); + grant.subsume(std::move(*trim_event_grant)); } - // Move pending jobs from the trimmed region to `prior_pending_jobs` - // - prior_pending_jobs.insert(std::make_move_iterator(trimmed_region.pending_jobs.begin()), - std::make_move_iterator(trimmed_region.pending_jobs.end())); - - return batt::OkStatus(); -} - -//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ - -/*explicit*/ VolumeTrimmer::RecoveryVisitor::RecoveryVisitor(slot_offset_type trim_pos) noexcept - : log_trim_pos_{trim_pos} -{ - LLFS_VLOG(1) << "RecoveryVisitor created;" << BATT_INSPECT(trim_pos); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_raw_data(const SlotParse&, - const Ref&) /*override*/ -{ - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_prepare_job( - const SlotParse& slot, const Ref& prepare) /*override*/ -{ - const usize slot_size = packed_sizeof_slot(prepare.get()); - - this->trimmer_grant_size_ += slot_size; - - LLFS_VLOG(1) << "RecoveryVisitor::on_prepare_job(slot=" << slot.offset << "); trimmer_grant_size " - << (this->trimmer_grant_size_ - slot_size) << " -> " << this->trimmer_grant_size_; - - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_commit_job(const SlotParse& slot, - const PackedCommitJob& commit) /*override*/ -{ - const usize slot_size = packed_sizeof_slot(commit); - - this->trimmer_grant_size_ += slot_size; - - LLFS_VLOG(1) << "RecoveryVisitor::on_commit_job(slot=" << slot.offset << "); trimmer_grant_size " - << (this->trimmer_grant_size_ - slot_size) << " -> " << this->trimmer_grant_size_; - - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_rollback_job(const SlotParse&, - const PackedRollbackJob&) /*override*/ -{ - // TODO [tastolfi 2022-11-23] Figure out whether we need to do anything here to avoid leaking log - // grant... - // - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_volume_attach( - const SlotParse& slot, const PackedVolumeAttachEvent& attach) /*override*/ -{ - this->refresh_info_.most_recent_attach_slot[attach.id] = slot.offset.lower_bound; - - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_volume_detach( - const SlotParse& slot, const PackedVolumeDetachEvent& detach) /*override*/ -{ - this->refresh_info_.most_recent_attach_slot[detach.id] = slot.offset.lower_bound; - - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_volume_ids(const SlotParse& slot, - const PackedVolumeIds&) /*override*/ -{ - LLFS_VLOG(1) << "RecoveryVisitor::on_volume_ids(slot=" << slot.offset << ")"; - - this->refresh_info_.most_recent_ids_slot = slot.offset.lower_bound; - - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_volume_recovered( - const SlotParse&, const PackedVolumeRecovered&) /*override*/ -{ - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_volume_format_upgrade( - const SlotParse&, const PackedVolumeFormatUpgrade&) /*override*/ -{ - return batt::OkStatus(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -Status VolumeTrimmer::RecoveryVisitor::on_volume_trim( - const SlotParse& slot, const VolumeTrimEvent& trim_event) /*override*/ -{ - const bool is_pending = slot_less_than(this->log_trim_pos_, trim_event.new_trim_pos); - - LLFS_VLOG(1) << "RecoveryVisitor::on_volume_trim(slot=" << slot.offset << ") trimmed_region == " - << SlotRange{trim_event.old_trim_pos, trim_event.new_trim_pos} - << BATT_INSPECT(is_pending); - - this->trimmer_grant_size_ += packed_sizeof_slot(trim_event); - - if (is_pending) { - if (this->trim_event_info_ != None) { - LLFS_LOG_WARNING() << "Multiple pending trim events found! Likely corrupted log..."; - // - // TODO [tastolfi 2022-11-28] Fail recovery? - } - - this->trim_event_info_.emplace(); - this->trim_event_info_->trim_event_slot = slot.offset; - this->trim_event_info_->trimmed_region_slot_range = SlotRange{ - trim_event.old_trim_pos, - trim_event.new_trim_pos, - }; - } else { - // If the trim is pending (branch above), then the trimmer will rescan the trimmed region and - // recover the information in the trim event. Otherwise, we need to apply this information - // here. + if (grant.size() >= kTrimEventGrantSize) { + // Use as much of the trimmed space as necessary to restore the metadata refresher's grant. // - batt::make_copy(trim_event.trimmed_prepare_jobs) | - batt::seq::for_each([this](const TrimmedPrepareJob& job) { - auto page_ids = batt::make_copy(job.page_ids) | batt::seq::collect_vec(); - - LLFS_VLOG(1) << " -- " << BATT_INSPECT(job.prepare_slot) << "," - << BATT_INSPECT_RANGE(page_ids); - - this->pending_jobs_.emplace(job.prepare_slot, std::move(page_ids)); - }); + LLFS_VLOG(1) << "trim_volume_log(): updating metadata refresher grant;" + << BATT_INSPECT(metadata_refresher.grant_required()) + << BATT_INSPECT(trimmed_space->size()); - batt::make_copy(trim_event.committed_jobs) | - batt::seq::for_each([this](slot_offset_type prepare_slot) { - LLFS_VLOG(1) << " -- commit{prepare_slot=" << prepare_slot << "}"; - this->pending_jobs_.erase(prepare_slot); - }); + BATT_REQUIRE_OK(metadata_refresher.update_grant_partial(*trimmed_space)); } return batt::OkStatus(); diff --git a/src/llfs/volume_trimmer.hpp b/src/llfs/volume_trimmer.hpp index 5e38923..053aba2 100644 --- a/src/llfs/volume_trimmer.hpp +++ b/src/llfs/volume_trimmer.hpp @@ -10,6 +10,8 @@ #ifndef LLFS_VOLUME_TRIMMER_HPP #define LLFS_VOLUME_TRIMMER_HPP +#include +// #include #include #include @@ -17,6 +19,8 @@ #include #include #include +#include +#include #include #include @@ -24,75 +28,11 @@ namespace llfs { -using VolumePendingJobsUMap = - std::unordered_map>; - -struct VolumeTrimmedRegionInfo { - SlotRange slot_range; - std::vector resolved_jobs; - VolumePendingJobsUMap pending_jobs; - std::vector obsolete_roots; - Optional ids_to_refresh; - std::unordered_map - attachments_to_refresh; - usize grant_size_to_release = 0; - usize grant_size_to_reserve = 0; - - //+++++++++++-+-+--+----- --- -- - - - - - - bool requires_trim_event_slot() const - { - return !this->resolved_jobs.empty() || !this->obsolete_roots.empty() || - !this->pending_jobs.empty(); - } -}; - /** \brief Consumer of dropped root page refs. */ -using VolumeDropRootsFn = std::function)>; - -/** \brief Reads slots from the passed reader, up to the given slot upper bound, collecting the - * information needed to trim the log. - */ -StatusOr read_trimmed_region( - TypedSlotReader& slot_reader, slot_offset_type upper_bound, - VolumePendingJobsUMap& prior_pending_jobs); - -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -/** \brief Tracks when the Volume metadata was last refreshed. - */ -struct VolumeMetadataRefreshInfo { - using AttachSlotMap = - std::unordered_map; - - Optional most_recent_ids_slot; - AttachSlotMap most_recent_attach_slot; -}; - -inline bool operator==(const VolumeMetadataRefreshInfo& l, const VolumeMetadataRefreshInfo& r) -{ - return l.most_recent_ids_slot == r.most_recent_ids_slot // - && l.most_recent_attach_slot == r.most_recent_attach_slot; -} - -inline bool operator!=(const VolumeMetadataRefreshInfo& l, const VolumeMetadataRefreshInfo& r) -{ - return !(l == r); -} - -inline std::ostream& operator<<(std::ostream& out, const VolumeMetadataRefreshInfo& t) -{ - return out << "{.most_recent_ids_slot=" << t.most_recent_ids_slot - << ", .most_recent_attach_slot=" << batt::dump_range(t.most_recent_attach_slot) - << ",}"; -} - -/** \brief Appends any Volume metadata that will be lost when trimmed_region is trimmed to the end - * of the log using the passed grant. - */ -Status refresh_volume_metadata(TypedSlotWriter& slot_writer, batt::Grant& grant, - VolumeMetadataRefreshInfo& refresh_info, - VolumeTrimmedRegionInfo& trimmed_region); +using VolumeDropRootsFn = std::function root_ref_page_ids)>; //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- /** \brief Information about a durably committed PackedVolumeTrimEvent. @@ -105,23 +45,28 @@ struct VolumeTrimEventInfo { inline std::ostream& operator<<(std::ostream& out, const VolumeTrimEventInfo& t) { return out << "{.trim_event_slot=" << t.trim_event_slot - << ", .trimmed_region_slot_range=" << t.trimmed_region_slot_range << ",}"; + << ", .trimmed_region_slot_range=" << t.trimmed_region_slot_range + << "(size=" << t.trimmed_region_slot_range.size() << "),}"; } /** \brief Writes a trim event slot to the volume log. */ StatusOr write_trim_event(TypedSlotWriter& slot_writer, batt::Grant& grant, - VolumeTrimmedRegionInfo& trimmed_region, - VolumePendingJobsUMap& prior_pending_jobs); + const VolumeTrimmedRegionInfo& trimmed_region); /** \brief Decrement ref counts of obsolete roots in the given trimmed region and trim the log. */ -Status trim_volume_log(TypedSlotWriter& slot_writer, batt::Grant& grant, +Status trim_volume_log(const boost::uuids::uuid& trimmer_uuid, + TypedSlotWriter& slot_writer, batt::Grant& grant, Optional&& trim_event, VolumeTrimmedRegionInfo&& trimmed_region, - const VolumeDropRootsFn& drop_roots, - VolumePendingJobsUMap& prior_pending_jobs); + VolumeMetadataRefresher& metadata_refresher, + const VolumeDropRootsFn& drop_roots); + +// Forward-declaration. +// +class VolumeTrimmerRecoveryVisitor; //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- /** \brief Runs in the background, trimming a single Volume's main log as needed. @@ -129,23 +74,32 @@ Status trim_volume_log(TypedSlotWriter& slot_writer, batt::G class VolumeTrimmer { public: - /** \brief Reconstructs trimmer state during crash recovery. - */ - class RecoveryVisitor; - //+++++++++++-+-+--+----- --- -- - - - - - static VolumeDropRootsFn make_default_drop_roots_fn(PageCache& cache, PageRecycler& recycler, - const boost::uuids::uuid& trimmer_uuid); + /** \brief Creates and returns a function to drop trimmed page root refs. + */ + static VolumeDropRootsFn make_default_drop_roots_fn(PageCache& cache, PageRecycler& recycler); + + static StatusOr> recover( + const boost::uuids::uuid& trimmer_uuid, // + std::string&& name, // + TrimDelayByteCount trim_delay, // + LogDevice& volume_root_log, // + TypedSlotWriter& slot_writer, // + VolumeDropRootsFn&& drop_roots, // + SlotLockManager& trim_control, // + VolumeMetadataRefresher& metadata_refresher); //+++++++++++-+-+--+----- --- -- - - - - - explicit VolumeTrimmer(const boost::uuids::uuid& trimmer_uuid, std::string&& name, - SlotLockManager& trim_control, TrimDelayByteCount trim_delay, - std::unique_ptr&& log_reader, - TypedSlotWriter& slot_writer, - VolumeDropRootsFn&& drop_roots, - const RecoveryVisitor& recovery_visitor) noexcept; + explicit VolumeTrimmer(const boost::uuids::uuid& trimmer_uuid, // + std::string&& name, // + SlotLockManager& trim_control, // + TrimDelayByteCount trim_delay, // + std::unique_ptr&& log_reader, // + TypedSlotWriter& slot_writer, // + VolumeDropRootsFn&& drop_roots_fn, // + VolumeMetadataRefresher& metadata_refresher) noexcept; VolumeTrimmer(const VolumeTrimmer&) = delete; VolumeTrimmer& operator=(const VolumeTrimmer&) = delete; @@ -162,15 +116,20 @@ class VolumeTrimmer return this->name_; } - /** \brief Adds the given grant to the trim event grant held by this object, which is used to - * append the log. - */ - void push_grant(batt::Grant&& grant) noexcept; - void halt(); Status run(); + u64 grant_pool_size() const noexcept + { + return this->trimmer_grant_.size(); + } + + u64 trim_count() const noexcept + { + return this->trim_count_.load(); + } + //+++++++++++-+-+--+----- --- -- - - - - private: @@ -229,91 +188,14 @@ class VolumeTrimmer */ std::atomic halt_requested_{false}; - /** \brief Contains all PrepareJobs that haven't been resolved yet by a corresponding commit or - * rollback. + /** \brief Contains the last known slot(s) where Volume metadata (ids and attachments) was + * refreshed, and takes care of refreshing this information on trim. */ - VolumePendingJobsUMap pending_jobs_; + VolumeMetadataRefresher& metadata_refresher_; - /** \brief Contains the last known slot where Volume metadata (ids and attachments) was refreshed. + /** \brief The number of trim operations completed. */ - VolumeMetadataRefreshInfo refresh_info_; - - /** \brief When present, contains information read from the log region currently being trimmed. - */ - Optional trimmed_region_info_; - - /** \brief When present, contains information about the most recent durable TrimEvent slot. - */ - Optional latest_trim_event_; -}; - -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -// -class VolumeTrimmer::RecoveryVisitor : public VolumeEventVisitor -{ - public: - explicit RecoveryVisitor(slot_offset_type trim_pos) noexcept; - - //+++++++++++-+-+--+----- --- -- - - - - - // VolumeEventVisitor methods. - // - Status on_raw_data(const SlotParse&, const Ref&) override; - - Status on_prepare_job(const SlotParse&, const Ref&) override; - - Status on_commit_job(const SlotParse&, const PackedCommitJob&) override; - - Status on_rollback_job(const SlotParse&, const PackedRollbackJob&) override; - - Status on_volume_attach(const SlotParse& slot, const PackedVolumeAttachEvent& attach) override; - - Status on_volume_detach(const SlotParse& slot, const PackedVolumeDetachEvent& detach) override; - - Status on_volume_ids(const SlotParse& slot, const PackedVolumeIds&) override; - - Status on_volume_recovered(const SlotParse&, const PackedVolumeRecovered&) override; - - Status on_volume_format_upgrade(const SlotParse&, const PackedVolumeFormatUpgrade&) override; - - Status on_volume_trim(const SlotParse&, const VolumeTrimEvent&) override; - // - //+++++++++++-+-+--+----- --- -- - - - - - - /** \brief Returns the current last-known refresh information for all Volume metadata. - */ - const VolumeMetadataRefreshInfo& get_refresh_info() const noexcept - { - return this->refresh_info_; - } - - /** \brief Returns information about the most recent trim event slot. - */ - const Optional& get_trim_event_info() const noexcept - { - return this->trim_event_info_; - } - - /** \brief Returns the page transaction jobs that have been trimmed but not yet resolved, indexed - * by their prepare slot. - */ - const VolumePendingJobsUMap& get_pending_jobs() const noexcept - { - return this->pending_jobs_; - } - - /** \brief Returns the size of the grant required by the VolumeTrimmer task. - */ - usize get_trimmer_grant_size() const noexcept - { - return this->trimmer_grant_size_; - } - - private: - slot_offset_type log_trim_pos_; - VolumeMetadataRefreshInfo refresh_info_; - Optional trim_event_info_; - VolumePendingJobsUMap pending_jobs_; - usize trimmer_grant_size_ = 0; + std::atomic trim_count_{0}; }; } // namespace llfs diff --git a/src/llfs/volume_trimmer.test.cpp b/src/llfs/volume_trimmer.test.cpp index 89d550a..af67816 100644 --- a/src/llfs/volume_trimmer.test.cpp +++ b/src/llfs/volume_trimmer.test.cpp @@ -13,15 +13,19 @@ #include #include +#include + #include #include -#include #include +#include #include #include #include +#include +#include #include namespace { @@ -54,734 +58,1124 @@ using llfs::testing::FakeLogDevice; // - 4 pages constexpr usize kNumPages = 4; -constexpr usize kLogSize = 64 * kKiB; +constexpr usize kLogSize = 4 * kKiB; constexpr usize kMinTTF = 5; -constexpr usize kMaxTTF = 500; -constexpr usize kMinOpaqueDataSize = 10; -constexpr usize kMaxOpaqueDataSize = 1000; +constexpr usize kMaxTTF = 30; +constexpr usize kMinOpaqueDataSize = 1; +constexpr usize kMaxOpaqueDataSize = 100; constexpr usize kMaxSlotOverhead = 32; +constexpr int kStepLimit = 100 * 1000; +constexpr usize kDefaultPageIdsVecSize = 16; //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- // class VolumeTrimmerTest : public ::testing::Test { public: - VolumeTrimmerTest() - { - } + struct Config { + const llfs::PackedVolumeIds volume_ids{ + .main_uuid = llfs::random_uuid(), + .recycler_uuid = llfs::random_uuid(), + .trimmer_uuid = llfs::random_uuid(), + }; // ---- + }; - /** \brief Resets all test state variables to their initial values. - */ - void reset_state() - { - this->recovery_visitor = batt::None; - this->page_ref_count.fill(0); - this->pending_jobs.clear(); - this->committed_jobs.clear(); - this->job_grant = batt::None; - this->job_grant_size = 0; - this->leaked_job_grant_size = 0; - this->verified_client_slot = 0; - this->trim_delay_byte_count = 0; - this->last_recovered_trim_pos = 0; - this->initialize_log(); - } + struct JobInfo { + batt::SmallVec page_ids; - /** \brief Returns a pseudo-random usize value r, where r >= min_value and r <= max_value. - */ - usize pick_usize(usize min_value, usize max_value) - { - std::uniform_int_distribution pick{min_value, max_value}; - return pick(this->rng); - } + llfs::slot_offset_type prepare_slot_offset; - /** \brief Returns a pseudo-random boolean value. - */ - bool pick_branch() - { - return this->pick_usize(0, 1) == 1; - } + usize prepare_slot_size = 0; - /** \brief Creates a new MemoryLogDevice and initializes it with minimal llfs::Volume metadata. - */ - void initialize_log() - { - this->mem_log_device.emplace(kLogSize); + batt::Optional commit_slot_offset; - llfs::TypedSlotWriter slot_writer{*this->mem_log_device}; + usize commit_slot_size = 0; + }; - batt::StatusOr grant = slot_writer.reserve( - llfs::packed_sizeof_slot(this->volume_ids), batt::WaitForResource::kFalse); + struct SimRun { + std::default_random_engine rng; - ASSERT_TRUE(grant.ok()); + u64 trim_delay_byte_count = 0; - batt::StatusOr ids_slot = slot_writer.append(*grant, this->volume_ids); + llfs::slot_offset_type highest_seen_drop_pages_client_slot = 0; - ASSERT_TRUE(ids_slot.ok()); - EXPECT_EQ(ids_slot->lower_bound, 0u); + std::map pending_jobs; - this->metadata_state.most_recent_ids_slot = ids_slot->lower_bound; - } + std::map committed_jobs; - /** \brief Creates a new FakeLogDevice that wraps this->mem_log_device, giving it a pseudo-random - * time-to-failure. - * - * Also creates a new SlotLockManager (this->trim_control), SlotReadLock (this->trim_lock), and - * slot writer to be used by the VolumeTrimmer (object-under-test). - */ - void open_fake_log() - { - llfs::testing::FakeLogDeviceFactory factory = - llfs::testing::make_fake_log_device_factory(*this->mem_log_device); + llfs::LogDeviceSnapshot log_snapshot; - this->fake_log_state = factory.state(); + //----- --- -- - - - - - const llfs::slot_offset_type initial_trim_pos = - this->mem_log_device->slot_range(llfs::LogReadMode::kDurable).lower_bound; + /** \brief Resets all test state variables to their initial values and sets the random number + * generator seed. + */ + void reset(unsigned random_seed, const Config& config); - this->recovery_visitor.emplace(initial_trim_pos); + /** \brief Returns a pseudo-random usize value r, where r >= min_value and r <= max_value. + */ + usize pick_usize(usize min_value, usize max_value); - { - batt::StatusOr> status_or_fake_log = factory.open_log_device( - [&](llfs::LogDevice::Reader& log_reader) -> batt::StatusOr { - llfs::TypedSlotReader slot_reader{log_reader}; + /** \brief Returns a pseudo-random boolean value. + */ + bool pick_branch(); + }; - batt::StatusOr slots_read = slot_reader.run( - batt::WaitForResource::kFalse, - [&](const llfs::SlotParse& slot, const auto& payload) -> batt::Status { - BATT_REQUIRE_OK((*this->recovery_visitor)(slot, payload)); - return batt::OkStatus(); - }); + struct LogSession { + SimRun* sim = nullptr; - BATT_REQUIRE_OK(slots_read); + batt::Optional mem_device; - return log_reader.slot_offset(); - }); + std::shared_ptr fake_device_state = nullptr; - ASSERT_TRUE(status_or_fake_log.ok()); - this->fake_log = std::move(*status_or_fake_log); - } + std::unique_ptr fake_device = nullptr; - EXPECT_EQ(this->metadata_state.most_recent_ids_slot != batt::None, - recovery_visitor->get_refresh_info().most_recent_ids_slot != batt::None) - << BATT_INSPECT(this->metadata_state) << BATT_INSPECT(recovery_visitor->get_refresh_info()); + std::map prepare_job_ptr; - // Reset the simulated device failure time. - // - const auto fake_log_ttf = this->pick_usize(kMinTTF, kMaxTTF); - LLFS_VLOG(1) << BATT_INSPECT(fake_log_ttf); - this->fake_log_state->failure_time = this->fake_log_state->device_time + fake_log_ttf; + batt::Optional trim_control; - // Initialize the slot lock manager. - // - this->trim_lock.clear(); - this->trim_control.emplace(); - this->trim_lock = BATT_OK_RESULT_OR_PANIC( - this->trim_control->lock_slots(this->fake_log->slot_range(llfs::LogReadMode::kSpeculative), - "VolumeTrimmerTest::open_fake_log")); + llfs::SlotReadLock trim_lock; - // Create a slot writer for the fake log. - // - if (this->job_grant) { - this->job_grant_size = this->job_grant->size() + this->leaked_job_grant_size; - this->leaked_job_grant_size = 0; - } - this->job_grant = batt::None; - this->fake_slot_writer.emplace(*this->fake_log); + batt::Optional> slot_writer; - // Set `last_recovered_trim_pos` so we don't think that trim delay is failing to arrest log - // trimming immediately after (re-)opening the log. - // - this->last_recovered_trim_pos = - this->fake_log->slot_range(llfs::LogReadMode::kDurable).lower_bound; - } + batt::Optional job_grant; - /** \brief Create a VolumeTrimmer for testing. - */ - void initialize_trimmer() - { - ASSERT_TRUE(this->trim_control); - ASSERT_TRUE(this->fake_log); - ASSERT_TRUE(this->fake_slot_writer); - ASSERT_TRUE(this->recovery_visitor); - - if (this->job_grant_size > 0) { - batt::StatusOr reserved = - this->fake_slot_writer->reserve(this->job_grant_size, batt::WaitForResource::kFalse); - if (!reserved.ok()) { - LLFS_LOG_ERROR() << "Failed to reserve grant;" << BATT_INSPECT(reserved.status()) - << BATT_INSPECT(this->job_grant_size) - << BATT_INSPECT(this->fake_slot_writer->pool_size()); - } - ASSERT_TRUE(reserved.ok()); - this->job_grant.emplace(std::move(*reserved)); - } + batt::Optional metadata_refresher; - LLFS_VLOG(1) << BATT_INSPECT(this->job_grant_size) - << BATT_INSPECT(this->recovery_visitor->get_refresh_info()) - << BATT_INSPECT(this->recovery_visitor->get_trim_event_info()) - << BATT_INSPECT_RANGE(this->recovery_visitor->get_pending_jobs()) - << BATT_INSPECT(this->recovery_visitor->get_trimmer_grant_size()) - << BATT_INSPECT(this->trim_delay_byte_count); - - this->trimmer = std::make_unique( - this->volume_ids.trimmer_uuid, "TestTrimmer", *this->trim_control, - llfs::TrimDelayByteCount{this->trim_delay_byte_count}, - this->fake_log->new_reader(/*slot_lower_bound=*/batt::None, - llfs::LogReadMode::kSpeculative), - *this->fake_slot_writer, - [this](auto&&... args) -> decltype(auto) { - return this->handle_drop_roots(BATT_FORWARD(args)...); - }, - *this->recovery_visitor); + std::map pending_job_slot_lock; - EXPECT_EQ(this->trimmer->uuid(), this->volume_ids.trimmer_uuid); + bool is_initialized = false; - LLFS_VLOG(1) << "Starting trimmer task"; + //----- --- -- - - - - - this->trimmer_task.emplace(this->task_context.get_executor(), [this] { - this->trimmer_status = this->trimmer->run(); - }); - } + void initialize(const Config& config, SimRun& sim); - /** \brief Stop the current VolumeTrimmer. - */ - void shutdown_trimmer() - { - // Tell everything to shut down. - // - this->shutting_down = true; - auto on_scope_exit = batt::finally([&] { - this->shutting_down = false; - }); + /** \brief Called inside this->initialize to recover session state (this->prepare_job_ptr) from + * the log. + */ + template + void handle_recovered_slot(const llfs::SlotParse& slot, const T& /*event*/); - this->fake_log->close().IgnoreError(); - this->trimmer->halt(); - this->trim_control->halt(); + void handle_recovered_slot(const llfs::SlotParse& slot, const llfs::VolumeTrimEvent& trim); - // Before we close the MemoryLogDevice (to unblock the VolumeTrimmer task), take a snapshot so - // it can be restored afterward. - // - auto snapshot = - llfs::LogDeviceSnapshot::from_device(*this->mem_log_device, llfs::LogReadMode::kDurable); + void handle_recovered_slot(const llfs::SlotParse& slot, + const llfs::Ref& prepare); - this->mem_log_device->close().IgnoreError(); + void handle_recovered_slot(const llfs::SlotParse& /*slot*/, + const llfs::Ref& commit); - // Drain the executor's queue. - // - ASSERT_NO_FATAL_FAILURE(this->task_context.poll()); + /** \brief Returns true iff a fake log has been created and the simulated device time is at + * least the planned failure time. + */ + bool fake_log_has_failed() const; - // The trimmer task should be finished at this point. - // - ASSERT_TRUE(this->trimmer_task->try_join()) - << "n_tasks=" << batt::Task::backtrace_all(/*force=*/true); + /** \brief Appends an application-level opaque data slot to the fake log and updates the + * SlotLockManager's upper bound. + */ + void append_opaque_data_slot(); - LLFS_VLOG(1) << "Trimmer task joined"; + /** \brief Selects a pseudo-random trim point and updates the trim lock to cause the + * VolumeTrimmer under test to initiate a trim operation. + * + * This function purposely does not select a trim point that aligns with a slot boundary; it is + * up to the VolumeTrimmer to make sure that the log is trimmed at slot boundaries. + */ + void trim_log(); - // Re-open the log and verify. - // - this->mem_log_device.emplace(kLogSize); - this->mem_log_device->restore_snapshot(snapshot, llfs::LogReadMode::kDurable); - this->open_fake_log(); - } + /** \brief Appends a psuedo-randomly generated PrepareJob slot containing one or more PageId + * roots. + * + * Does not append the corresponding CommitJob slot. + * + * If there isn't enough space in the log to reserve Grant for both prepare and commit slots, + * this function returns without changing anything. + */ + void prepare_one_job(); - /** \brief Called in response to the VolumeTrimmer requesting that a set of PageId's be dropped - * due to a log trim operation. - * - * Verifies that the correct set of page ids is specified, according to the trimmed region pointed - * to by the VolumeTrimEvent record at client_slot. - */ - batt::Status handle_drop_roots(llfs::slot_offset_type client_slot, - llfs::Slice page_ids) - { - if (!llfs::slot_less_than(this->verified_client_slot, client_slot)) { - return batt::OkStatus(); - } + /** \brief Appends a CommitJob slot corresponding to a pending PrepareJob selected + * pseudo-randomly. + * + * If there are no pending jobs, this function has no effect. + */ + void commit_one_job(); - if (this->shutting_down) { - return llfs::make_status(llfs::StatusCode::kFakeLogDeviceExpectedFailure); - } + /** \brief Sets the log and associated objects in a "closed"/"halted" state so that the + * trimmer task will be unblocked; doesn't tear down everything yet (that happens in + * this->terminate()). + */ + void halt(); - std::unique_ptr log_reader = this->mem_log_device->new_reader( - /*slot_lower_bound=*/client_slot, llfs::LogReadMode::kSpeculative); - BATT_CHECK_NOT_NULLPTR(log_reader); + void terminate(); + }; - std::vector actual_page_ids(page_ids.begin(), page_ids.end()); - std::sort(actual_page_ids.begin(), actual_page_ids.end()); + struct TrimmerSession { + const Config* config = nullptr; - std::vector expected_page_ids; + SimRun* sim = nullptr; - LLFS_VLOG(1) << "handle_drop_roots(" << BATT_INSPECT(client_slot) << "," - << BATT_INSPECT_RANGE(page_ids) << ")"; + LogSession* log = nullptr; - llfs::TypedSlotReader slot_reader{*log_reader}; - batt::StatusOr n_read = slot_reader.run( - batt::WaitForResource::kFalse, - batt::make_case_of_visitor( - [&](const llfs::SlotParse& /*slot*/, const llfs::VolumeTrimEvent& trim_event) { - // TODO [tastolfi 2023-05-22] use/verify `slot` arg - - for (auto iter = this->committed_jobs.begin(); iter != this->committed_jobs.end(); - iter = this->committed_jobs.erase(iter)) { - const auto& [prepare_slot, page_ids] = *iter; - if (!llfs::slot_less_than(prepare_slot, trim_event.new_trim_pos)) { - break; - } - LLFS_VLOG(1) << " -- expecting: " << BATT_INSPECT(prepare_slot) - << BATT_INSPECT_RANGE(page_ids); - expected_page_ids.insert(expected_page_ids.end(), page_ids.begin(), page_ids.end()); - } + bool shutting_down = false; - // Just read a single slot! - // - return llfs::make_status(llfs::StatusCode::kBreakSlotReaderLoop); - }, - [&](const llfs::SlotParse& slot, const auto& payload) { - BATT_PANIC() << "There should always be a trim event slot at `client_slot`!" - << BATT_INSPECT(slot.offset) << ", payload type == " - << batt::name_of>(); + std::unique_ptr trimmer; - // Just read a single slot! - // - return llfs::make_status(llfs::StatusCode::kBreakSlotReaderLoop); - })); + llfs::slot_offset_type last_recovered_trim_pos = 0; - // Sort the expected page ids so we can compare it to the actual page ids. - // - std::sort(expected_page_ids.begin(), expected_page_ids.end()); + batt::FakeExecutionContext task_context; - this->verified_client_slot = client_slot; + batt::Optional trimmer_task; - [&] { - ASSERT_EQ(actual_page_ids, expected_page_ids); - }(); + batt::Status trimmer_task_exit_status; - return batt::OkStatus(); + //----- --- -- - - - - + + /** \brief Create a VolumeTrimmer for testing. + */ + void initialize(const Config& config, SimRun& sim, LogSession& log); + + /** \brief Checks all invariants. Uses GTEST ASSERT* statements to verify. + */ + void check_invariants(); + + /** \brief Called in response to the VolumeTrimmer requesting that a set of PageId's be dropped + * due to a log trim operation. + * + * Verifies that the correct set of page ids is specified, according to the trimmed region + * pointed to by the VolumeTrimEvent record at client_slot. + */ + batt::Status handle_drop_roots(boost::uuids::uuid const& trimmer_uuid, + llfs::slot_offset_type client_slot, + llfs::Slice page_ids); + + /** \brief Runs a random unit of work from the task_context. + */ + void run_one(); + + /** \brief Stop the current VolumeTrimmer. + */ + void terminate(); + }; +}; + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::SimRun::reset(unsigned random_seed, const Config& config) +{ + this->rng.seed(random_seed); + + // Prime the rng. + // + for (usize i = 0; i < 11; ++i) { + (void)this->rng(); } - /** \brief Returns true iff a fake log has been created and the simulated device time is at least - * the planned failure time. - */ - bool fake_log_has_failed() const + this->trim_delay_byte_count = this->pick_usize(0, 16) * 64; + this->highest_seen_drop_pages_client_slot = 0; + this->pending_jobs.clear(); + this->committed_jobs.clear(); + + // Create the initial log snapshot by creating a new MemoryLogDevice and initializing it with + // minimal llfs::Volume metadata. + // + llfs::MemoryLogDevice mem_log_device{kLogSize}; + + llfs::TypedSlotWriter slot_writer{mem_log_device}; + + batt::StatusOr grant = slot_writer.reserve( + llfs::packed_sizeof_slot(config.volume_ids), batt::WaitForResource::kFalse); + + ASSERT_TRUE(grant.ok()); + + batt::StatusOr ids_slot = slot_writer.append(*grant, config.volume_ids); + + ASSERT_TRUE(ids_slot.ok()); + EXPECT_EQ(ids_slot->lower_bound, 0u); + + this->log_snapshot = + llfs::LogDeviceSnapshot::from_device(mem_log_device, llfs::LogReadMode::kDurable); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize VolumeTrimmerTest::SimRun::pick_usize(usize min_value, usize max_value) +{ + std::uniform_int_distribution pick{min_value, max_value}; + return pick(this->rng); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +bool VolumeTrimmerTest::SimRun::pick_branch() +{ + return this->pick_usize(0, 1) == 1; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::initialize(const Config& config, SimRun& sim) +{ + BATT_CHECK_EQ(this->sim, nullptr); + + this->sim = std::addressof(sim); + + //+++++++++++-+-+--+----- --- -- - - - - + // Initialize the MemoryLogDevice from the current snapshot. + // { - return this->fake_log_state && - this->fake_log_state->device_time >= this->fake_log_state->failure_time; + BATT_CHECK_EQ(this->mem_device, batt::None); + BATT_CHECK(sim.log_snapshot); + + this->mem_device.emplace(kLogSize, sim.log_snapshot, llfs::LogReadMode::kDurable); } - /** \brief Appends an application-level opaque data slot to the fake log and updates the - * SlotLockManager's upper bound. - */ - void append_opaque_data_slot() + //+++++++++++-+-+--+----- --- -- - - - - + // Initialize the FakeLogDevice. + // + llfs::VolumeMetadata metadata; { - ASSERT_TRUE(this->fake_log); - ASSERT_TRUE(this->fake_slot_writer); - ASSERT_TRUE(this->trim_control); + llfs::VolumeMetadataRecoveryVisitor metadata_visitor{metadata}; - const usize data_size = this->pick_usize(kMinOpaqueDataSize, kMaxOpaqueDataSize); - std::vector buffer(data_size, 'a'); - std::string_view data_str{buffer.data(), buffer.size()}; + llfs::testing::FakeLogDeviceFactory factory = + llfs::testing::make_fake_log_device_factory(*this->mem_device); - llfs::PackAsRawData to_pack_as_raw{data_str}; - auto&& payload = llfs::PackableRef{to_pack_as_raw}; + this->fake_device_state = factory.state(); - const usize slot_size = llfs::packed_sizeof_slot(payload); - batt::StatusOr slot_grant = - this->fake_slot_writer->reserve(slot_size, batt::WaitForResource::kFalse); + BATT_CHECK(this->prepare_job_ptr.empty()); - ASSERT_TRUE(slot_grant.ok() || this->fake_log_has_failed()) - << BATT_INSPECT(slot_grant.status()) << BATT_INSPECT(slot_size) - << BATT_INSPECT(this->fake_slot_writer->pool_size()); + batt::StatusOr> status_or_fake_log = factory.open_log_device( + [&](llfs::LogDevice::Reader& log_reader) -> batt::StatusOr { + llfs::TypedSlotReader slot_reader{log_reader}; - batt::StatusOr slot_range = - this->fake_slot_writer->append(*slot_grant, payload); + batt::StatusOr slots_read = slot_reader.run( + batt::WaitForResource::kFalse, + [&](const llfs::SlotParse& slot, const auto& payload) -> batt::Status { + BATT_REQUIRE_OK(metadata_visitor(slot, payload)); + this->handle_recovered_slot(slot, payload); + return batt::OkStatus(); + }); - ASSERT_TRUE(slot_range.ok() || this->fake_log_has_failed()) - << BATT_INSPECT(slot_range) << BATT_INSPECT(this->fake_log_has_failed()); + BATT_REQUIRE_OK(slots_read); - if (slot_range.ok()) { - this->trim_control->update_upper_bound(slot_range->upper_bound); - } + return log_reader.slot_offset(); + }); + + ASSERT_TRUE(status_or_fake_log.ok()) << BATT_INSPECT(status_or_fake_log.status()) + << BATT_INSPECT(this->fake_device_state->device_time) + << BATT_INSPECT(this->fake_device_state->failure_time); + + this->fake_device = std::move(*status_or_fake_log); - LLFS_VLOG(1) << "Appended opaque data: " << batt::c_str_literal(data_str); + ASSERT_TRUE(metadata.ids) << BATT_INSPECT( + fake_device->slot_range(llfs::LogReadMode::kSpeculative)); + + EXPECT_EQ(metadata.ids->main_uuid, config.volume_ids.main_uuid); + EXPECT_EQ(metadata.ids->recycler_uuid, config.volume_ids.recycler_uuid); + EXPECT_EQ(metadata.ids->trimmer_uuid, config.volume_ids.trimmer_uuid); } - /** \brief Selects a pseudo-random trim point and updates the trim lock to cause the VolumeTrimmer - * under test to initiate a trim operation. - * - * This function purposely does not select a trim point that aligns with a slot boundary; it is up - * to the VolumeTrimmer to make sure that the log is trimmed at slot boundaries. - */ - void trim_log() + //+++++++++++-+-+--+----- --- -- - - - - + // Reset the simulated device failure time. + // { - ASSERT_TRUE(this->fake_log); - ASSERT_TRUE(this->trim_control); - - const llfs::SlotRange log_range = this->fake_log->slot_range(llfs::LogReadMode::kSpeculative); - const llfs::SlotRange lock_range = this->trim_lock.slot_range(); - const llfs::SlotRange new_range{this->pick_usize(lock_range.lower_bound, log_range.upper_bound), - log_range.upper_bound}; + const auto fake_log_ttf = this->sim->pick_usize(kMinTTF, kMaxTTF); + LLFS_VLOG(1) << BATT_INSPECT(fake_log_ttf); + this->fake_device_state->failure_time = this->fake_device_state->device_time + fake_log_ttf; + } - LLFS_VLOG(1) << "Trimming log;" << BATT_INSPECT(log_range) << BATT_INSPECT(lock_range) - << BATT_INSPECT(new_range); + //+++++++++++-+-+--+----- --- -- - - - - + // Create a new SlotLockManager. + // + { + BATT_CHECK_EQ(this->trim_control, batt::None); + this->trim_control.emplace(); - this->trim_lock = BATT_OK_RESULT_OR_PANIC(this->trim_control->update_lock( - std::move(this->trim_lock), new_range, "VolumeTrimmerTest::trim_log")); + BATT_CHECK(!this->trim_lock); + this->trim_lock = BATT_OK_RESULT_OR_PANIC(this->trim_control->lock_slots( + this->fake_device->slot_range(llfs::LogReadMode::kSpeculative), + "VolumeTrimmerTest::LogSession::initialize")); } - /** \brief Appends a psuedo-randomly generated PrepareJob slot containing one or more PageId - * roots. - * - * Does not append the corresponding CommitJob slot. - */ - void prepare_one_job() + //+++++++++++-+-+--+----- --- -- - - - - + // Create a SlotWriter and initialize the job_grant. + // { - ASSERT_TRUE(this->fake_slot_writer); + BATT_CHECK_EQ(this->slot_writer, batt::None); + BATT_CHECK_NOT_NULLPTR(this->fake_device); - LLFS_VLOG(1) << "Generating prepare job"; + this->slot_writer.emplace(*this->fake_device); - std::vector page_ids; + BATT_CHECK_EQ(this->job_grant, batt::None); - const usize n_pages = this->pick_usize(1, kNumPages * 2); - for (usize i = 0; i < n_pages; ++i) { - page_ids.emplace_back(llfs::PageId{this->pick_usize(0, kNumPages - 1)}); - LLFS_VLOG(1) << " -- " << page_ids.back(); - } + this->job_grant.emplace( + BATT_OK_RESULT_OR_PANIC(this->slot_writer->reserve(0, batt::WaitForResource::kFalse))); + } - batt::BoxedSeq page_ids_seq = - batt::as_seq(page_ids) | batt::seq::decayed() | batt::seq::boxed(); + //+++++++++++-+-+--+----- --- -- - - - - + // Reacquire slot locks and grant for all pending jobs. + // + { + BATT_CHECK(this->pending_job_slot_lock.empty()); - llfs::PrepareJob prepare{ - .new_page_ids = batt::seq::Empty{} | batt::seq::boxed(), - .deleted_page_ids = batt::seq::Empty{} | batt::seq::boxed(), - .page_device_ids = batt::seq::Empty{} | batt::seq::boxed(), - .user_data = llfs::PackableRef{page_ids_seq}, - }; + for (const auto& [slot_offset, job_info] : this->sim->pending_jobs) { + ASSERT_NE(this->prepare_job_ptr.count(slot_offset), 0) + << "PackedPrepareJob slot not found for pending job at slot: " << slot_offset; - const usize n_to_reserve = (packed_sizeof_slot(prepare) + - packed_sizeof_slot(batt::StaticType{})) * - 2; + llfs::SlotReadLock slot_lock = BATT_OK_RESULT_OR_PANIC(this->trim_control->lock_slots( + llfs::SlotRange{slot_offset, slot_offset + 1}, "open_fake_log()")); - batt::StatusOr slot_grant = - this->fake_slot_writer->reserve(n_to_reserve, batt::WaitForResource::kFalse); + this->pending_job_slot_lock[slot_offset] = std::move(slot_lock); - if (!slot_grant.ok()) { - LLFS_VLOG(1) << "Not enough space in the log; cancelling job"; - return; - } + batt::Grant single_job_grant = BATT_OK_RESULT_OR_PANIC( + this->slot_writer->reserve(job_info.commit_slot_size, batt::WaitForResource::kFalse)); - { - batt::Grant trim_grant = - BATT_OK_RESULT_OR_PANIC(slot_grant->spend(packed_sizeof_slot(prepare))); - this->trimmer->push_grant(std::move(trim_grant)); + this->job_grant->subsume(std::move(single_job_grant)); } + } - batt::StatusOr prepare_slot = - this->fake_slot_writer->append(*slot_grant, std::move(prepare)); + //+++++++++++-+-+--+----- --- -- - - - - + // Initialize the VolumeMetadataRefresher. + // + { + BATT_CHECK_EQ(this->metadata_refresher, batt::None); + BATT_CHECK_NE(this->slot_writer, batt::None); - ASSERT_TRUE(prepare_slot.ok() || this->fake_log_has_failed()); + this->metadata_refresher.emplace(*this->slot_writer, batt::make_copy(metadata)); - if (!this->job_grant) { - this->job_grant.emplace(std::move(*slot_grant)); - } else { - this->job_grant->subsume(std::move(*slot_grant)); - } + // Reserve some grant for refreshing metadata. + // + if (this->metadata_refresher->grant_required() > 0) { + batt::StatusOr metadata_grant = + this->slot_writer->reserve(std::min(this->slot_writer->pool_size(), + this->metadata_refresher->grant_required()), + batt::WaitForResource::kFalse); - if (prepare_slot.ok()) { - LLFS_VLOG(1) << "Wrote prepare slot at " << *prepare_slot - << BATT_INSPECT(this->job_grant->size()); + ASSERT_TRUE(metadata_grant.ok()) << BATT_INSPECT(metadata_grant.status()); - this->pending_jobs.emplace_back(prepare_slot->lower_bound, std::move(page_ids)); + batt::Status update_status = this->metadata_refresher->update_grant_partial(*metadata_grant); + + ASSERT_TRUE(update_status.ok()) << BATT_INSPECT(update_status); } } - /** \brief Appends a CommitJob slot corresponding to a pending PrepareJob selected - * pseudo-randomly. - */ - void commit_one_job() - { - ASSERT_FALSE(this->pending_jobs.empty()); - ASSERT_TRUE(this->job_grant); - ASSERT_TRUE(this->trimmer); + // Done! + // + this->is_initialized = true; +} - LLFS_VLOG(1) << "Committing job"; +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void VolumeTrimmerTest::LogSession::handle_recovered_slot(const llfs::SlotParse& slot, + const T& /*event*/) +{ + LLFS_VLOG(1) << BATT_INSPECT(slot.offset) << BATT_INSPECT(batt::name_of()); +} - const usize grant_size_before = this->job_grant->size(); +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::handle_recovered_slot(const llfs::SlotParse& slot, + const llfs::VolumeTrimEvent& trim) +{ + using T = std::decay_t; - const usize job_i = this->pick_usize(0, this->pending_jobs.size() - 1); + LLFS_VLOG(1) << BATT_INSPECT(slot.offset) << BATT_INSPECT(batt::name_of()) << "; " << trim; +} - const llfs::PackedCommitJob commit{ - .reserved_ = {}, - .prepare_slot = this->pending_jobs[job_i].first, - }; +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::handle_recovered_slot( + const llfs::SlotParse& slot, const llfs::Ref& prepare) +{ + using T = std::decay_t; - const llfs::slot_offset_type log_upper_bound = - this->mem_log_device->slot_range(llfs::LogReadMode::kSpeculative).upper_bound; + LLFS_VLOG(1) << BATT_INSPECT(slot.offset) << BATT_INSPECT(batt::name_of()); - ASSERT_LT(commit.prepare_slot, log_upper_bound) - << BATT_INSPECT(job_i) << BATT_INSPECT(this->pending_jobs.size()) - << BATT_INSPECT_RANGE(this->pending_jobs); - ASSERT_GE(this->job_grant->size(), llfs::packed_sizeof_slot(commit) * 2); + this->prepare_job_ptr[slot.offset.lower_bound] = prepare.pointer(); +} - { - batt::Grant trim_grant = BATT_OK_RESULT_OR_PANIC( - this->job_grant->spend(llfs::packed_sizeof_slot(commit), batt::WaitForResource::kFalse)); - this->trimmer->push_grant(std::move(trim_grant)); - } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::handle_recovered_slot( + const llfs::SlotParse& slot, const llfs::Ref& commit) +{ + using T = std::decay_t; - batt::StatusOr commit_slot = - this->fake_slot_writer->append(*this->job_grant, std::move(commit)); + LLFS_VLOG(1) << BATT_INSPECT(slot.offset) << BATT_INSPECT(batt::name_of()); - const usize grant_size_after = this->job_grant->size(); - const usize grant_size_spent = grant_size_before - grant_size_after; + this->prepare_job_ptr.erase(commit.get().prepare_slot_offset); +} - ASSERT_GE(grant_size_before, grant_size_after); +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +bool VolumeTrimmerTest::LogSession::fake_log_has_failed() const +{ + return this->fake_device_state && + this->fake_device_state->device_time >= this->fake_device_state->failure_time; +} - if (!commit_slot.ok() && grant_size_spent > 0) { - this->leaked_job_grant_size += grant_size_spent; - } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::append_opaque_data_slot() +{ + ASSERT_TRUE(this->sim); + ASSERT_TRUE(this->fake_device); + ASSERT_TRUE(this->slot_writer); + ASSERT_TRUE(this->trim_control); - ASSERT_TRUE(commit_slot.ok() || this->fake_log_has_failed()); + const usize data_size = this->sim->pick_usize(kMinOpaqueDataSize, kMaxOpaqueDataSize); + batt::SmallVec buffer(data_size, 'a'); + std::string_view data_str{buffer.data(), buffer.size()}; - if (commit_slot.ok()) { - LLFS_VLOG(1) << "Wrote commit slot at " << *commit_slot - << " (prepare_slot=" << commit.prepare_slot << ")"; + llfs::PackAsRawData to_pack_as_raw{data_str}; + auto&& payload = llfs::PackableRef{to_pack_as_raw}; - std::swap(this->pending_jobs[job_i], this->pending_jobs.back()); + const usize slot_size = llfs::packed_sizeof_slot(payload); + batt::StatusOr slot_grant = + this->slot_writer->reserve(slot_size, batt::WaitForResource::kFalse); - LLFS_VLOG(1) << "adding {slot=" << commit_slot->lower_bound - << ", page_ids=" << batt::dump_range(this->pending_jobs.back().second) - << "} to committed_jobs"; + ASSERT_TRUE(slot_grant.ok() || this->fake_log_has_failed()) + << BATT_INSPECT(slot_grant.status()) << BATT_INSPECT(slot_size) + << BATT_INSPECT(this->slot_writer->pool_size()); - this->committed_jobs.emplace(commit_slot->lower_bound, - std::move(this->pending_jobs.back().second)); + batt::StatusOr slot_range = this->slot_writer->append(*slot_grant, payload); - this->pending_jobs.pop_back(); - } + ASSERT_TRUE(slot_range.ok() || this->fake_log_has_failed()) + << BATT_INSPECT(slot_range) << BATT_INSPECT(this->fake_log_has_failed()); + + if (slot_range.ok()) { + this->trim_control->update_upper_bound(slot_range->upper_bound); } - /** \brief Checks all invariants. - */ - void check_invariants() - { - // Check to make sure that not too much of the log was trimmed. - // - { - BATT_CHECK(this->trim_control); - const llfs::slot_offset_type least_locked_slot = this->trim_control->get_lower_bound(); + LLFS_VLOG(1) << "Appended opaque data: " << batt::c_str_literal(data_str); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::trim_log() +{ + BATT_CHECK_NOT_NULLPTR(this->sim); - BATT_CHECK_NOT_NULLPTR(this->fake_log); - const llfs::slot_offset_type trim_pos = - this->fake_log->slot_range(llfs::LogReadMode::kDurable).lower_bound; + ASSERT_TRUE(this->fake_device); + ASSERT_TRUE(this->trim_control); - if (least_locked_slot >= this->last_recovered_trim_pos + this->trim_delay_byte_count) { - ASSERT_GE((i64)least_locked_slot - (i64)trim_pos, (i64)this->trim_delay_byte_count) - << BATT_INSPECT(this->trim_delay_byte_count) << BATT_INSPECT(least_locked_slot) - << BATT_INSPECT(trim_pos) << BATT_INSPECT(this->last_recovered_trim_pos); - } else { - ASSERT_EQ(trim_pos, this->last_recovered_trim_pos) - << BATT_INSPECT(least_locked_slot) << BATT_INSPECT(this->trim_delay_byte_count); - } - } + const llfs::SlotRange log_range = this->fake_device->slot_range(llfs::LogReadMode::kSpeculative); + const llfs::SlotRange lock_range = this->trim_lock.slot_range(); + const llfs::SlotRange new_range{ + this->sim->pick_usize(lock_range.lower_bound, log_range.upper_bound), log_range.upper_bound}; + + LLFS_VLOG(1) << "Trimming log;" << BATT_INSPECT(log_range) << BATT_INSPECT(lock_range) + << BATT_INSPECT(new_range); + + this->trim_lock = BATT_OK_RESULT_OR_PANIC(this->trim_control->update_lock( + std::move(this->trim_lock), new_range, "VolumeTrimmerTest::trim_log")); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::prepare_one_job() +{ + ASSERT_TRUE(this->slot_writer); + + LLFS_VLOG(1) << "Generating prepare job"; + + batt::SmallVec page_ids; + + const usize n_pages = this->sim->pick_usize(1, kNumPages * 2); + for (usize i = 0; i < n_pages; ++i) { + page_ids.emplace_back(llfs::PageId{this->sim->pick_usize(0, kNumPages - 1)}); + LLFS_VLOG(1) << " -- " << page_ids.back(); } - /** \brief Forces the trimmer task to shut down; usually because we have failed an ASSERT. - */ - void force_shut_down() - { - // Only need to do something if the trimmer_task exists. - // - if (!this->trimmer_task) { - return; - } + batt::BoxedSeq page_ids_seq = + batt::as_seq(page_ids) | batt::seq::decayed() | batt::seq::boxed(); - // Halt everything that might be blocking the trimmer_task. - // - if (this->trimmer) { - this->trimmer->halt(); - } - if (this->trim_control) { - this->trim_control->halt(); - } - if (this->fake_log) { - this->fake_log->close().IgnoreError(); - } + llfs::PrepareJob prepare{ + .new_page_ids = batt::seq::Empty{} | batt::seq::boxed(), + .deleted_page_ids = batt::seq::Empty{} | batt::seq::boxed(), + .page_device_ids = batt::seq::Empty{} | batt::seq::boxed(), + .user_data = llfs::PackableRef{page_ids_seq}, + }; - // Run tasks until there are no more. - // - for (;;) { - batt::UniqueHandler<> action = this->task_context.pop_ready_handler([this](usize /*n*/) { - return 0; - }); - if (!action) { - break; - } - action(); - } + const usize prepare_slot_size = packed_sizeof_slot(prepare); + const usize commit_slot_size = packed_sizeof_commit_slot(prepare); + const usize n_to_reserve = (prepare_slot_size + commit_slot_size); - // The trimmer_task should now be joined. - // - BATT_CHECK_EQ(this->trimmer_task->try_join(), batt::Task::IsDone{true}); + batt::StatusOr slot_grant = + this->slot_writer->reserve(n_to_reserve, batt::WaitForResource::kFalse); + + if (!slot_grant.ok()) { + LLFS_VLOG(1) << "Not enough space in the log; cancelling job"; + return; } - //+++++++++++-+-+--+----- --- -- - - - - + batt::StatusOr> prepare_slot = + this->slot_writer->typed_append(*slot_grant, std::move(prepare)); + + ASSERT_TRUE(prepare_slot.ok() || this->fake_log_has_failed()); + + if (prepare_slot.ok()) { + LLFS_VLOG(1) << "Wrote prepare slot at " << *prepare_slot + << BATT_INSPECT(this->job_grant->size()) + << BATT_INSPECT(this->slot_writer->pool_size()); + + BATT_CHECK(this->job_grant); + this->job_grant->subsume(std::move(*slot_grant)); + + this->prepare_job_ptr[prepare_slot->slot.offset.lower_bound] = prepare_slot->payload; + + this->sim->pending_jobs.emplace( + prepare_slot->slot.offset.lower_bound, + JobInfo{ + .page_ids = std::move(page_ids), + .prepare_slot_offset = prepare_slot->slot.offset.lower_bound, + .prepare_slot_size = prepare_slot_size, + .commit_slot_offset = batt::None, + .commit_slot_size = commit_slot_size, + }); + + this->pending_job_slot_lock.emplace(prepare_slot->slot.offset.lower_bound, + BATT_OK_RESULT_OR_PANIC(this->trim_control->lock_slots( + prepare_slot->slot.offset, "prepare_one_job"))); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::commit_one_job() +{ + BATT_CHECK_NOT_NULLPTR(this->sim); + + if (this->sim->pending_jobs.empty()) { + return; + } - std::default_random_engine rng{1234567}; + ASSERT_TRUE(this->job_grant); - u64 trim_delay_byte_count = 0; + LLFS_VLOG(1) << "Committing job"; - llfs::slot_offset_type last_recovered_trim_pos = 0; + const usize grant_size_before = this->job_grant->size(); - std::array page_ref_count; + const usize job_i = this->sim->pick_usize(0, this->sim->pending_jobs.size() - 1); - const llfs::PackedVolumeIds volume_ids{ - .main_uuid = llfs::random_uuid(), - .recycler_uuid = llfs::random_uuid(), - .trimmer_uuid = llfs::random_uuid(), + const auto iter = std::next(this->sim->pending_jobs.begin(), job_i); + + llfs::slot_offset_type prepare_slot_offset = iter->first; + JobInfo& job_info = iter->second; + + const llfs::CommitJob commit{ + .prepare_slot_offset = prepare_slot_offset, + .packed_prepare = this->prepare_job_ptr[prepare_slot_offset], }; - llfs::VolumeMetadataRefreshInfo metadata_state; + BATT_CHECK_NOT_NULLPTR(commit.packed_prepare); + + const llfs::slot_offset_type log_upper_bound = + this->mem_device->slot_range(llfs::LogReadMode::kSpeculative).upper_bound; - batt::Optional mem_log_device; + ASSERT_LT(commit.prepare_slot_offset, log_upper_bound) + << BATT_INSPECT(job_i) << BATT_INSPECT(this->sim->pending_jobs.size()); - std::shared_ptr fake_log_state = nullptr; + const usize commit_slot_size = llfs::packed_sizeof_slot(commit); - std::unique_ptr fake_log = nullptr; + ASSERT_GE(this->job_grant->size(), commit_slot_size); + EXPECT_EQ(commit_slot_size, job_info.commit_slot_size); - batt::Optional> fake_slot_writer; + batt::StatusOr commit_slot = + this->slot_writer->append(*this->job_grant, std::move(commit)); - batt::FakeExecutionContext task_context; + const usize grant_size_after = this->job_grant->size(); - batt::Optional trim_control; + ASSERT_GE(grant_size_before, grant_size_after); + ASSERT_TRUE(commit_slot.ok() || this->fake_log_has_failed()); - llfs::SlotReadLock trim_lock; + if (commit_slot.ok()) { + LLFS_VLOG(1) << "Wrote commit slot at " << *commit_slot + << " (prepare_slot=" << commit.prepare_slot_offset << ")"; - batt::Optional recovery_visitor; + job_info.commit_slot_offset = commit_slot->lower_bound; - std::unique_ptr trimmer; + this->sim->committed_jobs.emplace(commit_slot->lower_bound, std::move(job_info)); + + this->pending_job_slot_lock.erase(prepare_slot_offset); + this->prepare_job_ptr.erase(prepare_slot_offset); + this->sim->pending_jobs.erase(iter); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::halt() +{ + this->fake_device->close().IgnoreError(); + this->trim_control->halt(); - batt::Status trimmer_status; + // Before we close the MemoryLogDevice (to unblock the VolumeTrimmer task), take a snapshot so + // it can be restored afterward. + // + this->sim->log_snapshot = + llfs::LogDeviceSnapshot::from_device(*this->mem_device, llfs::LogReadMode::kDurable); - std::vector>> pending_jobs; + this->mem_device->close().IgnoreError(); +} - batt::Optional job_grant; +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::LogSession::terminate() +{ + this->is_initialized = false; - usize job_grant_size = 0; - usize leaked_job_grant_size = 0; + // Tear down the slot lock state. + // + this->pending_job_slot_lock.clear(); + this->trim_lock.clear(); + this->trim_control = batt::None; - std::map> committed_jobs; + // Tear down the slot writer. + // + this->metadata_refresher = batt::None; + this->job_grant = batt::None; + this->slot_writer = batt::None; - batt::Optional trimmer_task; + // Release all pointers into the log buffer. + // + this->prepare_job_ptr.clear(); - llfs::slot_offset_type verified_client_slot = 0; + // Tear down the log device. + // + this->fake_device = nullptr; + this->fake_device_state = nullptr; + this->mem_device = batt::None; - bool shutting_down = false; -}; + // Reset pointers. + // + this->sim = nullptr; +} -//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -TEST_F(VolumeTrimmerTest, RandomizedTest) +void VolumeTrimmerTest::TrimmerSession::initialize(const Config& config, SimRun& sim, + LogSession& log) { - const bool kExtraTesting = batt::getenv_as("LLFS_EXTRA_TESTING").value_or(0); - const usize kNumSeeds = kExtraTesting ? 10 * 1000 * 1000 : 10 * 1000; - const int kLogEveryN = kExtraTesting ? 10 * 1000 : 1000; - const usize kInitialSeed = 0; - usize crash_count = 0; + this->shutting_down = false; - // Make sure there are no running tasks when we return. + //+++++++++++-+-+--+----- --- -- - - - - + // Set config, sim run, log session pointers. // - auto on_scope_exit = batt::finally([&] { - this->force_shut_down(); + { + BATT_CHECK_EQ(this->config, nullptr); + + this->config = std::addressof(config); + + BATT_CHECK_EQ(this->sim, nullptr); + + this->sim = std::addressof(sim); + + BATT_CHECK(log.is_initialized); + BATT_CHECK_EQ(this->log, nullptr); + + this->log = std::addressof(log); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Create a new VolumeTrimmer. + // + { + BATT_CHECK_NOT_NULLPTR(log.fake_device); + BATT_CHECK_NE(log.slot_writer, batt::None); + BATT_CHECK_EQ(this->trimmer, nullptr); + + batt::StatusOr> new_trimmer = llfs::VolumeTrimmer::recover( + this->config->volume_ids.trimmer_uuid, // + /*name=*/"TestTrimmer", // + llfs::TrimDelayByteCount{this->sim->trim_delay_byte_count}, // + *log.fake_device, // + *log.slot_writer, // + [this](auto&&... args) -> decltype(auto) { + return this->handle_drop_roots(BATT_FORWARD(args)...); + }, + *this->log->trim_control, // + *this->log->metadata_refresher // + ); + + ASSERT_TRUE(new_trimmer.ok()) << BATT_INSPECT(new_trimmer.status()); + + LLFS_VLOG(1) << "initialize_trimmer(): " << (void*)std::addressof(**new_trimmer); + this->trimmer = std::move(*new_trimmer); + + EXPECT_EQ(this->trimmer->uuid(), this->config->volume_ids.trimmer_uuid); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Set the recovered trim pos now; IMPORTANT: this must be done _after_ calling + // VolumeTrimmer::recover, as this function may finish a partially completed trim, causing the + // trim pos to move forward. + // + { + this->last_recovered_trim_pos = + log.fake_device->slot_range(llfs::LogReadMode::kDurable).lower_bound; + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Start the trimmer task. + // + LLFS_VLOG(1) << "Starting trimmer task"; + + this->trimmer_task.emplace(this->task_context.get_executor(), [this] { + this->trimmer_task_exit_status = this->trimmer->run(); }); +} - for (usize seed_i = kInitialSeed; seed_i < kInitialSeed + kNumSeeds; seed_i += 1) { - LLFS_LOG_INFO_EVERY_N(kLogEveryN) << "Starting new test run with empty log;" - << BATT_INSPECT(seed_i) << BATT_INSPECT(crash_count); +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::TrimmerSession::check_invariants() +{ + // Check to make sure that not too much of the log was trimmed. + // + if (this->sim && this->log && this->log->fake_device) { + const llfs::slot_offset_type trim_pos = + this->log->fake_device->slot_range(llfs::LogReadMode::kDurable).lower_bound; - this->rng.seed(seed_i * 741461423ull); + if (this->log->trim_control) { + const llfs::slot_offset_type least_locked_slot = this->log->trim_control->get_lower_bound(); - this->reset_state(); + if (least_locked_slot >= this->last_recovered_trim_pos + this->sim->trim_delay_byte_count) { + ASSERT_GE((i64)least_locked_slot - (i64)trim_pos, (i64)this->sim->trim_delay_byte_count) + << BATT_INSPECT(this->sim->trim_delay_byte_count) << BATT_INSPECT(least_locked_slot) + << BATT_INSPECT(trim_pos) << BATT_INSPECT(this->last_recovered_trim_pos); + } else { + ASSERT_EQ(trim_pos, this->last_recovered_trim_pos) + << BATT_INSPECT(least_locked_slot) << BATT_INSPECT(this->sim->trim_delay_byte_count); + } + } - // Pick a trim delay setting for this test run. + // No pending jobs should be trimmed! // - this->trim_delay_byte_count = this->pick_usize(0, 16) * 64; + for (const auto& [prepare_slot_offset, job_info] : this->sim->pending_jobs) { + ASSERT_TRUE(!llfs::slot_less_than(prepare_slot_offset, trim_pos)) + << BATT_INSPECT(prepare_slot_offset) << BATT_INSPECT(trim_pos); + } + } +} - this->open_fake_log(); - this->initialize_trimmer(); +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +batt::Status VolumeTrimmerTest::TrimmerSession::handle_drop_roots( + boost::uuids::uuid const& trimmer_uuid, llfs::slot_offset_type client_slot, + llfs::Slice page_ids) +{ + BATT_CHECK_NOT_NULLPTR(this->config); + BATT_CHECK_NOT_NULLPTR(this->log); - // Main test loop: for each step, randomly take some action that advances the state of the - // log/trimmer. - // - LLFS_VLOG(1) << "Entering test loop"; + EXPECT_EQ(trimmer_uuid, this->config->volume_ids.trimmer_uuid); + + // Ignore calls to drop roots for client_slot values that have already been seen, just like + // PageAllocator does. + // + if (!llfs::slot_less_than(this->sim->highest_seen_drop_pages_client_slot, client_slot)) { + return batt::OkStatus(); + } - for (int step_i = 0; !this->fake_log_has_failed(); ++step_i) { - LLFS_VLOG(1) << "Step " << step_i; + // If we have already encountered a simulated fatal error, then return error status immediately. + // + if (this->shutting_down) { + return llfs::make_status(llfs::StatusCode::kFakeLogDeviceExpectedFailure); + } - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); + batt::SmallVec actual_page_ids(page_ids.begin(), + page_ids.end()); + std::sort(actual_page_ids.begin(), actual_page_ids.end()); - // Let the VolumeTrimmer task run. - // - if (this->pick_branch()) { - LLFS_VLOG(1) << "Checking for tasks ready to run"; - batt::UniqueHandler<> action = this->task_context.pop_ready_handler([this](usize n) { - return this->pick_usize(0, n - 1); - }); - if (action) { - ASSERT_NO_FATAL_FAILURE(action()); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); - } - } + //+++++++++++-+-+--+----- --- -- - - - - + // Read the log, starting at `client_slot` in order to: + // 1. Verify there is a VolumeTrimEvent at the specified slot + // 2. Get the trim position so we can select the "in-scope" jobs to build the expected page_ids + // list. + // + batt::Optional> next_trim_event; + { + BATT_CHECK_NE(this->log->mem_device, batt::None); - // Trim the log. - // - if (this->pick_branch()) { - ASSERT_NO_FATAL_FAILURE(this->trim_log()); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); - } + std::unique_ptr log_reader = this->log->mem_device->new_reader( + /*slot_lower_bound=*/client_slot, llfs::LogReadMode::kSpeculative); - // Write opaque user data slot, if there is enough log space. - // - if (this->fake_slot_writer->pool_size() > kMaxOpaqueDataSize + kMaxSlotOverhead && - this->pick_branch()) { - ASSERT_NO_FATAL_FAILURE(this->append_opaque_data_slot()) << BATT_INSPECT(seed_i); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); - } + BATT_CHECK_NOT_NULLPTR(log_reader); - // Write PrepareJob slot. - // - if (this->pick_branch()) { - ASSERT_NO_FATAL_FAILURE(this->prepare_one_job()); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); - } + llfs::TypedSlotReader slot_reader{*log_reader}; + batt::StatusOr n_read = slot_reader.run( + batt::WaitForResource::kFalse, + batt::make_case_of_visitor( + [&](const llfs::SlotParse& slot, const llfs::VolumeTrimEvent& trim_event) { + next_trim_event.emplace(slot, trim_event); + return llfs::make_status(llfs::StatusCode::kBreakSlotReaderLoop); + }, + [&](const llfs::SlotParse& slot, const auto& payload) { + BATT_PANIC() << "There should always be a trim event slot at `client_slot`!" + << BATT_INSPECT(slot.offset) << ", payload type == " + << batt::name_of>(); - // Write CommitJob slot, if there is a pending job. - // - if (!this->pending_jobs.empty() && this->pick_branch()) { - ASSERT_NO_FATAL_FAILURE(this->commit_one_job()) << BATT_INSPECT(seed_i); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); - } + // Just read a single slot! + // + return llfs::make_status(llfs::StatusCode::kBreakSlotReaderLoop); + })); + [&] { + ASSERT_TRUE(n_read.ok() || n_read.status() == llfs::StatusCode::kBreakSlotReaderLoop); + }(); + [&] { + ASSERT_TRUE(next_trim_event) << "No trim event found at the specified offset!"; + }(); + } - // Simulate a crash/recovery (rate=1%) - // - if (this->pick_usize(1, 100) <= 1) { - LLFS_VLOG(1) << "Simulating crash/recovery..." - << BATT_INSPECT(this->last_recovered_trim_pos) - << BATT_INSPECT(this->fake_log->slot_range(llfs::LogReadMode::kDurable)) - << BATT_INSPECT(this->trim_control->get_lower_bound()); + //+++++++++++-+-+--+----- --- -- - - - - + // Build the list of expected page ids and compare it to actual. + // + batt::SmallVec expected_page_ids; + if (next_trim_event) { + const llfs::VolumeTrimEvent& trim_event = next_trim_event->payload; - ASSERT_NO_FATAL_FAILURE(this->shutdown_trimmer()) << BATT_INSPECT(seed_i); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); + for (auto iter = this->sim->committed_jobs.begin(); iter != this->sim->committed_jobs.end(); + iter = this->sim->committed_jobs.erase(iter)) { + const auto& [commit_slot, job_info] = *iter; - ASSERT_NO_FATAL_FAILURE(this->initialize_trimmer()); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); + [&] { + ASSERT_EQ(this->sim->pending_jobs.count(job_info.prepare_slot_offset), 0u) + << "Trimmed region contains pending jobs!"; + }(); - crash_count += 1; + if (!llfs::slot_less_than(commit_slot, trim_event.new_trim_pos)) { + break; } + const auto& page_ids = job_info.page_ids; + + LLFS_VLOG(1) << " -- expecting: " << BATT_INSPECT(commit_slot) + << BATT_INSPECT_RANGE(page_ids); + + expected_page_ids.insert(expected_page_ids.end(), page_ids.begin(), page_ids.end()); } - LLFS_VLOG(1) << "Exited loop; joining trimmer task"; + // Sort the expected page ids so we can compare it to the actual page ids. + // + std::sort(expected_page_ids.begin(), expected_page_ids.end()); + } + [&] { + ASSERT_EQ(actual_page_ids, expected_page_ids); + }(); + + //+++++++++++-+-+--+----- --- -- - - - - + // Remember that we processed this update. + // + this->sim->highest_seen_drop_pages_client_slot = client_slot; + + return batt::OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::TrimmerSession::run_one() +{ + BATT_CHECK_NOT_NULLPTR(this->sim); + BATT_CHECK_NOT_NULLPTR(this->log); + + LLFS_VLOG(1) << "Checking for tasks ready to run"; + batt::UniqueHandler<> action = this->task_context.pop_ready_handler([this](usize n) { + return this->sim->pick_usize(0, n - 1); + }); + if (action) { + ASSERT_NO_FATAL_FAILURE(action()); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +void VolumeTrimmerTest::TrimmerSession::terminate() +{ + LLFS_VLOG(1) << "TrimmerSession::terminate()" << BATT_INSPECT(this->trimmer->grant_pool_size()); + + // Tell everything to shut down. + // + this->shutting_down = true; + auto on_scope_exit = batt::finally([&] { + this->shutting_down = false; + this->trimmer = nullptr; + this->last_recovered_trim_pos = 0; + this->trimmer_task = batt::None; + this->config = nullptr; + this->sim = nullptr; + this->log = nullptr; + }); + + if (this->log) { + this->log->halt(); + } + if (this->trimmer) { + this->trimmer->halt(); + } + + // Drain the executor's queue. + // + ASSERT_NO_FATAL_FAILURE(this->task_context.poll()); + + // The trimmer task should be finished at this point. + // + if (this->trimmer_task) { + ASSERT_TRUE(this->trimmer_task->try_join()) + << "n_tasks=" << batt::Task::backtrace_all(/*force=*/true); + + LLFS_VLOG(1) << "Trimmer task joined:" << BATT_INSPECT(this->trimmer_task_exit_status); + } +} + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +TEST_F(VolumeTrimmerTest, RandomizedTest) +{ + static const bool kExtraTesting = // + batt::getenv_as("LLFS_EXTRA_TESTING") // + .value_or(0); + + static const usize kNumSeeds = // + batt::getenv_as("LLFS_TEST_NUM_SEEDS") // + .value_or(kExtraTesting ? (1000 * 1000 * 1000) : (1000 * 1000)); + + static const int kLogEveryN = // + batt::getenv_as("LLFS_TEST_LOG_EVERY_N") // + .value_or(kNumSeeds / 10); + + static const usize kInitialSeed = // + batt::getenv_as("LLFS_TEST_INITIAL_SEED") // + .value_or(0); + + static const usize kNumThreads = batt::getenv_as("LLFS_TEST_NUM_THREADS") // + .value_or(std::thread::hardware_concurrency()); + + LLFS_LOG_INFO() << BATT_INSPECT(kExtraTesting) << BATT_INSPECT(kNumSeeds) + << BATT_INSPECT(kLogEveryN) << BATT_INSPECT(kInitialSeed) + << BATT_INSPECT(kNumThreads); + + batt::Watch crash_count{0}; + batt::Watch max_log_size{0}; + + Config config; + + std::vector threads; + std::atomic running_count{0}; + + for (usize thread_i = 0; thread_i < kNumThreads; ++thread_i) { + running_count.fetch_add(1); + threads.emplace_back([&running_count, &crash_count, &config, thread_i] { + usize local_crash_count = 0; + for (usize seed_i = kInitialSeed + thread_i; seed_i < kInitialSeed + kNumSeeds; + seed_i += kNumThreads) { + //----- --- -- - - - - + LLFS_LOG_INFO_EVERY_N(kLogEveryN) + << "Starting new test run with empty log;" << BATT_INSPECT(seed_i) + << BATT_INSPECT(local_crash_count) << " (est. " << (local_crash_count * kNumThreads) + << " total)"; + + SimRun sim; + ASSERT_NO_FATAL_FAILURE(sim.reset(seed_i, config)); + + LogSession log; + ASSERT_NO_FATAL_FAILURE(log.initialize(config, sim)); + auto terminate_log = batt::finally([&] { + log.terminate(); + }); + + BATT_DEBUG_INFO(BATT_INSPECT(seed_i) << BATT_INSPECT( + log.fake_device->slot_range(llfs::LogReadMode::kSpeculative))); + + TrimmerSession trimmer; + ASSERT_NO_FATAL_FAILURE(trimmer.initialize(config, sim, log)); + auto terminate_trimmer = batt::finally([&] { + trimmer.terminate(); + }); + + // Main test loop: for each step, randomly take some action that advances the state of the + // log/trimmer. + // + LLFS_VLOG(1) << "Entering test loop"; + + for (int step_i = 0; !log.fake_log_has_failed() && step_i < kStepLimit; ++step_i) { + LLFS_VLOG(1) << "Step " << step_i; + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + + // Let the VolumeTrimmer task run. + // + if (sim.pick_branch()) { + ASSERT_NO_FATAL_FAILURE(trimmer.run_one()); + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + } + + // Trim the log. + // + if (sim.pick_branch()) { + ASSERT_NO_FATAL_FAILURE(log.trim_log()); + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + } + + // Write opaque user data slot, if there is enough log space. + // + if (log.slot_writer->pool_size() > kMaxOpaqueDataSize + kMaxSlotOverhead && + sim.pick_branch()) { + ASSERT_NO_FATAL_FAILURE(log.append_opaque_data_slot()) << BATT_INSPECT(seed_i); + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + } + + // Write PrepareJob slot. + // + if (sim.pick_branch()) { + ASSERT_NO_FATAL_FAILURE(log.prepare_one_job()); + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + } + + // Write CommitJob slot, if there is a pending job. + // + if (!sim.pending_jobs.empty() && sim.pick_branch()) { + ASSERT_NO_FATAL_FAILURE(log.commit_one_job()) << BATT_INSPECT(seed_i); + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + } + + // Simulate a crash/recovery (rate=5%) + // + if (sim.pick_usize(1, 100) <= 5) { + BATT_DEBUG_INFO(BATT_INSPECT(log.fake_log_has_failed()) + << BATT_INSPECT(log.fake_device_state->device_time) + << BATT_INSPECT(log.fake_device_state->failure_time)); + + LLFS_VLOG(1) << "Simulating crash/recovery..." + << BATT_INSPECT(trimmer.last_recovered_trim_pos) + << BATT_INSPECT(log.fake_device->slot_range(llfs::LogReadMode::kDurable)) + << BATT_INSPECT(log.trim_control->get_lower_bound()) + << BATT_INSPECT(log.fake_device->size()) + << BATT_INSPECT(log.fake_device->space()) + << BATT_INSPECT(log.fake_device->capacity()); + + trimmer.terminate(); + log.terminate(); + + ASSERT_NO_FATAL_FAILURE(log.initialize(config, sim)); + + BATT_DEBUG_INFO(BATT_INSPECT(log.job_grant->size())); + + ASSERT_NO_FATAL_FAILURE(trimmer.initialize(config, sim, log)); + ASSERT_NO_FATAL_FAILURE(trimmer.check_invariants()); + + local_crash_count += 1; + } + } + + LLFS_VLOG(1) << "Exited loop; joining trimmer task"; + } + + crash_count.fetch_add(local_crash_count); + + const usize n_threads_remaining = running_count.fetch_sub(1); + LLFS_LOG_INFO() << "Test thread " << thread_i << " finished (" << n_threads_remaining + << " still running)"; + }); + } - ASSERT_NO_FATAL_FAILURE(this->shutdown_trimmer()) << BATT_INSPECT(seed_i); - ASSERT_NO_FATAL_FAILURE(this->check_invariants()); + for (std::thread& t : threads) { + t.join(); } } diff --git a/src/llfs/volume_trimmer_recovery_visitor.cpp b/src/llfs/volume_trimmer_recovery_visitor.cpp new file mode 100644 index 0000000..c90f66a --- /dev/null +++ b/src/llfs/volume_trimmer_recovery_visitor.cpp @@ -0,0 +1,123 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#include +// + +namespace llfs { + +/*explicit*/ VolumeTrimmerRecoveryVisitor::VolumeTrimmerRecoveryVisitor( + slot_offset_type trim_pos) noexcept + : log_trim_pos_{trim_pos} +{ + LLFS_VLOG(1) << "VolumeTrimmerRecoveryVisitor created;" << BATT_INSPECT(trim_pos); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_raw_data(const SlotParse&, + const Ref&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_prepare_job(const SlotParse&, + const Ref&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_commit_job(const SlotParse&, + const Ref&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_rollback_job(const SlotParse&, + const PackedRollbackJob&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_volume_attach(const SlotParse&, + const PackedVolumeAttachEvent&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_volume_detach(const SlotParse&, + const PackedVolumeDetachEvent&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_volume_ids(const SlotParse&, + const PackedVolumeIds&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_volume_recovered(const SlotParse&, + const PackedVolumeRecovered&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_volume_format_upgrade( + const SlotParse&, const PackedVolumeFormatUpgrade&) /*override*/ +{ + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status VolumeTrimmerRecoveryVisitor::on_volume_trim(const SlotParse& slot, + const VolumeTrimEvent& trim_event) /*override*/ +{ + const bool is_pending = slot_less_than(this->log_trim_pos_, trim_event.new_trim_pos); + + LLFS_VLOG(1) << "VolumeTrimmerRecoveryVisitor::on_volume_trim(slot=" << slot.offset + << ") trimmed_region == " + << SlotRange{trim_event.old_trim_pos, trim_event.new_trim_pos} + << BATT_INSPECT(is_pending) << trim_event; + + if (is_pending) { + if (this->trim_event_info_ != None) { + LLFS_LOG_WARNING() << "Multiple pending trim events found! Likely corrupted log..."; + // + // TODO [tastolfi 2022-11-28] Force recovery to fail? + } + + this->trim_event_info_.emplace(); + this->trim_event_info_->trim_event_slot = slot.offset; + this->trim_event_info_->trimmed_region_slot_range = SlotRange{ + trim_event.old_trim_pos, + trim_event.new_trim_pos, + }; + } + + return OkStatus(); +} + +} //namespace llfs diff --git a/src/llfs/volume_trimmer_recovery_visitor.hpp b/src/llfs/volume_trimmer_recovery_visitor.hpp new file mode 100644 index 0000000..1c1c481 --- /dev/null +++ b/src/llfs/volume_trimmer_recovery_visitor.hpp @@ -0,0 +1,73 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// Part of the LLFS Project, under Apache License v2.0. +// See https://www.apache.org/licenses/LICENSE-2.0 for license information. +// SPDX short identifier: Apache-2.0 +// +//+++++++++++-+-+--+----- --- -- - - - - + +#pragma once +#ifndef LLFS_VOLUME_TRIMMER_RECOVERY_VISITOR_HPP +#define LLFS_VOLUME_TRIMMER_RECOVERY_VISITOR_HPP + +#include +// +#include +#include +#include +#include +#include +#include +#include +#include + +namespace llfs { + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +/** \brief Reconstructs trimmer state during crash recovery. + */ +class VolumeTrimmerRecoveryVisitor : public VolumeEventVisitor +{ + public: + explicit VolumeTrimmerRecoveryVisitor(slot_offset_type trim_pos) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + // VolumeEventVisitor methods. + // + Status on_raw_data(const SlotParse&, const Ref&) override; + + Status on_prepare_job(const SlotParse&, const Ref&) override; + + Status on_commit_job(const SlotParse&, const Ref&) override; + + Status on_rollback_job(const SlotParse&, const PackedRollbackJob&) override; + + Status on_volume_attach(const SlotParse& slot, const PackedVolumeAttachEvent& attach) override; + + Status on_volume_detach(const SlotParse& slot, const PackedVolumeDetachEvent& detach) override; + + Status on_volume_ids(const SlotParse& slot, const PackedVolumeIds&) override; + + Status on_volume_recovered(const SlotParse&, const PackedVolumeRecovered&) override; + + Status on_volume_format_upgrade(const SlotParse&, const PackedVolumeFormatUpgrade&) override; + + Status on_volume_trim(const SlotParse&, const VolumeTrimEvent&) override; + // + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns information about the most recent trim event slot. + */ + const Optional& get_trim_event_info() const noexcept + { + return this->trim_event_info_; + } + + private: + slot_offset_type log_trim_pos_; + Optional trim_event_info_; +}; + +} //namespace llfs + +#endif // LLFS_VOLUME_TRIMMER_RECOVERY_VISITOR_HPP