From f88f8992a079a81a5c4b2b700c1a60dc12b5b53c Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Tue, 2 Jul 2024 17:23:49 -0400 Subject: [PATCH 01/11] Add IoRingLogDevice2 files. --- src/llfs/ioring_log_config2.cpp | 30 + src/llfs/ioring_log_config2.hpp | 45 ++ src/llfs/ioring_log_device.test.hpp | 196 ++++++ src/llfs/ioring_log_device2.cpp | 108 ++++ src/llfs/ioring_log_device2.hpp | 487 +++++++++++++++ src/llfs/ioring_log_device2.ipp | 791 ++++++++++++++++++++++++ src/llfs/ioring_log_device2.test.cpp | 594 ++++++++++++++++++ src/llfs/log_device_config2.cpp | 103 +++ src/llfs/log_device_config2.hpp | 132 ++++ src/llfs/log_device_runtime_options.hpp | 98 +++ src/llfs/nested_log_device_config.hpp | 129 ++++ src/llfs/packed_log_control_block2.hpp | 81 +++ src/llfs/packed_slot_offset.hpp | 21 + 13 files changed, 2815 insertions(+) create mode 100644 src/llfs/ioring_log_config2.cpp create mode 100644 src/llfs/ioring_log_config2.hpp create mode 100644 src/llfs/ioring_log_device.test.hpp create mode 100644 src/llfs/ioring_log_device2.cpp create mode 100644 src/llfs/ioring_log_device2.hpp create mode 100644 src/llfs/ioring_log_device2.ipp create mode 100644 src/llfs/ioring_log_device2.test.cpp create mode 100644 src/llfs/log_device_config2.cpp create mode 100644 src/llfs/log_device_config2.hpp create mode 100644 src/llfs/log_device_runtime_options.hpp create mode 100644 src/llfs/nested_log_device_config.hpp create mode 100644 src/llfs/packed_log_control_block2.hpp create mode 100644 src/llfs/packed_slot_offset.hpp diff --git a/src/llfs/ioring_log_config2.cpp b/src/llfs/ioring_log_config2.cpp new file mode 100644 index 0000000..f55a8cc --- /dev/null +++ b/src/llfs/ioring_log_config2.cpp @@ -0,0 +1,30 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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 + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*static*/ IoRingLogConfig2 IoRingLogConfig2::from_packed( + const FileOffsetPtr& packed_config) +{ + return IoRingLogConfig2{ + .control_block_offset = + packed_config.absolute_from_relative_offset(packed_config->control_block_offset), + .log_capacity = packed_config->logical_size, + .device_page_size_log2 = packed_config->device_page_size_log2, + .data_alignment_log2 = packed_config->data_alignment_log2, + }; +} + +} //namespace llfs diff --git a/src/llfs/ioring_log_config2.hpp b/src/llfs/ioring_log_config2.hpp new file mode 100644 index 0000000..02f1c5a --- /dev/null +++ b/src/llfs/ioring_log_config2.hpp @@ -0,0 +1,45 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_IORING_LOG_CONFIG2_HPP +#define LLFS_IORING_LOG_CONFIG2_HPP + +#include +// + +#include +#include +#include + +namespace llfs { + +struct PackedLogDeviceConfig2; + +struct IoRingLogConfig2 { + static IoRingLogConfig2 from_packed( + const FileOffsetPtr& packed_config); + + //+++++++++++-+-+--+----- --- -- - - - - + + i64 control_block_offset; + u64 log_capacity; + u16 device_page_size_log2; + u16 data_alignment_log2; + + //+++++++++++-+-+--+----- --- -- - - - - + + i64 control_block_size() const noexcept + { + return i64{1} << this->data_alignment_log2; + } +}; + +} //namespace llfs + +#endif // LLFS_IORING_LOG_CONFIG2_HPP diff --git a/src/llfs/ioring_log_device.test.hpp b/src/llfs/ioring_log_device.test.hpp new file mode 100644 index 0000000..dcffb72 --- /dev/null +++ b/src/llfs/ioring_log_device.test.hpp @@ -0,0 +1,196 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_IORINT_LOG_DEVICE_TEST_HPP +#define LLFS_IORINT_LOG_DEVICE_TEST_HPP + +#include +#include +#include + +#include + +#include +#include +#include +#include + +namespace llfs { + +template +T read_test_var(const char* name, T default_value) +{ + using value_type = decltype(default_value); + const value_type value = batt::getenv_as(name).value_or(default_value); + std::cout << name << "=" << value << std::endl; + return value; +} + +inline void run_log_device_benchmark( + const std::function&)>& inject_log_device) +{ + const usize log_size = read_test_var("LLFS_LOG_DEVICE_SIZE_KB", usize{1024 * 64}) * 1024; + const usize total_to_write = read_test_var("LLFS_LOG_DEVICE_WRITE_KB", usize{1024 * 1024}) * 1024; + const usize append_size = read_test_var("LLFS_LOG_DEVICE_APPEND_SIZE", usize{256}); + const usize trim_size = read_test_var("LLFS_LOG_DEVICE_TRIM_SIZE", usize{4 * 1024 * 1024}); + const usize trim_trigger = read_test_var("LLFS_LOG_DEVICE_TRIM_TRIGGER", // + usize{log_size - trim_size * 2}); + const usize repeat_count = read_test_var("LLFS_LOG_DEVICE_REPEAT", usize{3}); + + // Generate some random data. + // + std::vector data(32 * 1024 * 1024); + std::default_random_engine rng{1}; + for (u64& word : data) { + word = rng(); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // + for (usize retry = 0; retry < repeat_count; ++retry) { + inject_log_device(log_size, /*create=*/true, [&](llfs::LogDevice& log_device) { + const bool last_iteration = retry + 1 == repeat_count; + + const usize payload_size = append_size; + const usize header_size = llfs::packed_sizeof_varint(payload_size); + const usize slot_size = header_size + payload_size; + + auto start = std::chrono::steady_clock::now(); + + std::thread writer_thread{[&] { + llfs::LogDevice::Writer& log_writer = log_device.writer(); + + std::uniform_int_distribution pick_offset{ + 0, data.size() - (append_size + sizeof(u64) - 1) / sizeof(u64)}; + + usize n_written = 0; + while (n_written < total_to_write) { + BATT_CHECK_OK(log_writer.await(llfs::BytesAvailable{.size = slot_size})); + + llfs::StatusOr buffer = log_writer.prepare(slot_size); + BATT_CHECK_OK(buffer); + + // Pack the varint header (slot payload size). + // + *buffer = batt::get_or_panic(pack_varint_to(*buffer, payload_size)); + + // Pick a random offset and write it to the slot. + // + usize offset = pick_offset(rng); + *((little_u64*)buffer->data()) = offset; + *buffer += sizeof(little_u64); + + // Copy random data from the selected offset. + // + std::memcpy(buffer->data(), &data[offset], buffer->size()); + + BATT_CHECK_OK(log_writer.commit(slot_size)); + + n_written += slot_size; + } + + if (!last_iteration) { + BATT_CHECK_OK(log_device.flush()); + } + + const auto flushed_range = log_device.slot_range(LogReadMode::kDurable); + const auto committed_range = log_device.slot_range(LogReadMode::kSpeculative); + + LLFS_LOG_INFO() << BATT_INSPECT(flushed_range) << BATT_INSPECT(flushed_range.size()) + << BATT_INSPECT(committed_range) << BATT_INSPECT(committed_range.size()) + << BATT_INSPECT(last_iteration); + + log_device.halt(); + }}; + + std::thread trimmer_thread{[&] { + llfs::SlotRange durable = log_device.slot_range(llfs::LogReadMode::kDurable); + + const usize aligned_trim_size = batt::round_up_to(slot_size, trim_size); + + for (;;) { + llfs::Status sync_status = + log_device.sync(llfs::LogReadMode::kDurable, + llfs::SlotUpperBoundAt{durable.lower_bound + trim_trigger}); + + if (!sync_status.ok()) { + LLFS_LOG_INFO() << BATT_INSPECT(durable.lower_bound); + break; + } + + llfs::Status trim_status = log_device.trim(durable.lower_bound + aligned_trim_size); + + if (!trim_status.ok()) { + LLFS_LOG_INFO() << BATT_INSPECT(durable.lower_bound); + break; + } + + durable.lower_bound += aligned_trim_size; + } + }}; + + writer_thread.join(); + trimmer_thread.join(); + log_device.join(); + + auto finish = std::chrono::steady_clock::now(); + + double duration_sec = + double(std::chrono::duration_cast(finish - start).count()) / + (1000.0 * 1000.0); + + LLFS_LOG_INFO() << total_to_write << " bytes written in " << duration_sec + << " seconds; rate=" << (double(total_to_write) / duration_sec) / 1000000.0 + << "MB/s"; + }); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // + LLFS_LOG_INFO() << "Verifying log contents; opening..."; + + inject_log_device(log_size, /*create=*/false, [&](llfs::LogDevice& log_device) { + LLFS_LOG_INFO() << "Log recovered; verifying..."; + + const auto flushed_range = log_device.slot_range(LogReadMode::kDurable); + const auto committed_range = log_device.slot_range(LogReadMode::kSpeculative); + + LLFS_LOG_INFO() << BATT_INSPECT(flushed_range) << BATT_INSPECT(flushed_range.size()) + << BATT_INSPECT(committed_range) << BATT_INSPECT(committed_range.size()); + + std::unique_ptr log_reader = + log_device.new_reader(llfs::None, llfs::LogReadMode::kDurable); + + BATT_CHECK_NOT_NULLPTR(log_reader); + + llfs::SlotReader slot_reader{*log_reader}; + + llfs::StatusOr n_parsed = slot_reader.run( + batt::WaitForResource::kFalse, [&](const llfs::SlotParse& slot) -> llfs::Status { + EXPECT_EQ(slot.body.size(), append_size); + + const usize offset = *((const little_u64*)slot.body.data()); + + BATT_CHECK_LE(offset + slot.body.size(), data.size() * sizeof(u64)); + + EXPECT_EQ(0, + std::memcmp((const char*)&data[offset], slot.body.data() + sizeof(little_u64), + slot.body.size() - sizeof(little_u64))); + + return llfs::OkStatus(); + }); + + LLFS_LOG_INFO() << "Done;" << BATT_INSPECT(n_parsed); + }); +} + +} //namespace llfs + +#endif // LLFS_IORINT_LOG_DEVICE_TEST_HPP diff --git a/src/llfs/ioring_log_device2.cpp b/src/llfs/ioring_log_device2.cpp new file mode 100644 index 0000000..bb852e7 --- /dev/null +++ b/src/llfs/ioring_log_device2.cpp @@ -0,0 +1,108 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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 { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status initialize_log_device2(RawBlockFile& file, const IoRingLogConfig2& config) +{ + using AlignedUnit = std::aligned_storage_t; + + const usize device_page_size = usize{1} << config.device_page_size_log2; + const usize buffer_size = batt::round_up_to(sizeof(AlignedUnit), device_page_size); + + BATT_CHECK_EQ(buffer_size, device_page_size); + + std::unique_ptr buffer{new AlignedUnit[buffer_size / sizeof(AlignedUnit)]}; + std::memset(buffer.get(), 0, buffer_size); + + auto* control_block = reinterpret_cast(buffer.get()); + + control_block->magic = PackedLogControlBlock2::kMagic; + control_block->data_size = config.log_capacity; + control_block->trim_pos = 0; + control_block->flush_pos = 0; + control_block->generation = 0; + control_block->control_block_size = BATT_CHECKED_CAST(u32, device_page_size); + control_block->control_header_size = BATT_CHECKED_CAST(u32, sizeof(PackedLogControlBlock2)); + control_block->device_page_size_log2 = config.device_page_size_log2; + control_block->data_alignment_log2 = config.data_alignment_log2; + + BATT_REQUIRE_OK( + write_all(file, config.control_block_offset, ConstBuffer{buffer.get(), device_page_size})); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ IoRingLogDevice2Factory::IoRingLogDevice2Factory( + int fd, const FileOffsetPtr& packed_config, + const LogDeviceRuntimeOptions& options) noexcept + : IoRingLogDevice2Factory{fd, IoRingLogConfig2::from_packed(packed_config), options} +{ +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +/*explicit*/ IoRingLogDevice2Factory::IoRingLogDevice2Factory( + int fd, const IoRingLogConfig2& config, const LogDeviceRuntimeOptions& options) noexcept + : fd_{fd} + , config_{config} + , options_{options} +{ +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +IoRingLogDevice2Factory::~IoRingLogDevice2Factory() noexcept +{ + if (this->fd_ != -1) { + ::close(this->fd_); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr> IoRingLogDevice2Factory::open_ioring_log_device() +{ + BATT_ASSIGN_OK_RESULT(DefaultIoRingLogDeviceStorage storage, + DefaultIoRingLogDeviceStorage::make_new( + MaxQueueDepth{this->options_.max_concurrent_writes * 2}, this->fd_)); + + this->fd_ = -1; + + auto instance = + std::make_unique(this->config_, this->options_, std::move(storage)); + + Status open_status = instance->open(); + BATT_REQUIRE_OK(open_status); + + return instance; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr> IoRingLogDevice2Factory::open_log_device( + const LogScanFn& scan_fn) /*override*/ +{ + auto instance = this->open_ioring_log_device(); + BATT_REQUIRE_OK(instance); + + auto scan_status = + scan_fn(*(*instance)->new_reader(/*slot_lower_bound=*/None, LogReadMode::kDurable)); + BATT_REQUIRE_OK(scan_status); + + return instance; +} + +} //namespace llfs diff --git a/src/llfs/ioring_log_device2.hpp b/src/llfs/ioring_log_device2.hpp new file mode 100644 index 0000000..687ed70 --- /dev/null +++ b/src/llfs/ioring_log_device2.hpp @@ -0,0 +1,487 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_IORING_LOG_DEVICE2_HPP +#define LLFS_IORING_LOG_DEVICE2_HPP + +#include +// +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +namespace llfs { + +BATT_STRONG_TYPEDEF(slot_offset_type, TargetTrimPos); +BATT_STRONG_TYPEDEF(slot_offset_type, CommitPos); + +/** \brief Initializes an IoRingLogDevice2 using the given storage device and configuration (which + * includes offset within the passed device). + */ +Status initialize_log_device2(RawBlockFile& file, const IoRingLogConfig2& config); + +template +class IoRingLogDriver2 +{ + public: + using Self = IoRingLogDriver2; + using AlignedUnit = std::aligned_storage_t; + using EventLoopTask = typename StorageT::EventLoopTask; + + static constexpr batt::StaticType kTargetTrimPos{}; + static constexpr batt::StaticType kCommitPos{}; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief The size (bytes) of each preallocated completion handler memory object. + */ + static constexpr usize kHandlerMemorySize = 160; + + using HandlerMemory = batt::HandlerMemory; + + using HandlerMemoryStorage = + std::aligned_storage_t; + + //+++++++++++-+-+--+----- --- -- - - - - + + explicit IoRingLogDriver2(LogStorageDriverContext& context, // + const IoRingLogConfig2& config, // + const LogDeviceRuntimeOptions& options, // + StorageT&& storage // + ) noexcept; + + //---- + + Status set_trim_pos(slot_offset_type trim_pos) + { + this->observed_watch_[kTargetTrimPos].set_value(trim_pos); + return OkStatus(); + } + + slot_offset_type get_trim_pos() const + { + return this->trim_pos_.get_value(); + } + + StatusOr await_trim_pos(slot_offset_type min_offset) + { + return await_slot_offset(min_offset, this->trim_pos_); + } + + //---- + + slot_offset_type get_flush_pos() const + { + return this->flush_pos_.get_value(); + } + + StatusOr await_flush_pos(slot_offset_type min_offset) + { + return await_slot_offset(min_offset, this->flush_pos_); + } + + //---- + + Status set_commit_pos(slot_offset_type commit_pos) + { + this->observed_watch_[kCommitPos].set_value(commit_pos); + return OkStatus(); + } + + slot_offset_type get_commit_pos() const + { + return this->observed_watch_[kCommitPos].get_value(); + } + + StatusOr await_commit_pos(slot_offset_type min_offset) + { + return await_slot_offset(min_offset, this->observed_watch_[kCommitPos]); + } + + //---- + + Status open() noexcept; + + Status close(); + + void halt(); + + void join(); + + //+++++++++++-+-+--+----- --- -- - - - - + + private: + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Returns the current value of either the TargetTrimPos or CommitPos Watch. + */ + template + T observe(T) const noexcept + { + return T{this->observed_watch_[batt::StaticType{}].get_value()}; + } + + /** \brief Reads the control block into memory and initializes member data that depend on it. + */ + Status read_control_block(); + + /** \brief Reads the entire contents of the log into the ring buffer. + */ + Status read_log_data(); + + /** \brief Returns upper bound of known slot commit points recovered from the control block. + * + * This will either be the highest of the commit points inside the control block, without + * exceeding the recovered flush position, or the recovered trim position, whichever is greater. + */ + slot_offset_type recover_flushed_commit_point() const noexcept; + + /** \brief Starts at the value returned by this->recover_flushed_commit_point() and scans forward + * in the log data, parsing slot headers until we reach a partially flushed slot or run out of + * data. Calls this->reset_flush_pos with the slot upper bound of the highest confirmed slot from + * the scan. + */ + Status recover_flush_pos() noexcept; + + /** \brief Forces all slot lower bound pointers to `new_trim_pos`. + * + * Resets: + * - this->trim_pos_ + * - this->target_trim_pos_ + * + * Should only be called during recovery. + */ + void reset_trim_pos(slot_offset_type new_trim_pos); + + /** \brief Forces all slot upper bound pointers to `new_flush_pos`. + * + * Resets: + * - this->flush_pos_ + * - this->commit_pos_ + * - this->started_flush_upper_bound_ + * - this->known_flush_pos_ + * + * Should only be called during recovery. + */ + void reset_flush_pos(slot_offset_type new_flush_pos); + + /** \brief Checks to see what activities can be initiated to advance the state of the driver. + * + * - Starts flushing data from the ring buffer, if conditions are met (see start_flush) + * - Starts updating the control block, if trim/flush are out of date + * - Starts waiting for updates from the target trim and commit pos + */ + void poll() noexcept; + + /** \brief Initiates an async wait on the specified watched value (either CommitPos or + * TargetTrimPos), if there is not already a pending wait operation in progress. + */ + template + void wait_for_slot_offset_change(T observed_value) noexcept; + + /** \brief Checks to see if data can be flushed, and initiates at least one async write if so. + * + * The following conditions must be met to initiate an async write: + * + * - The current number of writes pending must be below the maximum limit + * - There must be some unflushed data in the ring buffer + * - The amount of unflushed data must be at least the flush delay threshold OR there are no + * writes pending currently + * + * If the unflushed data region spans the upper bound of the ring buffer (wrap-around), then up to + * two writes may be initiated by this function, provided the conditions above are still met after + * starting the first write. + */ + void start_flush(slot_offset_type observed_commit_pos); + + /** \brief Returns the passed slot range with the lower and upper bounds aligned to the nearest + * data page boundary. + */ + SlotRange get_aligned_range(const SlotRange& slot_range) const noexcept; + + /** \brief Returns the slot range corresponding to the trailing data page of the passed range. + */ + SlotRange get_aligned_tail(const SlotRange& aligned_range) const noexcept; + + /** \brief Starts writing the aligned range from the ring buffer to the storage media. + * + * Unlike start_flush, this function unconditionally starts an async write. + */ + void start_flush_write(const SlotRange& slot_range, const SlotRange& aligned_range) noexcept; + + /** \brief Completion handler for a completed data flush write operation. + * + * Will initiate another write if the tail of the aligned range is now dirty and another flush + * write has been initiated since the write that caused this handler to be called. + * + * Always calls poll unless there is an error status (result). + */ + void handle_flush_write(const SlotRange& slot_range, const SlotRange& aligned_range, + StatusOr result); + + /** \brief Updates this->known_flush_pos_ to include the passed flushed_range. + * + * Inserts the flushed range into this->flushed_ranges_ min-heap and then consumes all available + * contiguous ranges to advance this->known_flush_pos_. + * + * This should be called once some slot range is known to have been successfully flushed to the + * storage media. + */ + void update_known_flush_pos(const SlotRange& flushed_range) noexcept; + + /** \brief Initiates a rewrite of the control block if necessary. + * + * The control block must be updated when the target trim pos or unknown flush pos become out of + * sync with the last written values. + * + * Only one pending async write to the control block is allowed at a time. + */ + void start_control_block_update(slot_offset_type observed_target_trim_pos) noexcept; + + /** \brief I/O callback that handles the completion of a write to the control block. + */ + void handle_control_block_update(StatusOr result) noexcept; + + /** \brief Handles all write errors. + */ + void handle_write_error(Status status) noexcept; + + /** \brief Returns a wrapped handler for async writes. + * + * Automatically injects `this` as the first arg to `handler`, so that callers need not capture + * this in handler itself. + */ + template + auto make_write_handler(HandlerFn&& handler); + + /** \brief Allocates an array of HandlerMemoryStorage objects, adding each to the free pool linked + * list (this->handler_memory_pool_). + */ + void initialize_handler_memory_pool(); + + /** \brief Pops the next HandlerMemoryStorage object off this->handler_memory_pool_ and uses it to + * construct a new HandlerMemory object, returning a pointer to the newly constructed + * HandlerMemory. + * + * This function MUST only be called on the event loop thread. + */ + HandlerMemory* alloc_handler_memory() noexcept; + + /** \brief Destructs the passed HandlerMemory object and adds its storage back to the pool. + * + * This function MUST only be called on the event loop thread. + */ + void free_handler_memory(HandlerMemory* p_mem) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief The context passed in at construction time; provides access to the ring buffer. + */ + LogStorageDriverContext& context_; + + /** \brief The configuration of the log; passed in at construction time. + */ + const IoRingLogConfig2 config_; + + /** \brief Runtime configuration options, such as the maximum number of concurrent writes allowed. + */ + const LogDeviceRuntimeOptions options_; + + /** \brief Provides access to the underlying storage media. + */ + StorageT storage_; + + /** \brief The minimum IO page size for the storage media. + */ + const usize device_page_size_ = usize{1} << this->config_.device_page_size_log2; + + /** \brief The size to which data writes must be aligned; this is also the distance from the start + * of the control block to the start of the log data storage area. Typically aligns with memory + * pages (4k by default). + */ + const usize data_page_size_ = usize{1} << this->config_.data_alignment_log2; + + /** \brief The absolute file (media) offset of the beginning of the data region. + */ + i64 data_begin_; + + /** \brief The absolute file (media) offset of the end (non-inclusive) of the data region. + */ + i64 data_end_; + + /** \brief True iff storage_.on_work_started() has been called (inside open). + */ + std::atomic work_started_{false}; + + /** \brief True iff halt has been called. + */ + std::atomic halt_requested_{false}; + + /** \brief Stores the two externally modifiable Watch objects (target trim pos and commit pos). + */ + batt::StaticTypeMap, batt::Watch> + observed_watch_; + + /** \brief Stores the wait status of the two observed watches (i.e., whether we are currently + * waiting on updates for each watch). + */ + batt::StaticTypeMap, bool> waiting_; + + /** \brief The confirmed trim position; this is set only after a successful re-write of the + * control block. + */ + batt::Watch trim_pos_{0}; + + /** \brief The confirmed flush position; this is set only after a successful re-write of the + * control block. + */ + batt::Watch flush_pos_{0}; + + /** \brief The greatest lower bound of unflushed log data. This is compared against the commit + * pos to see whether there is any unflushed data. + */ + slot_offset_type unflushed_lower_bound_ = 0; + + /** \brief The least upper bound of contiguous flushed data in the log. Updates to this value + * should trigger an update of the control block. + */ + slot_offset_type known_flush_pos_ = 0; + + /** \brief The trailing aligned data page for the highest-offset pending flush operation. This is + * used to avoid concurrently updating the same data page. + */ + Optional flush_tail_; + + /** \brief Set to true when we are re-writing the control block; used to avoid concurrent writes + * to the control block. + */ + bool writing_control_block_ = false; + + /** \brief The current number of pending flush writes. This does not include writing to the + * control block. + */ + usize writes_pending_ = 0; + + /** \brief The maximum observed value of this->writes_pending_ (high water mark). + */ + usize writes_max_ = 0; + + /** \brief Buffer containing the control block structure. + */ + std::unique_ptr control_block_memory_; + + /** \brief The data buffer used to write updates to the control block; points at + * this->control_block_memory_. + */ + ConstBuffer control_block_buffer_; + + /** \brief Pointer to initialized control block for the log. + */ + PackedLogControlBlock2* control_block_ = nullptr; + + /** \brief A min-heap of confirmed flushed slot ranges; used to advance this->known_flush_pos_, + * which in turn drives the update of the control block and this->flush_pos_. + */ + std::vector flushed_ranges_; + + /** \brief A background task running the storage event loop. + */ + Optional event_loop_task_; + + /** \brief Used to make sure certain operations are race-free (i.e. they only happen on the event + * loop task thread). + */ + std::thread::id event_thread_id_; + + //+++++++++++-+-+--+----- --- -- - - - - + // HandlerMemory pool. + //----- --- -- - - - - + + /** \brief Aligned storage for HandlerMemory objects. + */ + std::unique_ptr handler_memory_; + + /** \brief The head of a single-linked list of free HandlerMemoryStorage objects. + */ + HandlerMemoryStorage* handler_memory_pool_ = nullptr; +}; + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +/** \brief An emphemeral LogDevice that stores data in memory only. + * + * The commit pos and flush pos are always in sync, so there is no difference between + * LogReadMode::kSpeculative and LogReadMode::kDurable for this log device type. + */ +template +class BasicIoRingLogDevice2 + : public BasicRingBufferLogDevice< + /*Impl=*/IoRingLogDriver2> +{ + public: + using Super = BasicRingBufferLogDevice< + /*Impl=*/IoRingLogDriver2>; + + //+++++++++++-+-+--+----- --- -- - - - - + + explicit BasicIoRingLogDevice2(const IoRingLogConfig2& config, + const LogDeviceRuntimeOptions& options, + StorageT&& storage) noexcept + : Super{RingBuffer::TempFile{.byte_size = BATT_CHECKED_CAST(usize, config.log_capacity)}, + config, options, std::move(storage)} + { + } +}; + +using IoRingLogDevice2 = BasicIoRingLogDevice2; + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +/** \brief A factory that produces IoRingLogDevice2 instances of the given size. + */ +class IoRingLogDevice2Factory : public LogDeviceFactory +{ + public: + explicit IoRingLogDevice2Factory( + int fd, const FileOffsetPtr& packed_config, + const LogDeviceRuntimeOptions& options) noexcept; + + explicit IoRingLogDevice2Factory(int fd, const IoRingLogConfig2& config, + const LogDeviceRuntimeOptions& options) noexcept; + + ~IoRingLogDevice2Factory() noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + + StatusOr> open_ioring_log_device(); + + StatusOr> open_log_device(const LogScanFn& scan_fn) override; + + //+++++++++++-+-+--+----- --- -- - - - - + private: + int fd_; + IoRingLogConfig2 config_; + LogDeviceRuntimeOptions options_; +}; + +} // namespace llfs + +#endif // LLFS_IORING_LOG_DEVICE2_HPP + +#include diff --git a/src/llfs/ioring_log_device2.ipp b/src/llfs/ioring_log_device2.ipp new file mode 100644 index 0000000..9799735 --- /dev/null +++ b/src/llfs/ioring_log_device2.ipp @@ -0,0 +1,791 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_IORING_LOG_DEVICE2_IPP +#define LLFS_IORING_LOG_DEVICE2_IPP + +#include +#include + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline /*explicit*/ IoRingLogDriver2::IoRingLogDriver2( + LogStorageDriverContext& context, // + const IoRingLogConfig2& config, // + const LogDeviceRuntimeOptions& options, // + StorageT&& storage // + ) noexcept + : context_{context} + , config_{config} + , options_{options} + , storage_{std::move(storage)} + , control_block_memory_{new AlignedUnit[(this->data_page_size_ + sizeof(AlignedUnit) - 1) / + sizeof(AlignedUnit)]} + , control_block_buffer_{this->control_block_memory_.get(), this->data_page_size_} +{ + BATT_CHECK_GE(this->config_.data_alignment_log2, this->config_.device_page_size_log2); + BATT_CHECK_GE(this->data_page_size_, sizeof(PackedLogControlBlock2)); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline Status IoRingLogDriver2::open() noexcept +{ + LLFS_VLOG(1) << "open()"; + + this->initialize_handler_memory_pool(); + + BATT_REQUIRE_OK(this->storage_.register_fd()); + BATT_REQUIRE_OK(this->storage_.register_buffers( + seq::single_item(MutableBuffer{this->control_block_memory_.get(), this->data_page_size_}) | + seq::boxed())); + + LLFS_VLOG(1) << "starting event loop"; + + this->storage_.on_work_started(); + this->work_started_.store(true); + + this->event_loop_task_.emplace(this->storage_, "IoRingLogDriver2::open()"); + + BATT_REQUIRE_OK(this->read_control_block()); + BATT_REQUIRE_OK(this->read_log_data()); + + this->storage_.post_to_event_loop([this](auto&&... /*ignored*/) { + LLFS_VLOG(1) << "initial event"; + + this->event_thread_id_ = std::this_thread::get_id(); + this->poll(); + }); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline Status IoRingLogDriver2::close() +{ + this->halt(); + this->join(); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void IoRingLogDriver2::halt() +{ + const bool previously_halted = this->halt_requested_.exchange(true); + if (!previously_halted) { + for (batt::Watch& watch : this->observed_watch_) { + watch.close(); + } + this->trim_pos_.close(); + this->flush_pos_.close(); + + if (this->work_started_.exchange(false)) { + this->storage_.on_work_finished(); + } + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void IoRingLogDriver2::join() +{ + if (this->event_loop_task_) { + this->event_loop_task_->join(); + this->event_loop_task_ = None; + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline Status IoRingLogDriver2::read_control_block() +{ + LLFS_VLOG(1) << "read_control_block()"; + + const usize control_block_size = this->data_page_size_; + + auto* p_control_block = + reinterpret_cast(this->control_block_memory_.get()); + + std::memset(p_control_block, 0, control_block_size); + + MutableBuffer mutable_buffer{p_control_block, control_block_size}; + + BATT_REQUIRE_OK(this->storage_.read_all(this->config_.control_block_offset, mutable_buffer)); + + const slot_offset_type recovered_trim_pos = p_control_block->trim_pos; + const slot_offset_type recovered_flush_pos = p_control_block->flush_pos; + + LLFS_VLOG(1) << BATT_INSPECT(recovered_trim_pos) << BATT_INSPECT(recovered_flush_pos) + << BATT_INSPECT(p_control_block->generation); + + LLFS_CHECK_SLOT_LE(recovered_trim_pos, recovered_flush_pos); + + this->reset_trim_pos(recovered_trim_pos); + this->reset_flush_pos(recovered_flush_pos); + + this->data_begin_ = this->config_.control_block_offset + this->data_page_size_; + this->data_end_ = this->data_begin_ + p_control_block->data_size; + + this->control_block_buffer_ = ConstBuffer{ + p_control_block, + batt::round_up_to(this->device_page_size_, p_control_block->control_header_size), + }; + this->control_block_ = p_control_block; + + // TODO [tastolfi 2024-06-11] verify control block values against config where possible. + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline Status IoRingLogDriver2::read_log_data() +{ + LLFS_VLOG(1) << "read_log_data()"; + + const slot_offset_type recovered_trim_pos = this->trim_pos_.get_value(); + const slot_offset_type recovered_flush_pos = this->flush_pos_.get_value(); + + const slot_offset_type aligned_trim_pos = + batt::round_down_bits(this->config_.device_page_size_log2, recovered_trim_pos); + + const slot_offset_type aligned_flush_pos = + batt::round_up_bits(this->config_.device_page_size_log2, recovered_flush_pos); + + const i64 read_begin_offset = + this->data_begin_ + BATT_CHECKED_CAST(i64, aligned_trim_pos % this->context_.buffer_.size()); + + MutableBuffer mutable_buffer = resize_buffer(this->context_.buffer_.get_mut(aligned_trim_pos), + aligned_flush_pos - aligned_trim_pos); + + const i64 read_end_offset = read_begin_offset + BATT_CHECKED_CAST(i64, mutable_buffer.size()); + + LLFS_VLOG(1) << BATT_INSPECT(recovered_trim_pos) << BATT_INSPECT(recovered_flush_pos); + + if (read_end_offset <= this->data_end_) { + BATT_REQUIRE_OK(this->storage_.read_all(read_begin_offset, mutable_buffer)); + + } else { + // lower == lower offset in the file. + // + const usize lower_part_size = read_end_offset - this->data_end_; + const usize upper_part_size = this->data_end_ - read_begin_offset; + + BATT_CHECK_EQ(mutable_buffer.size(), lower_part_size + upper_part_size); + + MutableBuffer lower_part_buffer = mutable_buffer + upper_part_size; + MutableBuffer upper_part_buffer = resize_buffer(mutable_buffer, upper_part_size); + + BATT_CHECK_EQ(lower_part_buffer.size(), lower_part_size); + BATT_CHECK_EQ(upper_part_buffer.size(), upper_part_size); + + BATT_REQUIRE_OK(this->storage_.read_all(this->data_begin_, lower_part_buffer)); + BATT_REQUIRE_OK(this->storage_.read_all(read_begin_offset, upper_part_buffer)); + } + + return this->recover_flush_pos(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +slot_offset_type IoRingLogDriver2::recover_flushed_commit_point() const noexcept +{ + const slot_offset_type recovered_trim_pos = this->control_block_->trim_pos; + const slot_offset_type recovered_flush_pos = this->control_block_->flush_pos; + + slot_offset_type slot_offset = recovered_trim_pos; + + std::vector sorted_commit_points(this->control_block_->commit_points.begin(), + this->control_block_->commit_points.end()); + + std::sort(sorted_commit_points.begin(), sorted_commit_points.end(), SlotOffsetOrder{}); + + LLFS_VLOG(1) << BATT_INSPECT_RANGE(sorted_commit_points); + + auto iter = std::upper_bound(sorted_commit_points.begin(), sorted_commit_points.end(), + recovered_flush_pos, SlotOffsetOrder{}); + + if (iter != sorted_commit_points.begin()) { + --iter; + slot_offset = slot_max(slot_offset, *iter); + LLFS_VLOG(1) << " -- using commit point: " << *iter; + } + + LLFS_CHECK_SLOT_LE(slot_offset, recovered_flush_pos); + return slot_offset; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +Status IoRingLogDriver2::recover_flush_pos() noexcept +{ + const slot_offset_type recovered_flush_pos = this->flush_pos_.get_value(); + + slot_offset_type slot_offset = this->recover_flushed_commit_point(); + + ConstBuffer buffer = + resize_buffer(this->context_.buffer_.get(slot_offset), recovered_flush_pos - slot_offset); + + slot_offset_type confirmed_flush_pos = slot_offset; + + // This should be correct, since commit is called only once per atomic range, and atomic ranges + // are only recoverable if no part of the range (including the begin/end tokens) has been trimmed. + // + bool inside_atomic_range = false; + + for (;;) { + DataReader reader{buffer}; + const usize bytes_available_before = reader.bytes_available(); + Optional slot_body_size = reader.read_varint(); + + if (!slot_body_size) { + // Partially committed slot (couldn't even read a whole varint for the slot header!) Break + // out of the loop. + // + LLFS_VLOG(1) << " -- Incomplete slot header, exiting loop;" << BATT_INSPECT(slot_offset) + << BATT_INSPECT(bytes_available_before); + break; + } + + const usize bytes_available_after = reader.bytes_available(); + const usize slot_header_size = bytes_available_before - bytes_available_after; + const usize slot_size = slot_header_size + *slot_body_size; + + if (slot_size > buffer.size()) { + // Partially committed slot; break out of the loop without updating slot_offset (we're + // done!) + // + LLFS_VLOG(1) << " -- Incomplete slot body, exiting loop;" << BATT_INSPECT(slot_offset) + << BATT_INSPECT(bytes_available_before) << BATT_INSPECT(bytes_available_after) + << BATT_INSPECT(slot_header_size) << BATT_INSPECT(slot_body_size) + << BATT_INSPECT(slot_size); + break; + } + + // Check for control token; this indicates the beginning or end of an atomic slot range. + // + if (*slot_body_size == 0) { + if (slot_header_size == SlotWriter::WriterLock::kBeginAtomicRangeTokenSize) { + inside_atomic_range = true; + } else if (slot_header_size == SlotWriter::WriterLock::kEndAtomicRangeTokenSize) { + inside_atomic_range = false; + } + } + + buffer += slot_size; + slot_offset += slot_size; + + // If inside an atomic slot range, we hold off on updating the confirmed_flush_pos, just in + // case the flushed data is cut off before the end of the atomic range. + // + if (!inside_atomic_range) { + confirmed_flush_pos = slot_offset; + } + } + + LLFS_VLOG(1) << " -- Slot scan complete;" << BATT_INSPECT(slot_offset); + + this->reset_flush_pos(confirmed_flush_pos); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::reset_trim_pos(slot_offset_type new_trim_pos) +{ + this->trim_pos_.set_value(new_trim_pos); + this->observed_watch_[kTargetTrimPos].set_value(new_trim_pos); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::reset_flush_pos(slot_offset_type new_flush_pos) +{ + this->observed_watch_[kCommitPos].set_value(new_flush_pos); + this->unflushed_lower_bound_ = new_flush_pos; + this->known_flush_pos_ = new_flush_pos; + this->flush_pos_.set_value(new_flush_pos); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void IoRingLogDriver2::poll() noexcept +{ + auto observed_commit_pos = this->observe(CommitPos{}); + auto observed_target_trim_pos = this->observe(TargetTrimPos{}); + + LLFS_VLOG(1) << "poll()" << BATT_INSPECT(observed_commit_pos) + << BATT_INSPECT(observed_target_trim_pos); + + this->start_flush(observed_commit_pos); + this->start_control_block_update(observed_target_trim_pos); + this->wait_for_slot_offset_change(TargetTrimPos{observed_target_trim_pos}); + this->wait_for_slot_offset_change(CommitPos{observed_commit_pos}); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +template +inline void IoRingLogDriver2::wait_for_slot_offset_change(T observed_value) noexcept +{ + static constexpr batt::StaticType kKey; + + if (this->waiting_[kKey]) { + return; + } + this->waiting_[kKey] = true; + + HandlerMemory* const p_mem = this->alloc_handler_memory(); + + this->observed_watch_[kKey].async_wait( + observed_value, + + // Use pre-allocated memory to store the handler in the watch observer list. + // + batt::make_custom_alloc_handler( + *p_mem, [this, p_mem](const StatusOr& new_value) mutable { + // The callback must run on the IO event loop task thread, so post it + // here, re-using the pre-allocated handler memory. + // + this->storage_.post_to_event_loop(batt::make_custom_alloc_handler( + *p_mem, [this, p_mem, new_value](const StatusOr& /*ignored*/) { + // We no longer need the handler memory, so free now. + // + this->free_handler_memory(p_mem); + + this->waiting_[kKey] = false; + + if (!new_value.ok()) { + this->context_.update_error_status(new_value.status()); + return; + } + + this->poll(); + })); + })); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void IoRingLogDriver2::start_flush(slot_offset_type observed_commit_pos) +{ + slot_offset_type flush_upper_bound = this->unflushed_lower_bound_; + + for (usize repeat = 0; repeat < 2; ++repeat) { + //----- --- -- - - - - + + // Don't start a write if we are at the max concurrent writes limit. + // + if (this->writes_pending_ == this->options_.max_concurrent_writes) { + LLFS_VLOG(1) << "start_flush - at max writes pending"; + break; + } + + // Don't start a write if we have no data to flush. + // + const usize unflushed_size = slot_clamp_distance(flush_upper_bound, observed_commit_pos); + if (unflushed_size == 0) { + LLFS_VLOG(1) << "start_flush - unflushed_size == 0"; + break; + } + + // Don't start a write if there is already a pending write and we aren't at the threshold. + // + if (this->writes_pending_ != 0 && unflushed_size < this->options_.flush_delay_threshold) { + LLFS_VLOG(1) << "start_flush - no action taken: " << BATT_INSPECT(unflushed_size) + << BATT_INSPECT(this->writes_pending_); + break; + } + + // All conditions for writing have been met; calculate the aligned range and start flushing. + // + SlotRange slot_range{ + .lower_bound = flush_upper_bound, + .upper_bound = observed_commit_pos, + }; + + // Check for split range. + { + const usize physical_lower_bound = slot_range.lower_bound % this->context_.buffer_.size(); + const usize physical_upper_bound = slot_range.upper_bound % this->context_.buffer_.size(); + + if (physical_lower_bound > physical_upper_bound && physical_upper_bound != 0) { + const slot_offset_type new_upper_bound = + slot_range.lower_bound + (this->context_.buffer_.size() - physical_lower_bound); + + LLFS_VLOG(1) << "Clipping: " << slot_range.upper_bound << " -> " << new_upper_bound << ";" + << BATT_INSPECT(physical_lower_bound) << BATT_INSPECT(physical_upper_bound); + + slot_range.upper_bound = new_upper_bound; + } + } + + SlotRange aligned_range = this->get_aligned_range(slot_range); + + // If this flush would overlap with an ongoing one (at the last device page) then trim the + // aligned_range so it doesn't. + // + if (this->flush_tail_) { + if (slot_less_than(aligned_range.lower_bound, this->flush_tail_->upper_bound)) { + aligned_range.lower_bound = this->flush_tail_->upper_bound; + if (aligned_range.empty()) { + flush_upper_bound = this->flush_tail_->upper_bound; + continue; + } + } + } + + // Replace the current flush_tail_ slot range. + // + const SlotRange new_flush_tail = this->get_aligned_tail(aligned_range); + if (this->flush_tail_) { + BATT_CHECK_NE(new_flush_tail, *this->flush_tail_); + } + BATT_CHECK(!new_flush_tail.empty()); + this->flush_tail_.emplace(new_flush_tail); + + // Start writing! + // + this->start_flush_write(slot_range, aligned_range); + flush_upper_bound = this->unflushed_lower_bound_; + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline SlotRange IoRingLogDriver2::get_aligned_range( + const SlotRange& slot_range) const noexcept +{ + return SlotRange{ + .lower_bound = batt::round_down_bits(this->config_.data_alignment_log2, // + slot_range.lower_bound), + .upper_bound = batt::round_up_bits(this->config_.data_alignment_log2, // + slot_range.upper_bound), + }; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline SlotRange IoRingLogDriver2::get_aligned_tail( + const SlotRange& aligned_range) const noexcept +{ + return SlotRange{ + .lower_bound = aligned_range.upper_bound - this->data_page_size_, + .upper_bound = aligned_range.upper_bound, + }; +} + +//==#==========+=t=+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void IoRingLogDriver2::start_flush_write(const SlotRange& slot_range, + const SlotRange& aligned_range) noexcept +{ + LLFS_VLOG(1) << "start_flush_write(" << slot_range << ", " << aligned_range << ")"; + + this->unflushed_lower_bound_ = slot_max(this->unflushed_lower_bound_, slot_range.upper_bound); + + const i64 write_offset = + this->data_begin_ + (aligned_range.lower_bound % this->context_.buffer_.size()); + + ConstBuffer buffer = + resize_buffer(this->context_.buffer_.get(aligned_range.lower_bound), aligned_range.size()); + + BATT_CHECK_LE(write_offset + (i64)buffer.size(), this->data_end_); + + LLFS_VLOG(1) << " -- async_write_some(offset=" << write_offset << ".." + << write_offset + buffer.size() << ", size=" << buffer.size() << ")"; + + ++this->writes_pending_; + this->writes_max_ = std::max(this->writes_max_, this->writes_pending_); + + BATT_CHECK_LE(this->writes_pending_, this->options_.max_concurrent_writes); + + this->storage_.async_write_some(write_offset, buffer, + this->make_write_handler([this, slot_range, aligned_range]( + Self* this_, StatusOr result) { + --this_->writes_pending_; + this_->handle_flush_write(slot_range, aligned_range, result); + })); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void IoRingLogDriver2::handle_flush_write(const SlotRange& slot_range, + const SlotRange& aligned_range, + StatusOr result) +{ + LLFS_VLOG(1) << "handle_flush_result(result=" << result << ")" << BATT_INSPECT(slot_range); + + const usize bytes_written = result.ok() ? *result : 0; + + SlotRange aligned_tail = this->get_aligned_tail(aligned_range); + + SlotRange flushed_range{ + .lower_bound = slot_max(slot_range.lower_bound, aligned_range.lower_bound), + .upper_bound = slot_min(aligned_range.lower_bound + bytes_written, slot_range.upper_bound), + }; + + const bool is_tail = (this->flush_tail_ && *this->flush_tail_ == aligned_tail); + LLFS_DVLOG(1) << BATT_INSPECT(is_tail); + if (is_tail) { + this->flush_tail_ = None; + this->unflushed_lower_bound_ = flushed_range.upper_bound; + } + + LLFS_DVLOG(1) << BATT_INSPECT(flushed_range); + + if (!result.ok()) { + LLFS_VLOG(1) << "(handle_flush_write) error: " << result.status(); + this->handle_write_error(result.status()); + return; + } + + BATT_CHECK(!flushed_range.empty()); + + this->update_known_flush_pos(flushed_range); + + const auto observed_commit_pos = this->observe(CommitPos{}); + + if (!is_tail) { + SlotRange updated_range{ + .lower_bound = flushed_range.upper_bound, + .upper_bound = slot_min(aligned_range.upper_bound, observed_commit_pos), + }; + + if (!updated_range.empty()) { + this->start_flush_write(updated_range, this->get_aligned_range(updated_range)); + } + } + + this->poll(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::update_known_flush_pos(const SlotRange& flushed_range) noexcept +{ + // Insert the flushed_range into the min-heap. + // + this->flushed_ranges_.emplace_back(flushed_range); + std::push_heap(this->flushed_ranges_.begin(), this->flushed_ranges_.end(), SlotRangePriority{}); + + // Advance this->known_flush_pos_ as long as we see flushed ranges without gaps in between. + // + while (!this->flushed_ranges_.empty()) { + SlotRange& next_range = this->flushed_ranges_.front(); + + // Found a gap; we are done! + // + if (next_range.lower_bound != this->known_flush_pos_) { + LLFS_CHECK_SLOT_LT(this->known_flush_pos_, next_range.lower_bound); + break; + } + + this->known_flush_pos_ = next_range.upper_bound; + + // Pop the min range off the heap and keep going. + // + std::pop_heap(this->flushed_ranges_.begin(), this->flushed_ranges_.end(), SlotRangePriority{}); + this->flushed_ranges_.pop_back(); + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::start_control_block_update( + slot_offset_type observed_target_trim_pos) noexcept +{ + if (this->writing_control_block_ || this->trim_pos_.is_closed() || this->flush_pos_.is_closed()) { + return; + } + + BATT_CHECK_NOT_NULLPTR(this->control_block_) << "Forgot to call read_control_block()?"; + + const slot_offset_type effective_target_trim_pos = + slot_min(this->known_flush_pos_, observed_target_trim_pos); + const slot_offset_type observed_trim_pos = this->trim_pos_.get_value(); + const slot_offset_type observed_flush_pos = this->flush_pos_.get_value(); + + if (observed_trim_pos == effective_target_trim_pos && + observed_flush_pos == this->known_flush_pos_) { + return; + } + + LLFS_VLOG(1) << "start_control_block_update():" + << " trim=" << observed_trim_pos << "->" << observed_target_trim_pos + << " (effective=" << effective_target_trim_pos << ")" + << " flush=" << observed_flush_pos << "->" << this->known_flush_pos_; + + BATT_CHECK_EQ(observed_trim_pos, this->control_block_->trim_pos); + BATT_CHECK_EQ(observed_flush_pos, this->control_block_->flush_pos); + + const slot_offset_type latest_commit_pos = this->observe(CommitPos{}); + auto& next_commit_pos_slot = + this->control_block_->commit_points[this->control_block_->next_commit_i]; + + if (next_commit_pos_slot != latest_commit_pos) { + next_commit_pos_slot = latest_commit_pos; + + this->control_block_->next_commit_i = + (this->control_block_->next_commit_i + 1) % this->control_block_->commit_points.size(); + } + + LLFS_CHECK_SLOT_LE(effective_target_trim_pos, this->known_flush_pos_); + BATT_CHECK_LE(this->known_flush_pos_ - effective_target_trim_pos, this->config_.log_capacity); + + this->control_block_->trim_pos = effective_target_trim_pos; + this->control_block_->flush_pos = this->known_flush_pos_; + this->control_block_->generation = this->control_block_->generation + 1; + + this->writing_control_block_ = true; + + this->storage_.async_write_some_fixed( + this->config_.control_block_offset, this->control_block_buffer_, /*buf_index=*/0, + this->make_write_handler([](Self* this_, StatusOr result) { + this_->handle_control_block_update(result); + })); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::handle_control_block_update(StatusOr result) noexcept +{ + LLFS_VLOG(1) << "handle_control_block_update(" << result << ")"; + + BATT_CHECK(this->writing_control_block_); + + this->writing_control_block_ = false; + + if (!result.ok()) { + this->handle_write_error(result.status()); + return; + } + + if (BATT_CHECKED_CAST(usize, *result) != this->control_block_buffer_.size()) { + LLFS_LOG_ERROR() << "Failed to write entire log control block!"; + this->context_.update_error_status(batt::StatusCode::kInternal); + return; + } + + // We can now notify the outside world that the trim/flush pointers have been updated. + // + this->trim_pos_.set_value(this->control_block_->trim_pos); + this->flush_pos_.set_value(this->control_block_->flush_pos); + + this->poll(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::handle_write_error(Status status) noexcept +{ + this->trim_pos_.close(); + this->flush_pos_.close(); + this->context_.update_error_status(status); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +template +auto IoRingLogDriver2::make_write_handler(HandlerFn&& handler) +{ + HandlerMemory* const p_mem = this->alloc_handler_memory(); + + return batt::make_custom_alloc_handler( + *p_mem, [this, p_mem, handler = BATT_FORWARD(handler)](const StatusOr& result) { + this->free_handler_memory(p_mem); + handler(this, result); + }); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::initialize_handler_memory_pool() +{ + LLFS_VLOG(1) << "initialize_handler_memory_pool()"; + + BATT_CHECK_EQ(this->handler_memory_pool_, nullptr); + + usize pool_size = this->options_.max_concurrent_writes + 1 /* control block write */ + + 1 /* target_trim_pos_ wait */ + 1 /* commit_pos_ wait */; + + this->handler_memory_.reset(new HandlerMemoryStorage[pool_size]); + + for (usize i = 0; i < pool_size; ++i) { + HandlerMemoryStorage* p_storage = std::addressof(this->handler_memory_[i]); + auto pp_next = (HandlerMemoryStorage**)p_storage; + *pp_next = this->handler_memory_pool_; + this->handler_memory_pool_ = p_storage; + } +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +auto IoRingLogDriver2::alloc_handler_memory() noexcept -> HandlerMemory* +{ + BATT_CHECK_EQ(this->event_thread_id_, std::this_thread::get_id()); + + HandlerMemoryStorage* const p_storage = this->handler_memory_pool_; + auto pp_next = (HandlerMemoryStorage**)p_storage; + this->handler_memory_pool_ = *pp_next; + *pp_next = nullptr; + + return new (p_storage) HandlerMemory{}; +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +void IoRingLogDriver2::free_handler_memory(HandlerMemory* p_mem) noexcept +{ + BATT_CHECK(!p_mem->in_use()); + BATT_CHECK_EQ(this->event_thread_id_, std::this_thread::get_id()); + + p_mem->~HandlerMemory(); + + auto p_storage = (HandlerMemoryStorage*)p_mem; + auto pp_next = (HandlerMemoryStorage**)p_storage; + *pp_next = this->handler_memory_pool_; + this->handler_memory_pool_ = p_storage; +} + +} //namespace llfs + +#endif // LLFS_IORING_LOG_DEVICE2_IPP diff --git a/src/llfs/ioring_log_device2.test.cpp b/src/llfs/ioring_log_device2.test.cpp new file mode 100644 index 0000000..472cd35 --- /dev/null +++ b/src/llfs/ioring_log_device2.test.cpp @@ -0,0 +1,594 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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 +#include + +#include +#include +#include +#include +#include + +namespace { + +using namespace llfs::constants; +using namespace llfs::int_types; + +using llfs::Interval; +using llfs::None; +using llfs::Status; +using llfs::StatusOr; + +// Test Plan: +// - Use StorageSimulation to create an IoRingLogDevice2 with SimulatedLogDeviceStorage +// - Choose random operations: append, trim +// - Crash the simulation at a predetermined number of steps; when log is recovered, verify that +// the confirmed trim and flush never go backwards, and that all confirmed flushed data can be +// read. +// +class IoringLogDevice2SimTest : public ::testing::Test +{ + public: + static constexpr usize kNumSeeds = 250 * 1000; + static constexpr u64 kTestLogSize = 16 * kKiB; + static constexpr i64 kTestLogBegin = 7 * 4 * kKiB; + static constexpr i64 kTestLogEnd = + kTestLogBegin + 4 * kKiB /*control block (aligned)*/ + kTestLogSize; + static constexpr usize kTestMinSlotSize = 50; + static constexpr usize kTestMaxSlotSize = 1500; + static constexpr usize kNumSlots = 50; + static constexpr usize kDevicePageSize = 512; + static constexpr usize kDataAlignment = 4096; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief A single simulation run. + * + * It is safe to run multiple Scenarios concurrently on separate threads. + */ + struct Scenario { + const std::string kTestStorageName = "test_storage"; + + const llfs::IoRingLogConfig2 log_config{ + .control_block_offset = kTestLogBegin, + .log_capacity = kTestLogSize, + .device_page_size_log2 = batt::log2_ceil(kDevicePageSize), + .data_alignment_log2 = batt::log2_ceil(kDataAlignment), + }; + + /** \brief The RNG seed for this scenario. + */ + usize seed; + + /** \brief The entropy source for the simulation. + */ + std::mt19937 rng; + + /** \brief The simulation context. + */ + llfs::StorageSimulation sim; + + /** \brief LogDevice object-under-test; created in recover_log_device. initialize_log_storage + * MUST be called prior to recovering the log device for the first time. + */ + std::unique_ptr log_device; + + /** \brief The LogDevice::Writer interface for `this->log_device`. + */ + llfs::LogDevice::Writer* log_writer = nullptr; + + /** \brief The slot offset ranges of all appended slots that haven't been trimmed yet; these are + * not necessarily flushed yet. + */ + std::deque appended_slots; + + /** \brief The slot offset ranges of all slots have been speculatively trimmed; the trim + * operation isn't necessarily durable yet (hence 'maybe'). + */ + std::deque maybe_trimmed_slots; + + /** \brief The slot offset ranges of appended_slots that aren't yet known to be flushed/durable. + */ + std::deque maybe_flushed_slots; + + /** \brief The latest observed durable trim position. + */ + llfs::slot_offset_type observed_trim = 0; + + /** \brief The latest observed durable flush position. + */ + llfs::slot_offset_type observed_flush = 0; + + /** \brief The total number of bytes that have been appended to the log since the beginning of + * the scenario. + */ + usize total_bytes_appended = 0; + + /** \brief Distribution used to select appended slot payload sizes. + */ + std::uniform_int_distribution pick_slot_size{kTestMinSlotSize, kTestMaxSlotSize}; + + /** \brief The closest slot boundary (without going over) to the observed_flush position. + */ + llfs::slot_offset_type observed_slot_flush = 0; + + /** \brief The number of partially flushed slots discovered after simulated crash/recovery. + */ + usize partial_slot_flush_count = 0; + + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief Constructs a new Scenario with the given RNG seed. + */ + explicit Scenario(usize seed) noexcept + : seed{seed} + , rng{this->seed} + , sim{batt::StateMachineEntropySource{ + /*entropy_fn=*/[this](usize min_value, usize max_value) -> usize { + std::uniform_int_distribution pick_value{min_value, max_value}; + return pick_value(this->rng); + }}} + { + } + + /** \brief Writes initial data (control block) to the simulated media. + */ + void initialize_log_storage() noexcept + { + llfs::SimulatedLogDeviceStorage storage = sim.get_log_device_storage( + kTestStorageName, /*capacity=*/kTestLogSize, Interval{kTestLogBegin, kTestLogEnd}); + + ASSERT_FALSE(storage.is_initialized()); + + Status init_status = + llfs::initialize_log_device2(*storage.get_raw_block_file(), this->log_config); + + ASSERT_TRUE(init_status.ok()) << BATT_INSPECT(init_status); + + storage.set_initialized(true); + } + + /** \brief Opens the log from simulated media and creates the LogDevice object + * (`this->log_device`). + * + * Also initializes `this->log_writer`. + */ + void recover_log_device() noexcept + { + llfs::SimulatedLogDeviceStorage storage = + sim.get_log_device_storage(kTestStorageName, None, None); + + ASSERT_TRUE(storage.is_initialized()); + + auto p_device = + std::make_unique>( + this->log_config, llfs::LogDeviceRuntimeOptions::with_default_values(), + std::move(storage)); + + BATT_CHECK_OK(p_device->open()); + + this->log_device = std::move(p_device); + this->log_writer = std::addressof(this->log_device->writer()); + } + + /** \brief Shuts down the log device, deletes the object, and resets all pointers. + */ + void shutdown_log_device() noexcept + { + if (this->log_device) { + this->log_device->halt(); + this->log_device->join(); + this->log_device.reset(); + this->log_writer = nullptr; + } + } + + /** \brief Writes a single random-sized slot to the log. + * \return false if an error was encountered; true otherwise + */ + bool append_one_slot() noexcept + { + const usize payload_size = pick_slot_size(rng); + const usize header_size = llfs::packed_sizeof_varint(payload_size); + const usize slot_size = header_size + payload_size; + + if (!this->trim_to_reserve(slot_size)) { + return false; + } + + llfs::slot_offset_type slot_offset = this->log_writer->slot_offset(); + StatusOr buffer = this->log_writer->prepare(slot_size); + BATT_CHECK_OK(buffer); + + *buffer = batt::get_or_panic(llfs::pack_varint_to(*buffer, payload_size)); + std::memset(buffer->data(), 'a' + (slot_offset % 26), buffer->size()); + + StatusOr end_offset = this->log_writer->commit(slot_size); + BATT_CHECK_OK(end_offset); + BATT_CHECK_EQ(slot_offset + slot_size, *end_offset); + + this->appended_slots.push_back(llfs::SlotRange{ + .lower_bound = slot_offset, + .upper_bound = *end_offset, + }); + this->maybe_flushed_slots.push_back(this->appended_slots.back()); + this->total_bytes_appended += slot_size; + + return true; + } + + /** \brief Updates observed_trim, observed_flush, and maybe_flushed_slots based on re-reading + * the current durable range of the log. + */ + void update_trim_flush_pos() + { + llfs::SlotRange slot_range = this->log_device->slot_range(llfs::LogReadMode::kDurable); + + ASSERT_FALSE(llfs::slot_less_than(slot_range.lower_bound, this->observed_trim)) + << BATT_INSPECT(slot_range) << BATT_INSPECT(this->observed_trim); + + ASSERT_FALSE(llfs::slot_less_than(slot_range.upper_bound, this->observed_flush)) + << BATT_INSPECT(slot_range) << BATT_INSPECT(this->observed_flush); + + this->observed_trim = slot_range.lower_bound; + this->observed_flush = slot_range.upper_bound; + + while (!this->maybe_flushed_slots.empty() && + !llfs::slot_less_than(this->observed_flush, + this->maybe_flushed_slots.front().upper_bound)) { + this->observed_slot_flush = this->maybe_flushed_slots.front().upper_bound; + this->maybe_flushed_slots.pop_front(); + } + } + + /** \brief Attempts to reserve at least n_bytes for appending in the log, by trimming old slots + * if necessary. + * + * If this function returns true, all is fine; otherwise, it indicates a fatal error that should + * signal the end of the append phase of this scenario. + */ + bool trim_to_reserve(usize n_bytes) + { + u64 observed_space = this->log_writer->space(); + while (observed_space < n_bytes && !this->appended_slots.empty()) { + const llfs::SlotRange trimmed_slot = this->appended_slots.front(); + + this->appended_slots.pop_front(); + this->maybe_trimmed_slots.push_back(trimmed_slot); + + Status trim_status = this->log_device->trim(trimmed_slot.upper_bound); + if (!trim_status.ok()) { + break; + } + + Status await_status = this->log_writer->await(llfs::BytesAvailable{ + .size = trimmed_slot.size() + observed_space, + }); + if (!await_status.ok()) { + break; + } + + llfs::SlotRange slot_range = this->log_device->slot_range(llfs::LogReadMode::kDurable); + while (!this->maybe_trimmed_slots.empty() && + llfs::slot_less_than(this->maybe_trimmed_slots.front().lower_bound, + slot_range.lower_bound)) { + LLFS_CHECK_SLOT_LE(this->maybe_trimmed_slots.front().upper_bound, slot_range.lower_bound); + observed_space += this->maybe_trimmed_slots.front().size(); + this->maybe_trimmed_slots.pop_front(); + } + } + + // Update the observed trim/flush pointers. + // + this->update_trim_flush_pos(); + + // Return true iff we succeeded in trimming enough space. + // + return (observed_space >= n_bytes); + } + + /** \brief Verify that the recovered durable slot range doesn't violate our assumptions: + * + * - observed trim pos should never move backwards + * - slot range should always be at most the total size of the log + * - slot range should not have negative size (i.e., lower_bound <= upper_bound) + * - the observed flush pos should either be the same as the last one observed, or it should be + * not less than the highest observed slot upper bound for all appends. + */ + void verify_recovered_slot_range(const llfs::SlotRange& slot_range) + { + ASSERT_TRUE(!llfs::slot_less_than(slot_range.upper_bound, slot_range.lower_bound)) + << BATT_INSPECT(slot_range); + + ASSERT_LE(slot_range.size(), kTestLogSize) << BATT_INSPECT(slot_range); + + ASSERT_FALSE(llfs::slot_less_than(slot_range.lower_bound, observed_trim)) + << BATT_INSPECT(slot_range) << BATT_INSPECT(observed_trim) << BATT_INSPECT(seed); + + ASSERT_FALSE(llfs::slot_less_than(slot_range.upper_bound, observed_slot_flush)) + << BATT_INSPECT(slot_range) << BATT_INSPECT(observed_slot_flush); + + if (llfs::slot_less_than(slot_range.upper_bound, observed_flush)) { + this->partial_slot_flush_count += 1; + } + } + + /** \brief Moves any ranges in maybe_trimmed_slots to appended_slots, based on the recovered + * durable slot range. + * + * maybe_trimmed_slots contains ranges for which we aren't sure whether they were actually + * trimmed (via a durable update of the control block). By looking at the actual recovered slot + * range, we know for sure which ones still belong on this list. + * + * By the time this function returns, maybe_trimmed_slots will be empty, since we've either + * removed each slot range, or moved it from maybed_trimmed_slots to appended_slots. + */ + void resolve_maybe_trimmed_slots(const llfs::SlotRange& slot_range) + { + // Remove any actually trimmed slots from the maybe_trimmed list. + // + while (!this->maybe_trimmed_slots.empty() && + llfs::slot_less_than(this->maybe_trimmed_slots.front().lower_bound, + slot_range.lower_bound)) { + LLFS_CHECK_SLOT_LE(this->maybe_trimmed_slots.front().upper_bound, slot_range.lower_bound); + this->maybe_trimmed_slots.pop_front(); + } + + // Move any remaining maybe trimmed slots to the front of the appended list. + // + while (!this->maybe_trimmed_slots.empty()) { + this->appended_slots.push_front(this->maybe_trimmed_slots.back()); + this->maybe_trimmed_slots.pop_back(); + } + } + + /** \brief Verifies that the recovered log has the expected slot data, based on a prior append + * phase. + * + * \param slot_range the durable slot range of the recovered LogDevice + */ + void verify_recovered_slots(const llfs::SlotRange& slot_range) + { + std::unique_ptr log_reader = + this->log_device->new_reader(/*slot_lower_bound=*/None, llfs::LogReadMode::kDurable); + + llfs::SlotReader slot_reader{*log_reader}; + + StatusOr n_parsed = slot_reader.run( + batt::WaitForResource::kFalse, [&](const llfs::SlotParse& slot) -> Status { + BATT_CHECK_EQ(slot.offset, this->appended_slots.front()); + this->appended_slots.pop_front(); + + // Verify the data. + // + const char expected_ch = 'a' + (slot.offset.lower_bound % 26); + for (char actual_ch : slot.body) { + EXPECT_EQ(expected_ch, actual_ch); + if (actual_ch != expected_ch) { + LLFS_LOG_INFO() << BATT_INSPECT(slot.offset) << BATT_INSPECT(slot_range) + << BATT_INSPECT(this->observed_trim) + << BATT_INSPECT(this->observed_flush); + break; + } + } + + return batt::OkStatus(); + }); + + ASSERT_TRUE(n_parsed.ok()) << BATT_INSPECT(n_parsed); + + // If there are any unmatched slots, then assert they are in the unflushed range. + // + if (!this->appended_slots.empty()) { + ASSERT_GE(this->appended_slots.front().lower_bound, slot_range.upper_bound); + } + } + + /** \brief The simulation entry point. + * + * Consists of two phases: + * 1. append phase - append slots until we reach kNumSlots or an injected error stops us from + * making progress + * 2. verify phase - recover the log and check flush/trim pos and slot data + * + * If we run out of space in (1), we trim slots one at a time until the desired amount of space + * has been freed up. + */ + void run() + { + ASSERT_NO_FATAL_FAILURE(this->initialize_log_storage()); + + this->sim.run_main_task([this] { + { + ASSERT_NO_FATAL_FAILURE(this->recover_log_device()); + auto on_scope_exit = batt::finally([&] { + this->shutdown_log_device(); + }); + + this->sim.set_inject_failures_mode((seed % 2) == 1); + + for (usize i = 0; i < kNumSlots; ++i) { + LLFS_VLOG(1) << "Writing slot " << i << " of " << kNumSlots << ";" + << BATT_INSPECT(this->total_bytes_appended) << BATT_INSPECT(seed); + + if (!this->append_one_slot()) { + break; + } + } + } + + this->sim.crash_and_recover(); + this->sim.set_inject_failures_mode(false); + + { + ASSERT_NO_FATAL_FAILURE(this->recover_log_device()); + auto on_scope_exit = batt::finally([&] { + this->shutdown_log_device(); + }); + + llfs::SlotRange slot_range = this->log_device->slot_range(llfs::LogReadMode::kDurable); + + ASSERT_NO_FATAL_FAILURE(this->verify_recovered_slot_range(slot_range)); + ASSERT_NO_FATAL_FAILURE(this->resolve_maybe_trimmed_slots(slot_range)); + ASSERT_NO_FATAL_FAILURE(this->verify_recovered_slots(slot_range)); + } + }); + } + + }; // struct Scenario + +}; // class IoringLogDevice2SimTest + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- + +TEST_F(IoringLogDevice2SimTest, Simulation) +{ + const usize kNumThreads = std::thread::hardware_concurrency(); + const usize kUpdateInterval = kNumSeeds / 20; + + LLFS_LOG_INFO() << BATT_INSPECT(kNumSeeds); + + std::vector threads; + usize start_seed = 0; + usize seeds_remaining = kNumSeeds; + for (usize i = 0; i < kNumThreads; ++i) { + usize n_seeds = seeds_remaining / (kNumThreads - i); + usize end_seed = start_seed + n_seeds; + threads.emplace_back([&, start_seed, end_seed] { + for (usize seed = start_seed; seed < end_seed; seed += 1) { + LLFS_LOG_INFO_EVERY_N(kUpdateInterval) + << "progress=" << (seed - start_seed) * 100 / (end_seed - start_seed) << "%"; + + Scenario scenario{seed}; + ASSERT_NO_FATAL_FAILURE(scenario.run()); + } + }); + start_seed += n_seeds; + seeds_remaining -= n_seeds; + } + + for (std::thread& t : threads) { + t.join(); + } + + //EXPECT_GT(partial_slot_flush_count, 0); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +TEST(IoringLogDevice2Test, Benchmark) +{ + llfs::run_log_device_benchmark([&](usize log_size, bool create, auto&& workload_fn) { + //+++++++++++-+-+--+----- --- -- - - - - + // Set configuration and options. + // + llfs::IoRingLogConfig2 config{ + .control_block_offset = 0, + .log_capacity = log_size, + .device_page_size_log2 = 9, + .data_alignment_log2 = 12, + }; + + llfs::LogDeviceRuntimeOptions options{ + .name = "test log", + .flush_delay_threshold = 2 * kMiB, + .max_concurrent_writes = 64, + }; + + const char* file_name = // + std::getenv("LLFS_LOG_DEVICE_FILE"); + + if (!file_name) { + LLFS_LOG_INFO() << "LLFS_LOG_DEVICE_FILE not specified; skipping benchmark test"; + return; + } + + std::cout << "LLFS_LOG_DEVICE_FILE=" << batt::c_str_literal(file_name) << std::endl; + + LLFS_LOG_INFO() << BATT_INSPECT(file_name); + + //+++++++++++-+-+--+----- --- -- - - - - + // Erase any existing file. + // + if (create) { + { + std::filesystem::path file_path{file_name}; + std::filesystem::remove_all(file_path); + ASSERT_FALSE(std::filesystem::exists(file_path)); + } + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Create a new log file and size it to the configured capacity. + // + llfs::StatusOr status_or_fd = [&] { + if (create) { + return llfs::create_file_read_write(file_name, llfs::OpenForAppend{false}); + } else { + return llfs::open_file_read_write(file_name, llfs::OpenForAppend{false}, + llfs::OpenRawIO{true}); + } + }(); + + ASSERT_TRUE(status_or_fd.ok()) << BATT_INSPECT(status_or_fd); + + const int fd = *status_or_fd; + + if (create) { + llfs::Status enable_raw_status = llfs::enable_raw_io_fd(fd, true); + + ASSERT_TRUE(enable_raw_status.ok()) << BATT_INSPECT(enable_raw_status); + + llfs::Status truncate_status = + llfs::truncate_fd(fd, /*size=*/config.control_block_size() + config.log_capacity); + + ASSERT_TRUE(truncate_status.ok()); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Initialize the IoRing and IoRing::File inside a storage object wrapper. + // + llfs::StatusOr status_or_storage = + llfs::DefaultIoRingLogDeviceStorage::make_new(llfs::MaxQueueDepth{256}, fd); + + ASSERT_TRUE(status_or_storage.ok()) << BATT_INSPECT(status_or_storage.status()); + + llfs::DefaultIoRingLogDeviceStorage& storage = *status_or_storage; + + if (create) { + llfs::DefaultIoRingLogDeviceStorage::RawBlockFileImpl file{storage}; + + //+++++++++++-+-+--+----- --- -- - - - - + // Write the initial contents of the file. + // + llfs::Status init_status = llfs::initialize_log_device2(file, config); + + ASSERT_TRUE(init_status.ok()) << BATT_INSPECT(init_status); + } + + //+++++++++++-+-+--+----- --- -- - - - - + // Create LogDevice object and open. + // + llfs::IoRingLogDevice2 log_device{config, options, std::move(storage)}; + batt::Status open_status = log_device.driver().open(); + + ASSERT_TRUE(open_status.ok()) << BATT_INSPECT(open_status); + + //+++++++++++-+-+--+----- --- -- - - - - + // Run the passed workload. + // + workload_fn(log_device); + }); +} + +} // namespace diff --git a/src/llfs/log_device_config2.cpp b/src/llfs/log_device_config2.cpp new file mode 100644 index 0000000..33f4188 --- /dev/null +++ b/src/llfs/log_device_config2.cpp @@ -0,0 +1,103 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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 +// + +#ifndef LLFS_DISABLE_IO_URING + +#include +#include +#include + +#include +#include + +#include + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +BATT_PRINT_OBJECT_IMPL(PackedLogDeviceConfig2, // + (control_block_offset) // + (logical_size) // + (device_page_size_log2) // + (data_alignment_log2) // +) + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status configure_storage_object(StorageFileBuilder::Transaction& txn, + FileOffsetPtr p_config, + const LogDeviceConfigOptions2& options) +{ + const i64 logical_size = round_up_to_page_size_multiple(options.log_size); + + const u16 device_page_size_log2 = + options.device_page_size_log2.value_or(LogDeviceConfigOptions2::kDefaultDevicePageSizeLog2); + + const u16 data_alignment_log2 = + options.data_alignment_log2.value_or(LogDeviceConfigOptions2::kDefaultDataAlignmentLog2); + + BATT_CHECK_GE(data_alignment_log2, device_page_size_log2); + + const i64 data_page_size = i64{1} << data_alignment_log2; + + const i64 physical_size = /*control block*/ data_page_size + /*data pages*/ logical_size; + + Interval blocks_offset = txn.reserve_aligned(/*bits=*/data_alignment_log2, physical_size); + + BATT_CHECK_EQ(blocks_offset.size(), physical_size) + << BATT_INSPECT(logical_size) << BATT_INSPECT(options.log_size); + + p_config->control_block_offset = + p_config.relative_from_absolute_offset(blocks_offset.lower_bound); + p_config->logical_size = logical_size; + p_config->device_page_size_log2 = device_page_size_log2; + p_config->data_alignment_log2 = data_alignment_log2; + p_config->uuid = options.uuid.value_or(boost::uuids::random_generator{}()); + + BATT_CHECK_EQ(BATT_CHECKED_CAST(i64, blocks_offset.lower_bound + physical_size), + blocks_offset.upper_bound); + + // Initialize the log page headers before flushing config slot. + // + txn.require_pre_flush_action([config = IoRingLogConfig2::from_packed(p_config), + blocks_offset](RawBlockFile& file) -> Status // + { + Status truncate_status = + file.truncate_at_least(blocks_offset.upper_bound); + + BATT_REQUIRE_OK(truncate_status); + + return initialize_log_device2(file, config); + }); + + return OkStatus(); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr> recover_storage_object( + const batt::SharedPtr& /*storage_context*/, const std::string& file_name, + const FileOffsetPtr& p_config, LogDeviceRuntimeOptions options) +{ + const int flags = O_DIRECT | O_SYNC | O_RDWR; + + int fd = batt::syscall_retry([&] { + return ::open(file_name.c_str(), flags); + }); + BATT_REQUIRE_OK(batt::status_from_retval(fd)); + + return std::make_unique(fd, p_config, options); +} + +} // namespace llfs + +#endif // LLFS_DISABLE_IO_URING diff --git a/src/llfs/log_device_config2.hpp b/src/llfs/log_device_config2.hpp new file mode 100644 index 0000000..689b762 --- /dev/null +++ b/src/llfs/log_device_config2.hpp @@ -0,0 +1,132 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_PACKED_LOG_DEVICE_CONFIG2_HPP +#define LLFS_PACKED_LOG_DEVICE_CONFIG2_HPP + +#include +// + +#ifndef LLFS_DISABLE_IO_URING + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include +#include + +namespace llfs { + +struct LogDeviceConfigOptions2; +struct PackedLogDeviceConfig2; + +//+++++++++++-+-+--+----- --- -- - - - - + +Status configure_storage_object(StorageFileBuilder::Transaction&, + FileOffsetPtr p_config, + const LogDeviceConfigOptions2& options); + +StatusOr> recover_storage_object( + const batt::SharedPtr& storage_context, const std::string& file_name, + const FileOffsetPtr& p_config, LogDeviceRuntimeOptions options); + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +struct LogDeviceConfigOptions2 { + using PackedConfigType = PackedLogDeviceConfig2; + + static constexpr u16 kDefaultDevicePageSizeLog2 = 9 /*=log2(512)*/; + static constexpr u16 kDefaultDataAlignmentLog2 = 12 /*=log2(4096)*/; + + //+++++++++++-+-+--+----- --- -- - - - - + + // The unique identifier for the log; if None, a random UUID will be generated. + // + Optional uuid; + + // The capacity in bytes of the log. + // + usize log_size; + + Optional device_page_size_log2; + Optional data_alignment_log2; +}; + +inline bool operator==(const LogDeviceConfigOptions2& l, const LogDeviceConfigOptions2& r) +{ + return l.uuid == r.uuid // + && l.log_size == r.log_size // + && l.device_page_size_log2 == r.device_page_size_log2 // + && l.data_alignment_log2 == r.data_alignment_log2; +} + +inline bool operator!=(const LogDeviceConfigOptions2& l, const LogDeviceConfigOptions2& r) +{ + return !(l == r); +} + +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +// +struct PackedLogDeviceConfig2 : PackedConfigSlotHeader { + static constexpr usize kSize = PackedConfigSlot::kSize; + + // The offset of flush block 0 relative to this structure. + // + little_i64 control_block_offset; + + // The capacity of the log. + // + little_u64 logical_size; + + // The log2 of the device page size where this log is stored. + // + little_u16 device_page_size_log2; + + // The log2 of the data alignment for the log. + // + little_u16 data_alignment_log2; + + // Reserved for future use (set to 0 for now). + // + u8 pad1_[24]; + + //+++++++++++-+-+--+----- --- -- - - - - +}; + +BATT_STATIC_ASSERT_EQ(sizeof(PackedLogDeviceConfig2), PackedLogDeviceConfig2::kSize); + +std::ostream& operator<<(std::ostream& out, const PackedLogDeviceConfig2& t); + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - + +template <> +struct PackedConfigTagFor { + static constexpr u32 value = PackedConfigSlot::Tag::kLogDevice2; +}; + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - + +std::ostream& operator<<(std::ostream& out, const PackedLogDeviceConfig2& t); + +} // namespace llfs + +#endif // LLFS_DISABLE_IO_URING + +#endif // LLFS_PACKED_LOG_DEVICE_CONFIG_HPP diff --git a/src/llfs/log_device_runtime_options.hpp b/src/llfs/log_device_runtime_options.hpp new file mode 100644 index 0000000..daec2f3 --- /dev/null +++ b/src/llfs/log_device_runtime_options.hpp @@ -0,0 +1,98 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_LOG_DEVICE_RUNTIME_OPTIONS_HPP +#define LLFS_LOG_DEVICE_RUNTIME_OPTIONS_HPP + +#include +// + +#include +#include + +#include +#include + +#include +#include +#include + +namespace llfs { + +struct LogDeviceRuntimeOptions { + using Self = LogDeviceRuntimeOptions; + + static constexpr usize kDefaultFlushDelayThreshold = 1 * kMiB; + static constexpr usize kDefaultMaxConcurrentWrites = 64; + + //+++++++++++-+-+--+----- --- -- - - - - + + static LogDeviceRuntimeOptions with_default_values() + { + return LogDeviceRuntimeOptions{}; + } + + static i32 next_id() + { + static std::atomic n{1}; + return n.fetch_add(1); + } + + //+++++++++++-+-+--+----- --- -- - - - - + + // The debug name of this log. + // + std::string name = batt::to_string("(anonymous log ", Self::next_id(), ")"); + + usize flush_delay_threshold = kDefaultFlushDelayThreshold; + + usize max_concurrent_writes = kDefaultMaxConcurrentWrites; + + //+++++++++++-+-+--+----- --- -- - - - - + + Self& set_name(std::string_view name) + { + this->name = name; + return *this; + } + + usize queue_depth() const + { + return this->max_concurrent_writes; + } + + Self& set_queue_depth(usize n) + { + this->max_concurrent_writes = n; + return *this; + } + + /** \brief Sets queue depth to the smallest power of 2 that is not greater than `max_n` *and* not + * greater than the current value of queue depth. + * + * `max_n` must be at least 2. + */ + Self& limit_queue_depth(usize max_n) + { + BATT_CHECK_GT(max_n, 1); + while (this->queue_depth() > max_n) { + this->set_queue_depth(this->queue_depth() / 2); + } + return *this; + } + + usize queue_depth_mask() const + { + return this->queue_depth() - 1; + } +}; + +} //namespace llfs + +#endif // LLFS_LOG_DEVICE_RUNTIME_OPTIONS_HPP diff --git a/src/llfs/nested_log_device_config.hpp b/src/llfs/nested_log_device_config.hpp new file mode 100644 index 0000000..13f5ce1 --- /dev/null +++ b/src/llfs/nested_log_device_config.hpp @@ -0,0 +1,129 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_NESTED_LOG_DEVICE_CONFIG_HPP +#define LLFS_NESTED_LOG_DEVICE_CONFIG_HPP + +#include +// + +#include +#include +#include +#include +#include + +#include + +namespace llfs { + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// These types are provided for the convenience of more complex configs that nest one or more +// LogDevice configs. +// + +//+++++++++++-+-+--+----- --- -- - - - - +// +struct CreateNewLogDevice { + LogDeviceConfigOptions options; +}; + +inline bool operator==(const CreateNewLogDevice& l, const CreateNewLogDevice& r) +{ + return l.options == r.options; +} + +inline bool operator!=(const CreateNewLogDevice& l, const CreateNewLogDevice& r) +{ + return !(l == r); +} + +//+++++++++++-+-+--+----- --- -- - - - - +// +struct CreateNewLogDeviceWithDefaultSize { + Optional uuid; + Optional pages_per_block_log2; +}; + +inline bool operator==(const CreateNewLogDeviceWithDefaultSize& l, + const CreateNewLogDeviceWithDefaultSize& r) +{ + return l.uuid == r.uuid // + && l.pages_per_block_log2 == r.pages_per_block_log2; +} + +inline bool operator!=(const CreateNewLogDeviceWithDefaultSize& l, + const CreateNewLogDeviceWithDefaultSize& r) +{ + return !(l == r); +} + +//+++++++++++-+-+--+----- --- -- - - - - +// +struct LinkToExistingLogDevice { + boost::uuids::uuid uuid; +}; + +inline bool operator==(const LinkToExistingLogDevice& l, const LinkToExistingLogDevice& r) +{ + return l.uuid == r.uuid; +} + +inline bool operator!=(const LinkToExistingLogDevice& l, const LinkToExistingLogDevice& r) +{ + return !(l == r); +} + +//+++++++++++-+-+--+----- --- -- - - - - +// +struct CreateNewLogDevice2 { + LogDeviceConfigOptions2 options; +}; + +inline bool operator==(const CreateNewLogDevice2& l, const CreateNewLogDevice2& r) +{ + return l.options == r.options; +} + +inline bool operator!=(const CreateNewLogDevice2& l, const CreateNewLogDevice2& r) +{ + return !(l == r); +} + +//+++++++++++-+-+--+----- --- -- - - - - +// +struct CreateNewLogDevice2WithDefaultSize { + Optional uuid; + Optional device_page_size_log2; + Optional data_alignment_log2; +}; + +inline bool operator==(const CreateNewLogDevice2WithDefaultSize& l, + const CreateNewLogDevice2WithDefaultSize& r) +{ + return l.uuid == r.uuid // + && l.device_page_size_log2 == r.device_page_size_log2 // + && l.data_alignment_log2 == r.data_alignment_log2; +} + +inline bool operator!=(const CreateNewLogDevice2WithDefaultSize& l, + const CreateNewLogDevice2WithDefaultSize& r) +{ + return !(l == r); +} + +//+++++++++++-+-+--+----- --- -- - - - - + +using NestedLogDeviceConfig = + std::variant; + +} //namespace llfs + +#endif // LLFS_NESTED_LOG_DEVICE_CONFIG_HPP diff --git a/src/llfs/packed_log_control_block2.hpp b/src/llfs/packed_log_control_block2.hpp new file mode 100644 index 0000000..7d7b721 --- /dev/null +++ b/src/llfs/packed_log_control_block2.hpp @@ -0,0 +1,81 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_PACKED_LOG_CONTROL_BLOCK2_HPP +#define LLFS_PACKED_LOG_CONTROL_BLOCK2_HPP + +#include +// +#include +#include + +#include + +#include + +namespace llfs { + +struct PackedLogControlBlock2 { + static constexpr u64 kMagic = 0x128095f84cfba8b0ull; + + /** \brief Must always be set to PackedLogControlBlock2::kMagic. + */ + big_u64 magic; + + /** \brief The total capacity in bytes of the log. + */ + little_i64 data_size; + + /** \brief The current trim (logical) offset in bytes from the beginning of the log. + */ + PackedSlotOffset trim_pos; + + /** \brief The current flush (logical) offset in bytes from the beginning of the log. + */ + PackedSlotOffset flush_pos; + + /** \brief The number of times this control block has been updated. + */ + little_u64 generation; + + /** \brief The size in bytes of the control block in storage. The data portion of the log always + * starts at the offset of this structure plus this->control_block_size. + */ + little_u32 control_block_size; + + /** \brief The size of this structure. This may be smaller than this->control_block_size; the + * difference is padding so that the data region will be correctly aligned (see + * this->data_alignment_log2). + */ + little_u32 control_header_size; + + /** \brief The size of the storage media page, log2. + */ + little_i16 device_page_size_log2; + + /** \brief The number of bits to which reads/writes within the log data region must be aligned. + */ + little_i16 data_alignment_log2; + + /** \brief The index of the next element of this->commit_points to be overwritten. + */ + little_u32 next_commit_i; + + /** \brief A rolling history of known commit points; this is updated each time the control block + * is updated. It is used to speed up log recovery, especially in the case where the log device + * was cleanly shut down. + */ + std::array commit_points; +}; + +BATT_STATIC_ASSERT_EQ(sizeof(PackedLogControlBlock2), 512); + +} //namespace llfs + +#endif // LLFS_PACKED_LOG_CONTROL_BLOCK2_HPP diff --git a/src/llfs/packed_slot_offset.hpp b/src/llfs/packed_slot_offset.hpp new file mode 100644 index 0000000..007d52b --- /dev/null +++ b/src/llfs/packed_slot_offset.hpp @@ -0,0 +1,21 @@ +//#=##=##=#==#=#==#===#+==#+==========+==+=+=+=+=+=++=+++=+++++=-++++=-+++++++++++ +// +// 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_PACKED_SLOT_OFFSET_HPP +#define LLFS_PACKED_SLOT_OFFSET_HPP + +#include + +namespace llfs { + +using PackedSlotOffset = little_u64; + +} //namespace llfs + +#endif // LLFS_PACKED_SLOT_OFFSET_HPP From 9a067c2bee80887e9968c3fbd1539b26fb5e328d Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Tue, 2 Jul 2024 17:24:06 -0400 Subject: [PATCH 02/11] Upgrade to batteries/0.53.0. --- conanfile.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conanfile.py b/conanfile.py index 0b6c173..48a6060 100644 --- a/conanfile.py +++ b/conanfile.py @@ -69,7 +69,7 @@ def configure(self): def requirements(self): - self.requires("batteries/0.52.4", **VISIBLE) + self.requires("batteries/0.53.0", **VISIBLE) self.requires("boost/1.83.0", **VISIBLE) self.requires("cli11/2.3.2", **VISIBLE) self.requires("glog/0.6.0", **VISIBLE) From 79a57ac33c21eff030972abb05b98f0efc584e17 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Tue, 2 Jul 2024 17:25:05 -0400 Subject: [PATCH 03/11] Refactor IoRingLogDevice benchmark test into cpp/hpp. --- src/llfs/ioring_log_device.test.cpp | 106 ++++------------------------ 1 file changed, 14 insertions(+), 92 deletions(-) diff --git a/src/llfs/ioring_log_device.test.cpp b/src/llfs/ioring_log_device.test.cpp index c4c936c..9e3caa2 100644 --- a/src/llfs/ioring_log_device.test.cpp +++ b/src/llfs/ioring_log_device.test.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -172,6 +173,8 @@ TEST(IoringLogDeviceTest, StorageFile) */ TEST(IoringLogDeviceTest, Benchmark) { + using llfs::read_test_var; + const char* file_name = // std::getenv("LLFS_LOG_DEVICE_FILE"); @@ -182,34 +185,18 @@ TEST(IoringLogDeviceTest, Benchmark) std::cout << "LLFS_LOG_DEVICE_FILE=" << batt::c_str_literal(file_name) << std::endl; - //---- --- -- - - - - - const auto read_var = [](const char* name, auto default_value) { - using value_type = decltype(default_value); - - const value_type value = - batt::getenv_as("LLFS_STORAGE_CONTEXT_QUEUE_DEPTH").value_or(default_value); + //+++++++++++-+-+--+----- --- -- - - - - - std::cout << name << "=" << value << std::endl; + const usize queue_depth = read_test_var("LLFS_STORAGE_CONTEXT_QUEUE_DEPTH", usize{64}); + const usize thread_pool_size = read_test_var("LLFS_STORAGE_CONTEXT_THREADS", usize{1}); + const usize pages_per_block = read_test_var("LLFS_LOG_DEVICE_PAGES_PER_BLOCK", usize{32}); + const usize log_queue_depth = read_test_var("LLFS_LOG_DEVICE_QUEUE_DEPTH", usize{1024}); - return value; - }; - //---- --- -- - - - - + //+++++++++++-+-+--+----- --- -- - - - - - const usize queue_depth = read_var("LLFS_STORAGE_CONTEXT_QUEUE_DEPTH", usize{64}); - const usize thread_pool_size = read_var("LLFS_STORAGE_CONTEXT_THREADS", usize{1}); - const usize pages_per_block = read_var("LLFS_LOG_DEVICE_PAGES_PER_BLOCK", usize{32}); - const usize log_size = read_var("LLFS_LOG_DEVICE_SIZE_KB", usize{1024 * 64}) * 1024; - const usize log_queue_depth = read_var("LLFS_LOG_DEVICE_QUEUE_DEPTH", usize{1024}); - const usize total_to_write = read_var("LLFS_LOG_DEVICE_WRITE_KB", usize{1024 * 1024}) * 1024; - const usize append_size = read_var("LLFS_LOG_DEVICE_APPEND_SIZE", usize{256}); - const usize trim_size = read_var("LLFS_LOG_DEVICE_TRIM_SIZE", usize{4 * 1024 * 1024}); - const usize trim_trigger = read_var("LLFS_LOG_DEVICE_TRIM_TRIGGER", // - usize{log_size - trim_size * 2}); - const usize repeat_count = read_var("LLFS_LOG_DEVICE_REPEAT", usize{3}); + llfs::run_log_device_benchmark([&](usize log_size, bool create, auto&& consume_log_fn) { + BATT_CHECK(create); - //+++++++++++-+-+--+----- --- -- - - - - - // - for (usize retry = 0; retry < repeat_count; ++retry) { auto scoped_ioring = llfs::ScopedIoRing::make_new(llfs::MaxQueueDepth{queue_depth}, llfs::ThreadPoolSize{thread_pool_size}); @@ -259,75 +246,10 @@ TEST(IoringLogDeviceTest, Benchmark) llfs::IoRingLogDevice& log_device = **status_or_log_device; - // Generate some random data. + // Run the workload. // - std::vector data(32 * 1024 * 1024); - std::default_random_engine rng{1}; - for (u64& word : data) { - word = rng(); - } - - auto start = std::chrono::steady_clock::now(); - - std::thread writer_thread{[&] { - llfs::LogDevice::Writer& log_writer = log_device.writer(); - - std::uniform_int_distribution pick_offset{ - 0, data.size() - (append_size + sizeof(u64) - 1) / sizeof(u64)}; - - usize n_written = 0; - while (n_written < total_to_write) { - BATT_CHECK_OK(log_writer.await(llfs::BytesAvailable{.size = append_size})); - - llfs::StatusOr buffer = log_writer.prepare(append_size); - BATT_CHECK_OK(buffer); - - std::memcpy(buffer->data(), &data[pick_offset(rng)], buffer->size()); - - BATT_CHECK_OK(log_writer.commit(buffer->size())); - - n_written += buffer->size(); - } - - BATT_CHECK_OK(log_device.flush()); - - log_device.halt(); - }}; - - std::thread trimmer_thread{[&] { - for (;;) { - llfs::SlotRange durable = log_device.slot_range(llfs::LogReadMode::kDurable); - - llfs::Status sync_status = - log_device.sync(llfs::LogReadMode::kDurable, - llfs::SlotUpperBoundAt{durable.lower_bound + trim_trigger}); - - if (!sync_status.ok()) { - break; - } - - llfs::Status trim_status = log_device.trim(durable.lower_bound + trim_size); - - if (!trim_status.ok()) { - break; - } - } - }}; - - writer_thread.join(); - trimmer_thread.join(); - log_device.join(); - - auto finish = std::chrono::steady_clock::now(); - - double duration_sec = - double(std::chrono::duration_cast(finish - start).count()) / - (1000.0 * 1000.0); - - LLFS_LOG_INFO() << total_to_write << " bytes written in " << duration_sec - << " seconds; rate=" << (double(total_to_write) / duration_sec) / 1000000.0 - << "MB/s"; - } + consume_log_fn(log_device); + }); } } // namespace From 9b3e15e187ecf0fb234fef7aeb44c94cd982403a Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Tue, 2 Jul 2024 17:25:57 -0400 Subject: [PATCH 04/11] Add RawBlockFileImpl and async_write_some to log device storage. --- src/llfs/ioring_log_device_storage.hpp | 82 ++++++++++++ src/llfs/simulated_log_device_storage.cpp | 4 +- src/llfs/simulated_log_device_storage.hpp | 145 +++++++++++++++++----- src/llfs/storage_simulation.cpp | 41 ++++-- src/llfs/storage_simulation.hpp | 4 +- 5 files changed, 231 insertions(+), 45 deletions(-) diff --git a/src/llfs/ioring_log_device_storage.hpp b/src/llfs/ioring_log_device_storage.hpp index 6572031..84058a5 100644 --- a/src/llfs/ioring_log_device_storage.hpp +++ b/src/llfs/ioring_log_device_storage.hpp @@ -12,9 +12,11 @@ #include // +#include #include #include #include +#include #include #include @@ -39,6 +41,10 @@ class DefaultIoRingLogDeviceStorage */ class EventLoopTask; + /** \brief Implementation of RawBlockFile based on an instance of DefaultIoRingLogDeviceStorage. + */ + class RawBlockFileImpl; + //+++++++++++-+-+--+----- --- -- - - - - DefaultIoRingLogDeviceStorage(const DefaultIoRingLogDeviceStorage&) = delete; @@ -96,6 +102,9 @@ class DefaultIoRingLogDeviceStorage return this->file_.read_all(offset, buffer); } + template + void async_write_some(i64 file_offset, const ConstBuffer& data, Handler&& handler); + template void async_write_some_fixed(i64 file_offset, const ConstBuffer& data, i32 buf_index, Handler&& handler); @@ -112,6 +121,18 @@ class DefaultIoRingLogDeviceStorage //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +template +inline void DefaultIoRingLogDeviceStorage::async_write_some(i64 file_offset, + const ConstBuffer& data, + Handler&& handler) +{ + this->file_.async_write_some(file_offset, data, BATT_FORWARD(handler)); +} + +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// template inline void DefaultIoRingLogDeviceStorage::async_write_some_fixed(i64 file_offset, const ConstBuffer& data, @@ -145,6 +166,67 @@ class DefaultIoRingLogDeviceStorage::EventLoopTask bool join_called_ = false; }; +//=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- + +class DefaultIoRingLogDeviceStorage::RawBlockFileImpl : public RawBlockFile +{ + public: + explicit RawBlockFileImpl(DefaultIoRingLogDeviceStorage& storage) noexcept : storage_{storage} + { + this->storage_.on_work_started(); + this->event_loop_task_.emplace(this->storage_, "RawBlockFileImpl"); + } + + ~RawBlockFileImpl() noexcept + { + BATT_CHECK(this->event_loop_task_); + + this->storage_.on_work_finished(); + this->event_loop_task_->join(); + } + + StatusOr write_some(i64 offset, const ConstBuffer& data) override + { + return batt::Task::await>([&](auto&& handler) { + this->storage_.file_.async_write_some(offset, data, BATT_FORWARD(handler)); + }); + } + + StatusOr read_some(i64 offset, const MutableBuffer& buffer) override + { + return batt::Task::await>([&](auto&& handler) { + this->storage_.file_.async_read_some(offset, buffer, BATT_FORWARD(handler)); + }); + } + + StatusOr get_size() override + { + return sizeof_fd(this->storage_.file_.get_fd()); + } + + Status truncate(i64 new_offset_upper_bound) override + { + return truncate_fd(this->storage_.file_.get_fd(), + BATT_CHECKED_CAST(u64, new_offset_upper_bound)); + } + + Status truncate_at_least(i64 minimum_size) override + { + StatusOr current_size = this->get_size(); + BATT_REQUIRE_OK(current_size); + + if (*current_size < minimum_size) { + return this->truncate(minimum_size); + } + + return batt::OkStatus(); + } + + private: + DefaultIoRingLogDeviceStorage& storage_; + Optional event_loop_task_; +}; + } //namespace llfs #endif // LLFS_IORING_LOG_DEVICE_FILE_HPP diff --git a/src/llfs/simulated_log_device_storage.cpp b/src/llfs/simulated_log_device_storage.cpp index 67a76b9..b440114 100644 --- a/src/llfs/simulated_log_device_storage.cpp +++ b/src/llfs/simulated_log_device_storage.cpp @@ -45,10 +45,10 @@ Status SimulatedLogDeviceStorage::DurableState::validate_args(i64 offset, usize if (offset % sizeof(AlignedBlock) != 0 || size != sizeof(AlignedBlock)) { return batt::StatusCode::kInvalidArgument; } - if (offset < this->config_.physical_offset) { + if (offset < this->file_offset_.lower_bound) { return batt::StatusCode::kOutOfRange; } - if (offset + size > this->config_.physical_offset + this->config_.physical_size) { + if (offset + static_cast(size) > this->file_offset_.upper_bound) { return batt::StatusCode::kOutOfRange; } if (this->simulation_.inject_failure()) { diff --git a/src/llfs/simulated_log_device_storage.hpp b/src/llfs/simulated_log_device_storage.hpp index f790105..f15050f 100644 --- a/src/llfs/simulated_log_device_storage.hpp +++ b/src/llfs/simulated_log_device_storage.hpp @@ -12,8 +12,9 @@ #include // -#include +#include #include +#include #include #include @@ -54,9 +55,11 @@ class SimulatedLogDeviceStorage //+++++++++++-+-+--+----- --- -- - - - - - explicit DurableState(StorageSimulation& simulation, const IoRingLogConfig& config) noexcept + explicit DurableState(StorageSimulation& simulation, usize log_size, + const Interval& file_offset) noexcept : simulation_{simulation} - , config_{config} + , log_size_{log_size} + , file_offset_{file_offset} { } @@ -72,9 +75,24 @@ class SimulatedLogDeviceStorage return this->simulation_; } - const IoRingLogConfig& config() const noexcept + usize log_size() const noexcept { - return this->config_; + return this->log_size_; + } + + const Interval& file_offset() const noexcept + { + return this->file_offset_; + } + + bool is_initialized() const noexcept + { + return this->is_initialized_; + } + + void set_initialized(bool b) noexcept + { + this->is_initialized_ = b; } //+++++++++++-+-+--+----- --- -- - - - - @@ -120,14 +138,57 @@ class SimulatedLogDeviceStorage //+++++++++++-+-+--+----- --- -- - - - - const i64 id_{SimulatedLogDeviceStorage::new_id()}; + StorageSimulation& simulation_; - const IoRingLogConfig config_; + + /** \brief The maximum capacity (bytes) of the log. + */ + const usize log_size_; + + /** \brief The valid range of offsets for this log device on the storage media. + */ + const Interval file_offset_; + + /** \brief Whether the simulated storage media has been initialized for this device. This is + * set by the StorageSimulation (or other external user) code. + */ + bool is_initialized_ = false; + batt::Mutex impl_; }; + //+++++++++++-+-+--+----- --- -- - - - - + + /** \brief RawBlockFile interface view of the DurableState. + * + * Used for initialization of the simulated media. + */ + class RawBlockFileImpl : public RawBlockFile + { + public: + explicit RawBlockFileImpl(std::shared_ptr&& durable_state) noexcept; + + //+++++++++++-+-+--+----- --- -- - - - - + + StatusOr write_some(i64 offset, const ConstBuffer& buffer_arg) override; + + StatusOr read_some(i64 offset, const MutableBuffer& buffer_arg) override; + + StatusOr get_size() override; + + //+++++++++++-+-+--+----- --- -- - - - - + private: + const std::shared_ptr durable_state_; + const u64 creation_step_; + }; + + //+++++++++++-+-+--+----- --- -- - - - - + class EphemeralState : public batt::RefCounted { public: + // Index values for this->work_count_per_caller_ array (see below). + // enum : usize { PUBLIC_API = 0, POST_TO_EVENT_LOOP, @@ -148,11 +209,33 @@ class SimulatedLogDeviceStorage return this->id_; } - const IoRingLogConfig& config() const noexcept + usize log_size() const noexcept + { + return this->durable_state_->log_size(); + } + + const Interval& file_offset() const noexcept + { + return this->durable_state_->file_offset(); + } + + bool is_initialized() const noexcept + { + return this->durable_state_->is_initialized(); + } + + void set_initialized(bool b) noexcept { - return this->durable_state_->config(); + this->durable_state_->set_initialized(b); } + std::unique_ptr get_raw_block_file() + { + return std::make_unique(batt::make_copy(this->durable_state_)); + } + + //----- --- -- - - - - + Status close(); void on_work_started(usize caller = PUBLIC_API); @@ -210,25 +293,6 @@ class SimulatedLogDeviceStorage std::atomic stopped_{false}; }; - class RawBlockFileImpl : public RawBlockFile - { - public: - explicit RawBlockFileImpl(std::shared_ptr&& durable_state) noexcept; - - //+++++++++++-+-+--+----- --- -- - - - - - - StatusOr write_some(i64 offset, const ConstBuffer& buffer_arg) override; - - StatusOr read_some(i64 offset, const MutableBuffer& buffer_arg) override; - - StatusOr get_size() override; - - //+++++++++++-+-+--+----- --- -- - - - - - private: - const std::shared_ptr durable_state_; - const u64 creation_step_; - }; - //+++++++++++-+-+--+----- --- -- - - - - class EventLoopTask @@ -267,11 +331,28 @@ class SimulatedLogDeviceStorage //+++++++++++-+-+--+----- --- -- - - - - - const IoRingLogConfig& config() const noexcept + usize log_size() const noexcept + { + return this->impl_->log_size(); + } + + bool is_initialized() const noexcept + { + return this->impl_->is_initialized(); + } + + void set_initialized(bool b) noexcept + { + this->impl_->set_initialized(b); + } + + std::unique_ptr get_raw_block_file() { - return this->impl_->config(); + return this->impl_->get_raw_block_file(); } + //+++++++++++-+-+--+----- --- -- - - - - + Status register_fd() { return batt::OkStatus(); @@ -318,6 +399,12 @@ class SimulatedLogDeviceStorage return this->impl_->read_all(offset, buffer); } + template + void async_write_some(i64 file_offset, const ConstBuffer& data, Handler&& handler) + { + this->impl_->async_write_some(file_offset, data, BATT_FORWARD(handler)); + } + // The buf_index arg here isn't used by the simulated impl, but it is needed by the other type // used to instantiate IoRingLogDriver, DefaultIoRingLogDeviceStorage. // diff --git a/src/llfs/storage_simulation.cpp b/src/llfs/storage_simulation.cpp index f7fa847..072e651 100644 --- a/src/llfs/storage_simulation.cpp +++ b/src/llfs/storage_simulation.cpp @@ -200,8 +200,8 @@ void StorageSimulation::handle_events(bool main_fn_done) //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -SimulatedLogDeviceStorage StorageSimulation::get_log_device_storage(const std::string& name, - Optional capacity) +SimulatedLogDeviceStorage StorageSimulation::get_log_device_storage( + const std::string& name, Optional capacity, Optional> file_offset) { auto iter = this->log_storage_.find(name); @@ -209,16 +209,10 @@ SimulatedLogDeviceStorage StorageSimulation::get_log_device_storage(const std::s // if (iter == this->log_storage_.end()) { BATT_CHECK(capacity.has_value()); + BATT_CHECK(file_offset.has_value()); - auto config = IoRingLogConfig::from_logical_size(*capacity); - auto durable_state = std::make_shared(*this, config); - - Status init_status = - initialize_ioring_log_device(*std::make_unique( - batt::make_copy(durable_state)), - config, ConfirmThisWillEraseAllMyData::kYes); - - BATT_CHECK_OK(init_status); + auto durable_state = + std::make_shared(*this, *capacity, *file_offset); iter = this->log_storage_.emplace(name, std::move(durable_state)).first; } @@ -235,8 +229,29 @@ std::unique_ptr StorageSimulation::get_log_device(const std::string& Optional capacity) { if (this->low_level_log_devices_) { - SimulatedLogDeviceStorage storage = this->get_log_device_storage(name, capacity); - IoRingLogConfig config = storage.config(); + auto file_offset = [&]() -> Optional> { + if (!capacity) { + return None; + } + auto config = IoRingLogConfig::from_logical_size(*capacity); + return Interval{ + .lower_bound = BATT_CHECKED_CAST(i64, config.physical_offset), + .upper_bound = BATT_CHECKED_CAST(i64, config.physical_offset + config.physical_size), + }; + }(); + SimulatedLogDeviceStorage storage = this->get_log_device_storage(name, capacity, file_offset); + + auto config = IoRingLogConfig::from_logical_size(storage.log_size()); + + if (!storage.is_initialized()) { + Status init_status = initialize_ioring_log_device(*storage.get_raw_block_file(), config, + ConfirmThisWillEraseAllMyData::kYes); + + BATT_CHECK_OK(init_status); + + storage.set_initialized(true); + } + auto options = IoRingLogDriverOptions::with_default_values(); options.name = name; options.limit_queue_depth(config.block_count()); diff --git a/src/llfs/storage_simulation.hpp b/src/llfs/storage_simulation.hpp index 4638080..9c44b41 100644 --- a/src/llfs/storage_simulation.hpp +++ b/src/llfs/storage_simulation.hpp @@ -15,6 +15,7 @@ #include #include +#include #include #include #include @@ -191,7 +192,8 @@ class StorageSimulation * \param name A unique name used to identify the LogDevice storage in the context of this * simulation */ - SimulatedLogDeviceStorage get_log_device_storage(const std::string& name, Optional capacity); + SimulatedLogDeviceStorage get_log_device_storage(const std::string& name, Optional capacity, + Optional> file_offset); /** \brief Creates/accesses a simulated LogDevice. * From ee5c533aedd480fc73c5b69a6a1e6d228585b9b0 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Tue, 2 Jul 2024 17:26:41 -0400 Subject: [PATCH 05/11] Add SlotRangePriority for sorting SlotRanges in a max-heap. --- src/llfs/slot.hpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/llfs/slot.hpp b/src/llfs/slot.hpp index cadc39e..655438b 100644 --- a/src/llfs/slot.hpp +++ b/src/llfs/slot.hpp @@ -322,6 +322,15 @@ struct SlotRangeOrder { } }; +struct SlotRangePriority { + template + bool operator()(const First& first, const Second& second) const + { + return slot_greater_or_equal(get_slot_range(first).lower_bound, + get_slot_range(second).upper_bound); + } +}; + //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- #define LLFS_CHECK_SLOT_LT(first, second) \ From 6cb9a237a4bc2a68fba2335a72fa80485c588233 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Tue, 2 Jul 2024 17:27:26 -0400 Subject: [PATCH 06/11] Add IoRingLogDevice2 configs to storage context/file. --- src/llfs/ioring_log_driver.ipp | 3 +- src/llfs/ioring_log_driver_options.cpp | 22 ------ src/llfs/ioring_log_driver_options.hpp | 83 +------------------- src/llfs/log_device_config.hpp | 62 --------------- src/llfs/packed_config.hpp | 1 + src/llfs/packed_page_user_slot.hpp | 3 +- src/llfs/page_allocator_config.cpp | 32 +++++++- src/llfs/page_allocator_config.hpp | 2 +- src/llfs/ring_buffer.cpp | 10 ++- src/llfs/storage_context.cpp | 35 +++++++++ src/llfs/storage_context.hpp | 6 ++ src/llfs/volume_config.cpp | 103 ++++++++++++++++--------- src/llfs/volume_config.hpp | 5 +- src/llfs/volume_runtime_options.cpp | 4 +- src/llfs/volume_runtime_options.hpp | 8 +- 15 files changed, 163 insertions(+), 216 deletions(-) delete mode 100644 src/llfs/ioring_log_driver_options.cpp diff --git a/src/llfs/ioring_log_driver.ipp b/src/llfs/ioring_log_driver.ipp index 906150b..6a92584 100644 --- a/src/llfs/ioring_log_driver.ipp +++ b/src/llfs/ioring_log_driver.ipp @@ -285,8 +285,7 @@ inline void BasicIoRingLogDriver::flush_task_main() << " block_size=0x" << this->calculate().block_size() << " block_capacity=0x" << this->calculate().block_capacity() << " queue_depth=" << std::dec << this->calculate().queue_depth(); - LLFS_VLOG(1) << "(driver=" << this->name_ - << ") buffer delay=" << this->options_.page_write_buffer_delay_usec << "usec"; + LLFS_VLOG(1) << "(driver=" << this->name_ << ")"; // Now tell the ops to start flushing data. They will write in parallel but only one at a // time will perform an async_wait on `commit_pos_`, to prevent thundering herd bottlenecks. diff --git a/src/llfs/ioring_log_driver_options.cpp b/src/llfs/ioring_log_driver_options.cpp deleted file mode 100644 index 51f6cf2..0000000 --- a/src/llfs/ioring_log_driver_options.cpp +++ /dev/null @@ -1,22 +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 -// -//+++++++++++-+-+--+----- --- -- - - - - - -#include -// - -namespace llfs { - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -/*static*/ IoRingLogDriverOptions IoRingLogDriverOptions::with_default_values() -{ - IoRingLogDriverOptions options; - return options; -} - -} // namespace llfs diff --git a/src/llfs/ioring_log_driver_options.hpp b/src/llfs/ioring_log_driver_options.hpp index 4bf22f4..8eb52de 100644 --- a/src/llfs/ioring_log_driver_options.hpp +++ b/src/llfs/ioring_log_driver_options.hpp @@ -10,89 +10,14 @@ #ifndef LLFS_IORING_LOG_DRIVER_OPTIONS_HPP #define LLFS_IORING_LOG_DRIVER_OPTIONS_HPP -#include - -#include -#include - -#include -#include - -namespace llfs { - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// Performance tuning options. +#include // -class IoRingLogDriverOptions -{ - public: - using Self = IoRingLogDriverOptions; - - static IoRingLogDriverOptions with_default_values(); - - static int next_id() - { - static std::atomic n{1}; - return n.fetch_add(1); - } - - //+++++++++++-+-+--+----- --- -- - - - - - IoRingLogDriverOptions() noexcept - { - } +#include - // The debug name of this log. - // - std::string name = batt::to_string("(anonymous log ", next_id(), ")"); - - // How long to wait for a full page worth of log data before flushing to disk. - // - u32 page_write_buffer_delay_usec = 0; // TODO [tastolfi 2021-06-21] remove or implement - - // How many log segments to flush in parallel. - // - usize queue_depth_log2 = 4; - - //+++++++++++-+-+--+----- --- -- - - - - - - usize queue_depth() const - { - return usize{1} << this->queue_depth_log2; - } - - Self& set_queue_depth(usize n) - { - this->queue_depth_log2 = batt::log2_ceil(n); - BATT_CHECK_EQ(this->queue_depth(), n) << "The queue depth must be a power of 2!"; - return *this; - } - - /** \brief Sets queue depth to the smallest power of 2 that is not greater than `max_n` *and* not - * greater than the current value of queue depth. - * - * `max_n` must be at least 2. - */ - Self& limit_queue_depth(usize max_n) - { - BATT_CHECK_GT(max_n, 1); - while (this->queue_depth() > max_n) { - --this->queue_depth_log2; - } - return *this; - } - - usize queue_depth_mask() const - { - return this->queue_depth() - 1; - } +namespace llfs { - Self& set_name(std::string_view name) - { - this->name = name; - return *this; - } -}; +using IoRingLogDriverOptions = LogDeviceRuntimeOptions; } // namespace llfs diff --git a/src/llfs/log_device_config.hpp b/src/llfs/log_device_config.hpp index c90305d..b016960 100644 --- a/src/llfs/log_device_config.hpp +++ b/src/llfs/log_device_config.hpp @@ -108,68 +108,6 @@ inline bool operator!=(const LogDeviceConfigOptions& l, const LogDeviceConfigOpt return !(l == r); } -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// These types are provided for the convenience of more complex configs that nest one or more -// LogDevice configs. -// - -//+++++++++++-+-+--+----- --- -- - - - - -// -struct CreateNewLogDevice { - LogDeviceConfigOptions options; -}; - -inline bool operator==(const CreateNewLogDevice& l, const CreateNewLogDevice& r) -{ - return l.options == r.options; -} - -inline bool operator!=(const CreateNewLogDevice& l, const CreateNewLogDevice& r) -{ - return !(l == r); -} - -//+++++++++++-+-+--+----- --- -- - - - - -// -struct CreateNewLogDeviceWithDefaultSize { - Optional uuid; - Optional pages_per_block_log2; -}; - -inline bool operator==(const CreateNewLogDeviceWithDefaultSize& l, - const CreateNewLogDeviceWithDefaultSize& r) -{ - return l.uuid == r.uuid // - && l.pages_per_block_log2 == r.pages_per_block_log2; -} - -inline bool operator!=(const CreateNewLogDeviceWithDefaultSize& l, - const CreateNewLogDeviceWithDefaultSize& r) -{ - return !(l == r); -} - -//+++++++++++-+-+--+----- --- -- - - - - -// -struct LinkToExistingLogDevice { - boost::uuids::uuid uuid; -}; - -inline bool operator==(const LinkToExistingLogDevice& l, const LinkToExistingLogDevice& r) -{ - return l.uuid == r.uuid; -} - -inline bool operator!=(const LinkToExistingLogDevice& l, const LinkToExistingLogDevice& r) -{ - return !(l == r); -} - -//+++++++++++-+-+--+----- --- -- - - - - - -using NestedLogDeviceConfig = - std::variant; - //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- // struct PackedLogDeviceConfig : PackedConfigSlotHeader { diff --git a/src/llfs/packed_config.hpp b/src/llfs/packed_config.hpp index e7bc76a..22c9f1b 100644 --- a/src/llfs/packed_config.hpp +++ b/src/llfs/packed_config.hpp @@ -32,6 +32,7 @@ struct PackedConfigSlotBase { static constexpr u16 kLogDevice = 3; // static constexpr u16 kPageDevice = 4; // static constexpr u16 kPageAllocator = 5; // + static constexpr u16 kLogDevice2 = 6; // Simpler, optimized version of kLogDevice // The range [0x1000..0x1fff] is reserved for continuation slots. diff --git a/src/llfs/packed_page_user_slot.hpp b/src/llfs/packed_page_user_slot.hpp index 5d59560..4a3e8ca 100644 --- a/src/llfs/packed_page_user_slot.hpp +++ b/src/llfs/packed_page_user_slot.hpp @@ -11,6 +11,7 @@ #define LLFS_PACKED_PAGE_USER_SLOT_HPP #include +#include #include @@ -20,8 +21,6 @@ namespace llfs { -using PackedSlotOffset = little_u64; - //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // Represents a user-specified logical timestamp for a page. The `user` here isn't necessarily an // end-user or human, it could be another part of the system (e.g., a Tablet). diff --git a/src/llfs/page_allocator_config.cpp b/src/llfs/page_allocator_config.cpp index ee738fa..1101684 100644 --- a/src/llfs/page_allocator_config.cpp +++ b/src/llfs/page_allocator_config.cpp @@ -94,6 +94,34 @@ Status configure_storage_object(StorageFileBuilder::Transaction& txn, //----- --- -- - - - - [&](const LinkToExistingLogDevice& link_to_existing) -> StatusOr { return link_to_existing.uuid; + }, + + //----- --- -- - - - - + [&](const CreateNewLogDevice2& create_new) -> StatusOr { + if (create_new.options.log_size < minimum_log_size) { + return {batt::StatusCode::kInvalidArgument}; + } + + BATT_ASSIGN_OK_RESULT( + const FileOffsetPtr p_log_device_config, + txn.add_object(create_new.options)); + + return p_log_device_config->uuid; + }, + + //----- --- -- - - - - + [&](const CreateNewLogDevice2WithDefaultSize& log_options) + -> StatusOr { + BATT_ASSIGN_OK_RESULT( + const FileOffsetPtr p_log_device_config, + txn.add_object(LogDeviceConfigOptions2{ + .uuid = log_options.uuid, + .log_size = minimum_log_size, + .device_page_size_log2 = LogDeviceConfigOptions2::kDefaultDevicePageSizeLog2, + .data_alignment_log2 = LogDeviceConfigOptions2::kDefaultDataAlignmentLog2, + })); + + return p_log_device_config->uuid; })); //+++++++++++-+-+--+----- --- -- - - - - @@ -146,8 +174,8 @@ StatusOr> recover_storage_object( const PageAllocatorRuntimeOptions& allocator_options, // const IoRingLogDriverOptions& log_options) { - StatusOr> log_factory = storage_context->recover_object( - batt::StaticType{}, p_allocator_config->log_device_uuid, log_options); + StatusOr> log_factory = + storage_context->recover_log_device(p_allocator_config->log_device_uuid, log_options); BATT_REQUIRE_OK(log_factory); diff --git a/src/llfs/page_allocator_config.hpp b/src/llfs/page_allocator_config.hpp index 89186d5..288b509 100644 --- a/src/llfs/page_allocator_config.hpp +++ b/src/llfs/page_allocator_config.hpp @@ -18,7 +18,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/llfs/ring_buffer.cpp b/src/llfs/ring_buffer.cpp index d3e0979..9053cd8 100644 --- a/src/llfs/ring_buffer.cpp +++ b/src/llfs/ring_buffer.cpp @@ -333,11 +333,13 @@ void RingBuffer::Impl::update_mapped_regions() noexcept // Map each half of the buffer into the underlying file. // - BATT_CHECK_EQ(mirror_0, - mmap(mirror_0, this->size_, mode, flags, this->fd_, this->offset_within_file_)); + BATT_CHECK_EQ((void*)mirror_0, (void*)mmap(mirror_0, this->size_, mode, flags, this->fd_, + this->offset_within_file_)) + << BATT_INSPECT(this->size_); - BATT_CHECK_EQ(mirror_1, - mmap(mirror_1, this->size_, mode, flags, this->fd_, this->offset_within_file_)); + BATT_CHECK_EQ((void*)mirror_1, (void*)mmap(mirror_1, this->size_, mode, flags, this->fd_, + this->offset_within_file_)) + << BATT_INSPECT(this->size_); LLFS_WARN_IF_NOT_OK(batt::status_from_retval(batt::syscall_retry([&] { return madvise(this->memory_, this->capacity_ * 2, MADV_SEQUENTIAL); diff --git a/src/llfs/storage_context.cpp b/src/llfs/storage_context.cpp index 94bfdfe..fadff0a 100644 --- a/src/llfs/storage_context.cpp +++ b/src/llfs/storage_context.cpp @@ -94,6 +94,41 @@ Status StorageContext::add_existing_file(const batt::SharedPtr& fil return OkStatus(); } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr> StorageContext::recover_log_device( + const boost::uuids::uuid& uuid, const LogDeviceRuntimeOptions& log_runtime_options) +{ + batt::SharedPtr info = this->find_object_by_uuid(uuid); + if (!info) { + return {batt::StatusCode::kNotFound}; + } + + switch (info->p_config_slot->tag) { + //----- --- -- - - - - + case PackedConfigSlotBase::Tag::kLogDevice: + return recover_storage_object( + batt::shared_ptr_from(this), info->storage_file->file_name(), + FileOffsetPtr{ + config_slot_cast(info->p_config_slot.object), + info->p_config_slot.file_offset}, + log_runtime_options); + + //----- --- -- - - - - + case PackedConfigSlotBase::Tag::kLogDevice2: + return recover_storage_object( + batt::shared_ptr_from(this), info->storage_file->file_name(), + FileOffsetPtr{ + config_slot_cast(info->p_config_slot.object), + info->p_config_slot.file_offset}, + log_runtime_options); + + //----- --- -- - - - - + default: + return ::llfs::make_status(::llfs::StatusCode::kStorageObjectTypeError); + } +} + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // void StorageContext::set_page_cache_options(const PageCacheOptions& options) diff --git a/src/llfs/storage_context.hpp b/src/llfs/storage_context.hpp index 0370e73..fe107c4 100644 --- a/src/llfs/storage_context.hpp +++ b/src/llfs/storage_context.hpp @@ -16,6 +16,7 @@ #ifndef LLFS_DISABLE_IO_URING #include +#include #include #include #include @@ -130,6 +131,11 @@ class StorageContext : public batt::RefCounted BATT_FORWARD(extra_options)...); } + /** \brief Special case for LogDevice recovery; handles both IoRingLogDevice and IoRingLogDevice2. + */ + StatusOr> recover_log_device( + const boost::uuids::uuid& uuid, const LogDeviceRuntimeOptions& log_runtime_options); + //+++++++++++-+-+--+----- --- -- - - - - private: // Passed in at creation time; used to schedule all background tasks needed by recovered objects diff --git a/src/llfs/volume_config.cpp b/src/llfs/volume_config.cpp index b7669b8..471872f 100644 --- a/src/llfs/volume_config.cpp +++ b/src/llfs/volume_config.cpp @@ -28,20 +28,14 @@ BATT_PRINT_OBJECT_IMPL(PackedVolumeConfig, // (recycler_log_uuid) // ) +namespace { + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // -Status configure_storage_object(StorageFileBuilder::Transaction& txn, - FileOffsetPtr p_config, - const VolumeConfigOptions& options) +StatusOr> add_recycler_log( + StorageFileBuilder::Transaction& txn, batt::StaticType, + const VolumeConfigOptions& options) { - BATT_CHECK(!options.root_log.uuid) - << "Creating a Volume from a pre-existing root log is not supported"; - - StatusOr> p_root_log_config = - txn.add_object(options.root_log); - - BATT_REQUIRE_OK(p_root_log_config); - const LogDeviceConfigOptions recycler_log_options{ .uuid = random_uuid(), .pages_per_block_log2 = IoRingLogConfig::kDefaultPagesPerBlockLog2 + 1, @@ -50,29 +44,67 @@ Status configure_storage_object(StorageFileBuilder::Transaction& txn, options.recycler_max_buffered_page_count), }; - StatusOr> p_recycler_log_config = - txn.add_object(recycler_log_options); - - BATT_REQUIRE_OK(p_recycler_log_config); + return txn.add_object(recycler_log_options); +} - p_config->uuid = options.base.uuid.value_or(random_uuid()); - p_config->slot_i = 0; - p_config->n_slots = 2; - p_config->max_refs_per_page = options.base.max_refs_per_page; - p_config->root_log_uuid = (*p_root_log_config)->uuid; - p_config->recycler_log_uuid = (*p_recycler_log_config)->uuid; +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +StatusOr> add_recycler_log( + StorageFileBuilder::Transaction& txn, batt::StaticType, + const VolumeConfigOptions& options) +{ + const LogDeviceConfigOptions2 recycler_log_options{ + .uuid = random_uuid(), + .log_size = PageRecycler::calculate_log_size( + PageRecyclerOptions{}.set_max_refs_per_page(options.base.max_refs_per_page), + options.recycler_max_buffered_page_count), + .device_page_size_log2 = None, + .data_alignment_log2 = None, + }; - p_config->slot_1.tag = PackedConfigSlotBase::Tag::kVolumeContinuation; - p_config->slot_1.slot_i = 1; - p_config->slot_1.n_slots = 2; - p_config->trim_lock_update_interval_bytes = options.base.trim_lock_update_interval; - p_config->trim_delay_byte_count = options.base.trim_delay_byte_count; + return txn.add_object(recycler_log_options); +} - if (!txn.packer().pack_string_to(&p_config->name, options.base.name)) { - return ::batt::StatusCode::kResourceExhausted; - } +} // namespace - return OkStatus(); +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +Status configure_storage_object(StorageFileBuilder::Transaction& txn, + FileOffsetPtr p_config, + const VolumeConfigOptions& options) +{ + return batt::case_of(options.root_log, [&](const auto& root_log_options) -> Status { + BATT_CHECK(!root_log_options.uuid) + << "Creating a Volume from a pre-existing root log is not supported"; + + using LogDeviceOptionsType = std::decay_t; + + auto p_root_log_config = txn.add_object(root_log_options); + BATT_REQUIRE_OK(p_root_log_config); + + auto p_recycler_log_config = + add_recycler_log(txn, batt::StaticType{}, options); + BATT_REQUIRE_OK(p_recycler_log_config); + + p_config->uuid = options.base.uuid.value_or(random_uuid()); + p_config->slot_i = 0; + p_config->n_slots = 2; + p_config->max_refs_per_page = options.base.max_refs_per_page; + p_config->root_log_uuid = (*p_root_log_config)->uuid; + p_config->recycler_log_uuid = (*p_recycler_log_config)->uuid; + + p_config->slot_1.tag = PackedConfigSlotBase::Tag::kVolumeContinuation; + p_config->slot_1.slot_i = 1; + p_config->slot_1.n_slots = 2; + p_config->trim_lock_update_interval_bytes = options.base.trim_lock_update_interval; + p_config->trim_delay_byte_count = options.base.trim_delay_byte_count; + + if (!txn.packer().pack_string_to(&p_config->name, options.base.name)) { + return ::batt::StatusCode::kResourceExhausted; + } + + return OkStatus(); + }); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -85,15 +117,14 @@ StatusOr> recover_storage_object( StatusOr> page_cache = storage_context->get_page_cache(); BATT_REQUIRE_OK(page_cache); - StatusOr> root_log_factory = storage_context->recover_object( - batt::StaticType{}, p_volume_config->root_log_uuid, - volume_runtime_options.root_log_options); + StatusOr> root_log_factory = + storage_context->recover_log_device(p_volume_config->root_log_uuid, + volume_runtime_options.root_log_options); BATT_REQUIRE_OK(root_log_factory); StatusOr> recycler_log_factory = - storage_context->recover_object(batt::StaticType{}, - p_volume_config->recycler_log_uuid, - volume_runtime_options.recycler_log_options); + storage_context->recover_log_device(p_volume_config->recycler_log_uuid, + volume_runtime_options.recycler_log_options); BATT_REQUIRE_OK(recycler_log_factory); VolumeRecoverParams params{ diff --git a/src/llfs/volume_config.hpp b/src/llfs/volume_config.hpp index cdf0ad1..4cdbf5a 100644 --- a/src/llfs/volume_config.hpp +++ b/src/llfs/volume_config.hpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,8 @@ #include +#include + namespace llfs { //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -61,7 +64,7 @@ struct VolumeConfigOptions { // Options controlling the creation of the root log (WAL). // - LogDeviceConfigOptions root_log; + std::variant root_log; // Used to calculate the minimum recycler log size. // diff --git a/src/llfs/volume_runtime_options.cpp b/src/llfs/volume_runtime_options.cpp index f59f50b..48a1c38 100644 --- a/src/llfs/volume_runtime_options.cpp +++ b/src/llfs/volume_runtime_options.cpp @@ -19,8 +19,8 @@ namespace llfs { .slot_visitor_fn = [](const SlotParse& /*slot*/, std::string_view /*user_data*/) -> Status { return OkStatus(); }, - .root_log_options = IoRingLogDriverOptions::with_default_values(), - .recycler_log_options = IoRingLogDriverOptions::with_default_values(), + .root_log_options = LogDeviceRuntimeOptions::with_default_values(), + .recycler_log_options = LogDeviceRuntimeOptions::with_default_values(), .trim_control = nullptr, }; } diff --git a/src/llfs/volume_runtime_options.hpp b/src/llfs/volume_runtime_options.hpp index 4aafea3..9b83557 100644 --- a/src/llfs/volume_runtime_options.hpp +++ b/src/llfs/volume_runtime_options.hpp @@ -10,7 +10,9 @@ #ifndef LLFS_VOLUME_RUNTIME_OPTIONS_HPP #define LLFS_VOLUME_RUNTIME_OPTIONS_HPP -#include +#include +// +#include #include #include @@ -32,11 +34,11 @@ struct VolumeRuntimeOptions { // Runtime options used to tune the behavior of the root log driver. // - IoRingLogDriverOptions root_log_options; + LogDeviceRuntimeOptions root_log_options; // Runtime options used to tune the behavior of the recycler log driver. // - IoRingLogDriverOptions recycler_log_options; + LogDeviceRuntimeOptions recycler_log_options; // (Optional) The SlotLockManager to use for trimming the recovered Volume's root log. If // `nullptr`, a new SlotLockManager will be created. From 8f2d4e3732d564f15152d409b419bd16dcd9cff9 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Thu, 18 Jul 2024 17:34:16 -0400 Subject: [PATCH 07/11] Only update control block with known commit pos values for flush pos. --- src/llfs/ioring_log_device2.hpp | 15 ++++++++-- src/llfs/ioring_log_device2.ipp | 53 ++++++++++++++++++++++++++++----- 2 files changed, 58 insertions(+), 10 deletions(-) diff --git a/src/llfs/ioring_log_device2.hpp b/src/llfs/ioring_log_device2.hpp index 687ed70..bfb7743 100644 --- a/src/llfs/ioring_log_device2.hpp +++ b/src/llfs/ioring_log_device2.hpp @@ -26,7 +26,9 @@ #include #include +#include #include +#include namespace llfs { @@ -178,6 +180,7 @@ class IoRingLogDriver2 * - this->commit_pos_ * - this->started_flush_upper_bound_ * - this->known_flush_pos_ + * - this->known_flushed_commit_pos_ * * Should only be called during recovery. */ @@ -210,7 +213,7 @@ class IoRingLogDriver2 * two writes may be initiated by this function, provided the conditions above are still met after * starting the first write. */ - void start_flush(slot_offset_type observed_commit_pos); + void start_flush(CommitPos observed_commit_pos); /** \brief Returns the passed slot range with the lower and upper bounds aligned to the nearest * data page boundary. @@ -254,7 +257,7 @@ class IoRingLogDriver2 * * Only one pending async write to the control block is allowed at a time. */ - void start_control_block_update(slot_offset_type observed_target_trim_pos) noexcept; + void start_control_block_update(TargetTrimPos observed_target_trim_pos) noexcept; /** \brief I/O callback that handles the completion of a write to the control block. */ @@ -365,6 +368,10 @@ class IoRingLogDriver2 */ slot_offset_type known_flush_pos_ = 0; + /** \brief The highest observed commit pos known to be flushed. + */ + slot_offset_type known_flushed_commit_pos_ = 0; + /** \brief The trailing aligned data page for the highest-offset pending flush operation. This is * used to avoid concurrently updating the same data page. */ @@ -397,6 +404,10 @@ class IoRingLogDriver2 */ PackedLogControlBlock2* control_block_ = nullptr; + /** \brief A queue of observed commit positions that have triggered an async write. + */ + std::deque observed_commit_offsets_; + /** \brief A min-heap of confirmed flushed slot ranges; used to advance this->known_flush_pos_, * which in turn drives the update of the control block and this->flush_pos_. */ diff --git a/src/llfs/ioring_log_device2.ipp b/src/llfs/ioring_log_device2.ipp index 9799735..a0811f6 100644 --- a/src/llfs/ioring_log_device2.ipp +++ b/src/llfs/ioring_log_device2.ipp @@ -307,6 +307,8 @@ Status IoRingLogDriver2::recover_flush_pos() noexcept this->reset_flush_pos(confirmed_flush_pos); + BATT_CHECK_EQ(recovered_flush_pos, confirmed_flush_pos); + return OkStatus(); } @@ -327,6 +329,7 @@ void IoRingLogDriver2::reset_flush_pos(slot_offset_type new_flush_pos) this->observed_watch_[kCommitPos].set_value(new_flush_pos); this->unflushed_lower_bound_ = new_flush_pos; this->known_flush_pos_ = new_flush_pos; + this->known_flushed_commit_pos_ = new_flush_pos; this->flush_pos_.set_value(new_flush_pos); } @@ -393,7 +396,7 @@ inline void IoRingLogDriver2::wait_for_slot_offset_change(T observed_v //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // template -inline void IoRingLogDriver2::start_flush(slot_offset_type observed_commit_pos) +inline void IoRingLogDriver2::start_flush(CommitPos observed_commit_pos) { slot_offset_type flush_upper_bound = this->unflushed_lower_bound_; @@ -470,6 +473,19 @@ inline void IoRingLogDriver2::start_flush(slot_offset_type observed_co BATT_CHECK(!new_flush_tail.empty()); this->flush_tail_.emplace(new_flush_tail); + // Save this observed commit pos. + // + if (this->observed_commit_offsets_.empty() || + this->observed_commit_offsets_.back() != observed_commit_pos) { + // Sanity check: verify that observed commit offsets are in non-decreasing order. + // + if (!this->observed_commit_offsets_.empty()) { + LLFS_CHECK_SLOT_LT(this->observed_commit_offsets_.back(), observed_commit_pos); + } + + this->observed_commit_offsets_.push_back(observed_commit_pos); + } + // Start writing! // this->start_flush_write(slot_range, aligned_range); @@ -619,13 +635,32 @@ void IoRingLogDriver2::update_known_flush_pos(const SlotRange& flushed std::pop_heap(this->flushed_ranges_.begin(), this->flushed_ranges_.end(), SlotRangePriority{}); this->flushed_ranges_.pop_back(); } + + // Advance known_flushed_commit_pos_ by consuming values from this->observed_commit_offsets_. + // + while (!this->observed_commit_offsets_.empty()) { + const CommitPos previously_observed_commit_pos = this->observed_commit_offsets_.front(); + + // If the next observed commit pos is not yet flushed, we are done. + // + if (slot_less_than(this->known_flush_pos_, previously_observed_commit_pos)) { + break; + } + + // Sanity check: verify that observed commit offsets are in non-decreasing order. + // + LLFS_CHECK_SLOT_LE(this->known_flushed_commit_pos_, previously_observed_commit_pos); + + this->known_flushed_commit_pos_ = previously_observed_commit_pos; + this->observed_commit_offsets_.pop_front(); + } } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // template void IoRingLogDriver2::start_control_block_update( - slot_offset_type observed_target_trim_pos) noexcept + TargetTrimPos observed_target_trim_pos) noexcept { if (this->writing_control_block_ || this->trim_pos_.is_closed() || this->flush_pos_.is_closed()) { return; @@ -633,20 +668,22 @@ void IoRingLogDriver2::start_control_block_update( BATT_CHECK_NOT_NULLPTR(this->control_block_) << "Forgot to call read_control_block()?"; + const slot_offset_type effective_target_flush_pos = this->known_flushed_commit_pos_; const slot_offset_type effective_target_trim_pos = - slot_min(this->known_flush_pos_, observed_target_trim_pos); + slot_min(effective_target_flush_pos, observed_target_trim_pos); + const slot_offset_type observed_trim_pos = this->trim_pos_.get_value(); const slot_offset_type observed_flush_pos = this->flush_pos_.get_value(); if (observed_trim_pos == effective_target_trim_pos && - observed_flush_pos == this->known_flush_pos_) { + observed_flush_pos == effective_target_flush_pos) { return; } LLFS_VLOG(1) << "start_control_block_update():" << " trim=" << observed_trim_pos << "->" << observed_target_trim_pos << " (effective=" << effective_target_trim_pos << ")" - << " flush=" << observed_flush_pos << "->" << this->known_flush_pos_; + << " flush=" << observed_flush_pos << "->" << effective_target_flush_pos; BATT_CHECK_EQ(observed_trim_pos, this->control_block_->trim_pos); BATT_CHECK_EQ(observed_flush_pos, this->control_block_->flush_pos); @@ -662,11 +699,11 @@ void IoRingLogDriver2::start_control_block_update( (this->control_block_->next_commit_i + 1) % this->control_block_->commit_points.size(); } - LLFS_CHECK_SLOT_LE(effective_target_trim_pos, this->known_flush_pos_); - BATT_CHECK_LE(this->known_flush_pos_ - effective_target_trim_pos, this->config_.log_capacity); + LLFS_CHECK_SLOT_LE(effective_target_trim_pos, effective_target_flush_pos); + BATT_CHECK_LE(effective_target_flush_pos - effective_target_trim_pos, this->config_.log_capacity); this->control_block_->trim_pos = effective_target_trim_pos; - this->control_block_->flush_pos = this->known_flush_pos_; + this->control_block_->flush_pos = effective_target_flush_pos; this->control_block_->generation = this->control_block_->generation + 1; this->writing_control_block_ = true; From f1f0ebf81e753044c6a650e6ab676182c8c6f367 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Fri, 19 Jul 2024 11:58:42 -0400 Subject: [PATCH 08/11] Add comments; remove dead code devoted to flush_pos recovery. --- src/llfs/ioring_log_device2.hpp | 21 +--- src/llfs/ioring_log_device2.ipp | 135 +++---------------------- src/llfs/packed_log_control_block2.hpp | 10 +- 3 files changed, 20 insertions(+), 146 deletions(-) diff --git a/src/llfs/ioring_log_device2.hpp b/src/llfs/ioring_log_device2.hpp index bfb7743..a100f28 100644 --- a/src/llfs/ioring_log_device2.hpp +++ b/src/llfs/ioring_log_device2.hpp @@ -149,20 +149,6 @@ class IoRingLogDriver2 */ Status read_log_data(); - /** \brief Returns upper bound of known slot commit points recovered from the control block. - * - * This will either be the highest of the commit points inside the control block, without - * exceeding the recovered flush position, or the recovered trim position, whichever is greater. - */ - slot_offset_type recover_flushed_commit_point() const noexcept; - - /** \brief Starts at the value returned by this->recover_flushed_commit_point() and scans forward - * in the log data, parsing slot headers until we reach a partially flushed slot or run out of - * data. Calls this->reset_flush_pos with the slot upper bound of the highest confirmed slot from - * the scan. - */ - Status recover_flush_pos() noexcept; - /** \brief Forces all slot lower bound pointers to `new_trim_pos`. * * Resets: @@ -363,12 +349,13 @@ class IoRingLogDriver2 */ slot_offset_type unflushed_lower_bound_ = 0; - /** \brief The least upper bound of contiguous flushed data in the log. Updates to this value - * should trigger an update of the control block. + /** \brief The least upper bound of contiguous flushed data in the log. Updates to this value may + * cause this->known_flushed_commit_pos_ to advance. */ slot_offset_type known_flush_pos_ = 0; - /** \brief The highest observed commit pos known to be flushed. + /** \brief The highest observed commit pos known to be flushed. Updates to this value + * should trigger an update of the control block. */ slot_offset_type known_flushed_commit_pos_ = 0; diff --git a/src/llfs/ioring_log_device2.ipp b/src/llfs/ioring_log_device2.ipp index a0811f6..a069edd 100644 --- a/src/llfs/ioring_log_device2.ipp +++ b/src/llfs/ioring_log_device2.ipp @@ -201,114 +201,6 @@ inline Status IoRingLogDriver2::read_log_data() BATT_REQUIRE_OK(this->storage_.read_all(read_begin_offset, upper_part_buffer)); } - return this->recover_flush_pos(); -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -template -slot_offset_type IoRingLogDriver2::recover_flushed_commit_point() const noexcept -{ - const slot_offset_type recovered_trim_pos = this->control_block_->trim_pos; - const slot_offset_type recovered_flush_pos = this->control_block_->flush_pos; - - slot_offset_type slot_offset = recovered_trim_pos; - - std::vector sorted_commit_points(this->control_block_->commit_points.begin(), - this->control_block_->commit_points.end()); - - std::sort(sorted_commit_points.begin(), sorted_commit_points.end(), SlotOffsetOrder{}); - - LLFS_VLOG(1) << BATT_INSPECT_RANGE(sorted_commit_points); - - auto iter = std::upper_bound(sorted_commit_points.begin(), sorted_commit_points.end(), - recovered_flush_pos, SlotOffsetOrder{}); - - if (iter != sorted_commit_points.begin()) { - --iter; - slot_offset = slot_max(slot_offset, *iter); - LLFS_VLOG(1) << " -- using commit point: " << *iter; - } - - LLFS_CHECK_SLOT_LE(slot_offset, recovered_flush_pos); - return slot_offset; -} - -//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - -// -template -Status IoRingLogDriver2::recover_flush_pos() noexcept -{ - const slot_offset_type recovered_flush_pos = this->flush_pos_.get_value(); - - slot_offset_type slot_offset = this->recover_flushed_commit_point(); - - ConstBuffer buffer = - resize_buffer(this->context_.buffer_.get(slot_offset), recovered_flush_pos - slot_offset); - - slot_offset_type confirmed_flush_pos = slot_offset; - - // This should be correct, since commit is called only once per atomic range, and atomic ranges - // are only recoverable if no part of the range (including the begin/end tokens) has been trimmed. - // - bool inside_atomic_range = false; - - for (;;) { - DataReader reader{buffer}; - const usize bytes_available_before = reader.bytes_available(); - Optional slot_body_size = reader.read_varint(); - - if (!slot_body_size) { - // Partially committed slot (couldn't even read a whole varint for the slot header!) Break - // out of the loop. - // - LLFS_VLOG(1) << " -- Incomplete slot header, exiting loop;" << BATT_INSPECT(slot_offset) - << BATT_INSPECT(bytes_available_before); - break; - } - - const usize bytes_available_after = reader.bytes_available(); - const usize slot_header_size = bytes_available_before - bytes_available_after; - const usize slot_size = slot_header_size + *slot_body_size; - - if (slot_size > buffer.size()) { - // Partially committed slot; break out of the loop without updating slot_offset (we're - // done!) - // - LLFS_VLOG(1) << " -- Incomplete slot body, exiting loop;" << BATT_INSPECT(slot_offset) - << BATT_INSPECT(bytes_available_before) << BATT_INSPECT(bytes_available_after) - << BATT_INSPECT(slot_header_size) << BATT_INSPECT(slot_body_size) - << BATT_INSPECT(slot_size); - break; - } - - // Check for control token; this indicates the beginning or end of an atomic slot range. - // - if (*slot_body_size == 0) { - if (slot_header_size == SlotWriter::WriterLock::kBeginAtomicRangeTokenSize) { - inside_atomic_range = true; - } else if (slot_header_size == SlotWriter::WriterLock::kEndAtomicRangeTokenSize) { - inside_atomic_range = false; - } - } - - buffer += slot_size; - slot_offset += slot_size; - - // If inside an atomic slot range, we hold off on updating the confirmed_flush_pos, just in - // case the flushed data is cut off before the end of the atomic range. - // - if (!inside_atomic_range) { - confirmed_flush_pos = slot_offset; - } - } - - LLFS_VLOG(1) << " -- Slot scan complete;" << BATT_INSPECT(slot_offset); - - this->reset_flush_pos(confirmed_flush_pos); - - BATT_CHECK_EQ(recovered_flush_pos, confirmed_flush_pos); - return OkStatus(); } @@ -398,8 +290,13 @@ inline void IoRingLogDriver2::wait_for_slot_offset_change(T observed_v template inline void IoRingLogDriver2::start_flush(CommitPos observed_commit_pos) { + // Unflushed data comes after flushed. + // slot_offset_type flush_upper_bound = this->unflushed_lower_bound_; + // Repeat is for when unflushed data wraps around the end of the ring buffer, back to the + // beginning; in this case we start two writes. + // for (usize repeat = 0; repeat < 2; ++repeat) { //----- --- -- - - - - @@ -449,6 +346,8 @@ inline void IoRingLogDriver2::start_flush(CommitPos observed_commit_po } } + // Align to 512-byte boundaries for direct I/O + // SlotRange aligned_range = this->get_aligned_range(slot_range); // If this flush would overlap with an ongoing one (at the last device page) then trim the @@ -535,7 +434,9 @@ inline void IoRingLogDriver2::start_flush_write(const SlotRange& slot_ ConstBuffer buffer = resize_buffer(this->context_.buffer_.get(aligned_range.lower_bound), aligned_range.size()); - BATT_CHECK_LE(write_offset + (i64)buffer.size(), this->data_end_); + BATT_CHECK_LE(write_offset + (i64)buffer.size(), this->data_end_) + << "Data to flush extends beyond the end of the storage extent; forgot to handle wrap-around " + "case?"; LLFS_VLOG(1) << " -- async_write_some(offset=" << write_offset << ".." << write_offset + buffer.size() << ", size=" << buffer.size() << ")"; @@ -592,6 +493,10 @@ inline void IoRingLogDriver2::handle_flush_write(const SlotRange& slot const auto observed_commit_pos = this->observe(CommitPos{}); + // If is_tail is false, then there was a write initiated *after* this portion of the log. In this + // case, if the write was short, this function needs to initiate writing the remainder of the + // data. + // if (!is_tail) { SlotRange updated_range{ .lower_bound = flushed_range.upper_bound, @@ -687,18 +592,6 @@ void IoRingLogDriver2::start_control_block_update( BATT_CHECK_EQ(observed_trim_pos, this->control_block_->trim_pos); BATT_CHECK_EQ(observed_flush_pos, this->control_block_->flush_pos); - - const slot_offset_type latest_commit_pos = this->observe(CommitPos{}); - auto& next_commit_pos_slot = - this->control_block_->commit_points[this->control_block_->next_commit_i]; - - if (next_commit_pos_slot != latest_commit_pos) { - next_commit_pos_slot = latest_commit_pos; - - this->control_block_->next_commit_i = - (this->control_block_->next_commit_i + 1) % this->control_block_->commit_points.size(); - } - LLFS_CHECK_SLOT_LE(effective_target_trim_pos, effective_target_flush_pos); BATT_CHECK_LE(effective_target_flush_pos - effective_target_trim_pos, this->config_.log_capacity); diff --git a/src/llfs/packed_log_control_block2.hpp b/src/llfs/packed_log_control_block2.hpp index 7d7b721..5bb913c 100644 --- a/src/llfs/packed_log_control_block2.hpp +++ b/src/llfs/packed_log_control_block2.hpp @@ -63,15 +63,9 @@ struct PackedLogControlBlock2 { */ little_i16 data_alignment_log2; - /** \brief The index of the next element of this->commit_points to be overwritten. + /** \brief Padding; reserved for future use. */ - little_u32 next_commit_i; - - /** \brief A rolling history of known commit points; this is updated each time the control block - * is updated. It is used to speed up log recovery, especially in the case where the log device - * was cleanly shut down. - */ - std::array commit_points; + std::array reserved_; }; BATT_STATIC_ASSERT_EQ(sizeof(PackedLogControlBlock2), 512); From 47a16b27458afb2d44fb9d0db94d1df5a9506ef2 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Wed, 24 Jul 2024 18:07:30 -0400 Subject: [PATCH 09/11] CR feedback (everything but write handler changes) --- src/llfs/ioring_log_device2.hpp | 6 ++--- src/llfs/ioring_log_device2.ipp | 12 ++++++---- src/llfs/ioring_log_device2.test.cpp | 14 +++++------ src/llfs/log_device_config2.hpp | 2 +- src/llfs/packed_log_control_block2.hpp | 13 +++++++++-- src/llfs/status_code.cpp | 8 +++---- src/llfs/status_code.hpp | 1 + src/llfs/testing/test_config.cpp | 32 ++++++++++++++++++++++++++ src/llfs/testing/test_config.hpp | 8 +++++++ 9 files changed, 75 insertions(+), 21 deletions(-) diff --git a/src/llfs/ioring_log_device2.hpp b/src/llfs/ioring_log_device2.hpp index a100f28..b164f91 100644 --- a/src/llfs/ioring_log_device2.hpp +++ b/src/llfs/ioring_log_device2.hpp @@ -55,9 +55,9 @@ class IoRingLogDriver2 /** \brief The size (bytes) of each preallocated completion handler memory object. */ - static constexpr usize kHandlerMemorySize = 160; + static constexpr usize kHandlerMemorySizeBytes = 160; - using HandlerMemory = batt::HandlerMemory; + using HandlerMemory = batt::HandlerMemory; using HandlerMemoryStorage = std::aligned_storage_t; @@ -238,7 +238,7 @@ class IoRingLogDriver2 /** \brief Initiates a rewrite of the control block if necessary. * - * The control block must be updated when the target trim pos or unknown flush pos become out of + * The control block must be updated when the target trim pos or known flush pos becomes out of * sync with the last written values. * * Only one pending async write to the control block is allowed at a time. diff --git a/src/llfs/ioring_log_device2.ipp b/src/llfs/ioring_log_device2.ipp index a069edd..f66f9e6 100644 --- a/src/llfs/ioring_log_device2.ipp +++ b/src/llfs/ioring_log_device2.ipp @@ -140,7 +140,7 @@ inline Status IoRingLogDriver2::read_control_block() this->reset_trim_pos(recovered_trim_pos); this->reset_flush_pos(recovered_flush_pos); - this->data_begin_ = this->config_.control_block_offset + this->data_page_size_; + this->data_begin_ = this->config_.control_block_offset + control_block_size; this->data_end_ = this->data_begin_ + p_control_block->data_size; this->control_block_buffer_ = ConstBuffer{ @@ -151,6 +151,10 @@ inline Status IoRingLogDriver2::read_control_block() // TODO [tastolfi 2024-06-11] verify control block values against config where possible. + if (this->control_block_->magic != PackedLogControlBlock2::kMagic) { + return ::llfs::make_status(::llfs::StatusCode::kLogControlBlockBadMagic); + } + return OkStatus(); } @@ -230,8 +234,8 @@ void IoRingLogDriver2::reset_flush_pos(slot_offset_type new_flush_pos) template inline void IoRingLogDriver2::poll() noexcept { - auto observed_commit_pos = this->observe(CommitPos{}); - auto observed_target_trim_pos = this->observe(TargetTrimPos{}); + CommitPos observed_commit_pos = this->observe(CommitPos{}); + TargetTrimPos observed_target_trim_pos = this->observe(TargetTrimPos{}); LLFS_VLOG(1) << "poll()" << BATT_INSPECT(observed_commit_pos) << BATT_INSPECT(observed_target_trim_pos); @@ -491,7 +495,7 @@ inline void IoRingLogDriver2::handle_flush_write(const SlotRange& slot this->update_known_flush_pos(flushed_range); - const auto observed_commit_pos = this->observe(CommitPos{}); + const CommitPos observed_commit_pos = this->observe(CommitPos{}); // If is_tail is false, then there was a write initiated *after* this portion of the log. In this // case, if the write was short, this function needs to initiate writing the remainder of the diff --git a/src/llfs/ioring_log_device2.test.cpp b/src/llfs/ioring_log_device2.test.cpp index 472cd35..bc2b342 100644 --- a/src/llfs/ioring_log_device2.test.cpp +++ b/src/llfs/ioring_log_device2.test.cpp @@ -198,7 +198,7 @@ class IoringLogDevice2SimTest : public ::testing::Test */ bool append_one_slot() noexcept { - const usize payload_size = pick_slot_size(rng); + const usize payload_size = pick_slot_size(this->rng); const usize header_size = llfs::packed_sizeof_varint(payload_size); const usize slot_size = header_size + payload_size; @@ -211,7 +211,7 @@ class IoringLogDevice2SimTest : public ::testing::Test BATT_CHECK_OK(buffer); *buffer = batt::get_or_panic(llfs::pack_varint_to(*buffer, payload_size)); - std::memset(buffer->data(), 'a' + (slot_offset % 26), buffer->size()); + std::memset(buffer->data(), 'a' + (slot_offset % (('z' - 'a') + 1)), buffer->size()); StatusOr end_offset = this->log_writer->commit(slot_size); BATT_CHECK_OK(end_offset); @@ -331,7 +331,7 @@ class IoringLogDevice2SimTest : public ::testing::Test * range, we know for sure which ones still belong on this list. * * By the time this function returns, maybe_trimmed_slots will be empty, since we've either - * removed each slot range, or moved it from maybed_trimmed_slots to appended_slots. + * removed each slot range, or moved it from maybe_trimmed_slots to appended_slots. */ void resolve_maybe_trimmed_slots(const llfs::SlotRange& slot_range) { @@ -371,7 +371,7 @@ class IoringLogDevice2SimTest : public ::testing::Test // Verify the data. // - const char expected_ch = 'a' + (slot.offset.lower_bound % 26); + const char expected_ch = 'a' + (slot.offset.lower_bound % (('z' - 'a') + 1)); for (char actual_ch : slot.body) { EXPECT_EQ(expected_ch, actual_ch); if (actual_ch != expected_ch) { @@ -456,10 +456,12 @@ TEST_F(IoringLogDevice2SimTest, Simulation) const usize kNumThreads = std::thread::hardware_concurrency(); const usize kUpdateInterval = kNumSeeds / 20; + llfs::testing::TestConfig test_config; + LLFS_LOG_INFO() << BATT_INSPECT(kNumSeeds); std::vector threads; - usize start_seed = 0; + usize start_seed = test_config.get_random_seed(); usize seeds_remaining = kNumSeeds; for (usize i = 0; i < kNumThreads; ++i) { usize n_seeds = seeds_remaining / (kNumThreads - i); @@ -480,8 +482,6 @@ TEST_F(IoringLogDevice2SimTest, Simulation) for (std::thread& t : threads) { t.join(); } - - //EXPECT_GT(partial_slot_flush_count, 0); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - diff --git a/src/llfs/log_device_config2.hpp b/src/llfs/log_device_config2.hpp index 689b762..299b7c1 100644 --- a/src/llfs/log_device_config2.hpp +++ b/src/llfs/log_device_config2.hpp @@ -39,7 +39,7 @@ struct PackedLogDeviceConfig2; //+++++++++++-+-+--+----- --- -- - - - - -Status configure_storage_object(StorageFileBuilder::Transaction&, +Status configure_storage_object(StorageFileBuilder::Transaction& txn, FileOffsetPtr p_config, const LogDeviceConfigOptions2& options); diff --git a/src/llfs/packed_log_control_block2.hpp b/src/llfs/packed_log_control_block2.hpp index 5bb913c..67c0186 100644 --- a/src/llfs/packed_log_control_block2.hpp +++ b/src/llfs/packed_log_control_block2.hpp @@ -22,6 +22,13 @@ namespace llfs { struct PackedLogControlBlock2 { + /** \brief The control block structure should take up exactly 512 bytes, since that is the + * smallest atomic block size for the kinds of devices we care about. + */ + static constexpr usize kSize = 512; + + /** \brief Used to sanity check instances of this packed structure read from media. + */ static constexpr u64 kMagic = 0x128095f84cfba8b0ull; /** \brief Must always be set to PackedLogControlBlock2::kMagic. @@ -65,10 +72,12 @@ struct PackedLogControlBlock2 { /** \brief Padding; reserved for future use. */ - std::array reserved_; + std::array reserved_; }; -BATT_STATIC_ASSERT_EQ(sizeof(PackedLogControlBlock2), 512); +// Verify that the struct is the desired size. +// +BATT_STATIC_ASSERT_EQ(sizeof(PackedLogControlBlock2), PackedLogControlBlock2::kSize); } //namespace llfs diff --git a/src/llfs/status_code.cpp b/src/llfs/status_code.cpp index 8b9d539..b0b73d7 100644 --- a/src/llfs/status_code.cpp +++ b/src/llfs/status_code.cpp @@ -13,10 +13,7 @@ namespace llfs { -#define CODE_WITH_MSG_(code, msg) \ - { \ - code, msg " (" #code ")" \ - } +#define CODE_WITH_MSG_(code, msg) {code, msg " (" #code ")"} //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // @@ -136,6 +133,9 @@ bool initialize_status_codes() CODE_WITH_MSG_( StatusCode::kIoRingShutDown, "The operation could not be completed because the IoRing was shut down"), // 64, + CODE_WITH_MSG_(StatusCode::kLogControlBlockBadMagic, + "Log device control block magic number is not correct (is this really a log " + "device?)"), // 65, }); return initialized; } diff --git a/src/llfs/status_code.hpp b/src/llfs/status_code.hpp index 57ad85f..021ddd4 100644 --- a/src/llfs/status_code.hpp +++ b/src/llfs/status_code.hpp @@ -80,6 +80,7 @@ enum struct StatusCode { kPutViewUnknownLayoutId = 62, kPageCacheSlotNotInitialized = 63, kIoRingShutDown = 64, + kLogControlBlockBadMagic = 65, }; bool initialize_status_codes(); diff --git a/src/llfs/testing/test_config.cpp b/src/llfs/testing/test_config.cpp index 3f12c2f..f928ea0 100644 --- a/src/llfs/testing/test_config.cpp +++ b/src/llfs/testing/test_config.cpp @@ -8,12 +8,35 @@ #include // +#include +#include #include namespace llfs { namespace testing { +namespace { + +usize get_random_seed_from_env() +{ + static const usize seed = [] { + const char* varname = "LLFS_TEST_RANDOM_SEED"; + + Optional value = batt::getenv_as(varname); + if (!value) { + LLFS_LOG_INFO() << varname << " not defined; using default value 0"; + return usize{0}; + } + LLFS_LOG_INFO() << varname << " == " << *value; + return *value; + }(); + + return seed; +} + +} //namespace + //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - // TestConfig::TestConfig() noexcept @@ -25,6 +48,8 @@ TestConfig::TestConfig() noexcept this->low_level_log_device_sim_ = // batt::getenv_as("LLFS_LOW_LEVEL_LOG_DEVICE_SIM") // .value_or(1); + + this->random_seed_ = get_random_seed_from_env(); } //==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - @@ -82,5 +107,12 @@ std::filesystem::path TestConfig::data_file_path(std::string_view file_rel_path) return this->project_dir() / "testdata" / file_rel_path; } +//==#==========+==+=+=++=+++++++++++-+-+--+----- --- -- - - - - +// +usize TestConfig::get_random_seed() noexcept +{ + return this->random_seed_; +} + } //namespace testing } //namespace llfs diff --git a/src/llfs/testing/test_config.hpp b/src/llfs/testing/test_config.hpp index 24a9bcc..4e987fc 100644 --- a/src/llfs/testing/test_config.hpp +++ b/src/llfs/testing/test_config.hpp @@ -56,6 +56,12 @@ class TestConfig */ std::filesystem::path data_file_path(std::string_view file_rel_path) noexcept; + /** \brief Returns initial random seed for tests that use pseudo-random number generators. + * + * Tries to read from env var `LLFS_TEST_RANDOM_SEED`; defaults to 0. + */ + usize get_random_seed() noexcept; + //+++++++++++-+-+--+----- --- -- - - - - private: bool extra_testing_; @@ -63,6 +69,8 @@ class TestConfig bool low_level_log_device_sim_; std::filesystem::path project_dir_; + + usize random_seed_; }; } //namespace testing From d5d0be444a9ca6faec08c5b60a96533ace4181b0 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Wed, 24 Jul 2024 20:39:33 -0400 Subject: [PATCH 10/11] Rest of CR changes. --- src/llfs/ioring_log_device2.ipp | 64 +++++++++++++++++++++------------ 1 file changed, 42 insertions(+), 22 deletions(-) diff --git a/src/llfs/ioring_log_device2.ipp b/src/llfs/ioring_log_device2.ipp index f66f9e6..fc0b371 100644 --- a/src/llfs/ioring_log_device2.ipp +++ b/src/llfs/ioring_log_device2.ipp @@ -467,48 +467,68 @@ inline void IoRingLogDriver2::handle_flush_write(const SlotRange& slot { LLFS_VLOG(1) << "handle_flush_result(result=" << result << ")" << BATT_INSPECT(slot_range); - const usize bytes_written = result.ok() ? *result : 0; - - SlotRange aligned_tail = this->get_aligned_tail(aligned_range); - - SlotRange flushed_range{ - .lower_bound = slot_max(slot_range.lower_bound, aligned_range.lower_bound), - .upper_bound = slot_min(aligned_range.lower_bound + bytes_written, slot_range.upper_bound), - }; - - const bool is_tail = (this->flush_tail_ && *this->flush_tail_ == aligned_tail); + // Determine whether this write is the current flush_tail_, and clear it if so. + // + const bool is_tail = [&] { + SlotRange aligned_tail = this->get_aligned_tail(aligned_range); + return (this->flush_tail_ && *this->flush_tail_ == aligned_tail); + }(); LLFS_DVLOG(1) << BATT_INSPECT(is_tail); + if (is_tail) { this->flush_tail_ = None; - this->unflushed_lower_bound_ = flushed_range.upper_bound; } - LLFS_DVLOG(1) << BATT_INSPECT(flushed_range); - + // Write errors are fatal. + // if (!result.ok()) { LLFS_VLOG(1) << "(handle_flush_write) error: " << result.status(); this->handle_write_error(result.status()); return; } + // Calculate the (non-aligned) offset range of flushed data. + // + const usize bytes_written = *result; + + SlotRange flushed_range{ + // Use the logical (non-aligned) lower bound. + // + .lower_bound = slot_max(slot_range.lower_bound, aligned_range.lower_bound), + + // Take minimum here to account for short writes. + // + .upper_bound = slot_min(aligned_range.lower_bound + bytes_written, slot_range.upper_bound), + }; BATT_CHECK(!flushed_range.empty()); - this->update_known_flush_pos(flushed_range); + LLFS_DVLOG(1) << BATT_INSPECT(flushed_range); - const CommitPos observed_commit_pos = this->observe(CommitPos{}); + // Update this->known_flush_pos_ and this->known_flushed_commit_pos_ to reflect the write. + // + this->update_known_flush_pos(flushed_range); - // If is_tail is false, then there was a write initiated *after* this portion of the log. In this - // case, if the write was short, this function needs to initiate writing the remainder of the - // data. + // If there was no write after this one and the write was short, then we must adjust + // this->unflushed_lower_bound_ down to the actual end of flushed data so that when we call + // poll(), the remainder of `slot_range` is written. // - if (!is_tail) { - SlotRange updated_range{ + if (is_tail) { + this->unflushed_lower_bound_ = flushed_range.upper_bound; + + } else { + // If is_tail is false, then there was a write initiated *after* this portion of the log. In + // this case, if the write was short, this function needs to initiate writing the remainder of + // the data. + // + const CommitPos observed_commit_pos = this->observe(CommitPos{}); + + SlotRange unflushed_remainder{ .lower_bound = flushed_range.upper_bound, .upper_bound = slot_min(aligned_range.upper_bound, observed_commit_pos), }; - if (!updated_range.empty()) { - this->start_flush_write(updated_range, this->get_aligned_range(updated_range)); + if (!unflushed_remainder.empty()) { + this->start_flush_write(unflushed_remainder, this->get_aligned_range(unflushed_remainder)); } } From ec46609d8477d795fff27bb0a6aa58dff3190500 Mon Sep 17 00:00:00 2001 From: Anthony Astolfi Date: Thu, 25 Jul 2024 10:39:15 -0400 Subject: [PATCH 11/11] Resolve last CR thread by fixing doxygen comments. --- src/llfs/ioring_log_device2.hpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/llfs/ioring_log_device2.hpp b/src/llfs/ioring_log_device2.hpp index b164f91..e8afd44 100644 --- a/src/llfs/ioring_log_device2.hpp +++ b/src/llfs/ioring_log_device2.hpp @@ -423,10 +423,12 @@ class IoRingLogDriver2 }; //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+--------------- -/** \brief An emphemeral LogDevice that stores data in memory only. +/** \brief A LogDevice impl that flushes data on durable media asynchronously using an io_uring-like + * I/O layer. * - * The commit pos and flush pos are always in sync, so there is no difference between - * LogReadMode::kSpeculative and LogReadMode::kDurable for this log device type. + * This template is parameterized on the StorageT type, which implements the low-level async I/O + * methods and the async event threading model. IoRingLogDevice2 is type alias that instantiates + * this template with the default storage type, DefaultIoRingLogDeviceStorage. */ template class BasicIoRingLogDevice2 @@ -448,6 +450,8 @@ class BasicIoRingLogDevice2 } }; +/** \brief A fast, durable LogDevice implementation. + */ using IoRingLogDevice2 = BasicIoRingLogDevice2; //=#=#==#==#===============+=+=+=+=++=++++++++++++++-++-+--+-+----+---------------