add_library(crimson-seastore
+ cached_extent.cc
seastore_types.cc
segment_manager/ephemeral.cc
segment_manager.cc
journal.cc
+ cache.cc
+ root_block.cc
)
target_link_libraries(crimson-seastore
crimson)
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "crimson/os/seastore/cache.h"
+#include "crimson/common/log.h"
+
+namespace {
+ seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_filestore);
+ }
+}
+
+namespace crimson::os::seastore {
+
+Cache::~Cache()
+{
+ retire_extent(root);
+ root.reset();
+ for (auto i = dirty.begin(); i != dirty.end(); ) {
+ auto ptr = &*i;
+ dirty.erase(i++);
+ intrusive_ptr_release(ptr);
+ }
+ for (auto &i: extents) {
+ logger().error("~Cache: extent {} still alive", i);
+ }
+ ceph_assert(extents.empty());
+}
+
+void Cache::add_extent(CachedExtentRef ref)
+{
+ assert(ref->is_valid());
+ extents.insert(*ref);
+
+ ceph_assert(!ref->primary_ref_list_hook.is_linked());
+ if (ref->is_dirty()) {
+ intrusive_ptr_add_ref(&*ref);
+ dirty.push_back(*ref);
+ }
+ logger().debug("add_extent: {}", *ref);
+}
+
+void Cache::retire_extent(CachedExtentRef ref)
+{
+ logger().debug("retire_extent: {}", *ref);
+ assert(ref->is_valid());
+ extents.erase(*ref);
+
+ if (ref->is_dirty()) {
+ ceph_assert(ref->primary_ref_list_hook.is_linked());
+ dirty.erase(dirty.s_iterator_to(*ref));
+ intrusive_ptr_release(&*ref);
+ } else {
+ ceph_assert(!ref->primary_ref_list_hook.is_linked());
+ }
+}
+
+CachedExtentRef Cache::duplicate_for_write(
+ Transaction &t,
+ CachedExtentRef i) {
+ if (i->is_pending())
+ return i;
+
+ auto ret = i->duplicate_for_write();
+ ret->version++;
+ ret->state = CachedExtent::extent_state_t::MUTATION_PENDING;
+
+ if (ret->get_type() == extent_types_t::ROOT) {
+ t.root = ret->cast<RootBlock>();
+ }
+
+ t.add_to_retired_set(i);
+ t.add_mutated_extent(ret);
+
+ return ret;
+}
+
+std::optional<record_t> Cache::try_construct_record(Transaction &t)
+{
+ // First, validate read set
+ for (auto &i: t.read_set) {
+ if (i->state == CachedExtent::extent_state_t::INVALID)
+ return std::nullopt;
+ }
+
+ record_t record;
+
+ // Transaction is now a go, set up in-memory cache state
+ // invalidate now invalid blocks
+ for (auto &i: t.retired_set) {
+ logger().debug("try_construct_record: retiring {}", *i);
+ ceph_assert(!i->is_pending());
+ ceph_assert(i->is_valid());
+ retire_extent(i);
+ i->state = CachedExtent::extent_state_t::INVALID;
+ }
+
+ t.write_set.clear();
+
+ // Add new copy of mutated blocks, set_io_wait to block until written
+ record.deltas.reserve(t.mutated_block_list.size());
+ for (auto &i: t.mutated_block_list) {
+ logger().debug("try_construct_record: mutating {}", *i);
+ add_extent(i);
+ i->prepare_write();
+ i->set_io_wait();
+ record.deltas.push_back(
+ delta_info_t{
+ i->get_type(),
+ i->get_paddr(),
+ (segment_off_t)i->get_length(),
+ i->get_version(),
+ i->get_delta()
+ });
+ }
+
+ record.extents.reserve(t.fresh_block_list.size());
+ for (auto &i: t.fresh_block_list) {
+ logger().debug("try_construct_record: fresh block {}", *i);
+ bufferlist bl;
+ i->prepare_write();
+ bl.append(i->get_bptr());
+ if (i->get_type() == extent_types_t::ROOT) {
+ record.deltas.push_back(
+ delta_info_t{
+ extent_types_t::ROOT_LOCATION,
+ i->get_paddr(),
+ 0,
+ 0,
+ bufferlist()
+ });
+ }
+ record.extents.push_back(extent_t{std::move(bl)});
+ }
+
+ t.read_set.clear();
+ return std::make_optional<record_t>(std::move(record));
+}
+
+void Cache::complete_commit(
+ Transaction &t,
+ paddr_t final_block_start)
+{
+ if (t.root)
+ root = t.root;
+
+ paddr_t cur = final_block_start;
+ for (auto &i: t.fresh_block_list) {
+ i->set_paddr(cur);
+ cur.offset += i->get_length();
+ i->state = CachedExtent::extent_state_t::CLEAN;
+ logger().debug("complete_commit: fresh {}", *i);
+ i->on_initial_write();
+ add_extent(i);
+ }
+
+ // Add new copy of mutated blocks, set_io_wait to block until written
+ for (auto &i: t.mutated_block_list) {
+ i->state = CachedExtent::extent_state_t::DIRTY;
+ logger().debug("complete_commit: mutated {}", *i);
+ i->on_delta_write(final_block_start);
+ }
+
+ for (auto &i: t.mutated_block_list) {
+ i->complete_io();
+ }
+}
+
+Cache::mkfs_ertr::future<> Cache::mkfs(Transaction &t)
+{
+ t.root = alloc_new_extent<RootBlock>(t, RootBlock::SIZE);
+ return mkfs_ertr::now();
+}
+
+Cache::close_ertr::future<> Cache::close()
+{
+ return close_ertr::now();
+}
+
+Cache::replay_delta_ret
+Cache::replay_delta(paddr_t record_base, const delta_info_t &delta)
+{
+ if (delta.type == extent_types_t::ROOT_LOCATION) {
+ auto root_location = delta.paddr.is_relative()
+ ? record_base.add_relative(delta.paddr)
+ : delta.paddr;
+ logger().debug("replay_delta: found root addr {}", root_location);
+ return get_extent<RootBlock>(
+ root_location,
+ RootBlock::SIZE
+ ).safe_then([this, root_location](auto ref) {
+ logger().debug("replay_delta: finished reading root at {}", root_location);
+ root = ref;
+ return root->complete_load();
+ }).safe_then([this, root_location] {
+ logger().debug("replay_delta: finished loading root at {}", root_location);
+ return replay_delta_ret(replay_delta_ertr::ready_future_marker{});
+ });
+ }
+ // TODO
+ return replay_delta_ret(replay_delta_ertr::ready_future_marker{});
+}
+
+Cache::get_root_ret Cache::get_root(Transaction &t)
+{
+ if (t.root) {
+ return get_root_ret(
+ get_root_ertr::ready_future_marker{},
+ t.root);
+ } else {
+ auto ret = root;
+ return ret->wait_io().then([this, &t, ret] {
+ return get_root_ret(
+ get_root_ertr::ready_future_marker{},
+ ret);
+ });
+ }
+}
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <iostream>
+
+#include "seastar/core/shared_future.hh"
+
+#include "include/buffer.h"
+#include "crimson/os/seastore/seastore_types.h"
+#include "crimson/os/seastore/segment_manager.h"
+#include "crimson/common/errorator.h"
+#include "crimson/os/seastore/cached_extent.h"
+#include "crimson/os/seastore/root_block.h"
+
+namespace crimson::os::seastore {
+
+/**
+ * Transaction
+ *
+ * Representation of in-progress mutation. Used exclusively through Cache methods.
+ */
+class Transaction {
+ friend class Cache;
+
+ RootBlockRef root; ///< ref to root if mutated by transaction
+
+ segment_off_t offset = 0; ///< relative offset of next block
+
+ pextent_set_t read_set; ///< set of extents read by paddr
+ ExtentIndex write_set; ///< set of extents written by paddr
+
+ std::list<CachedExtentRef> fresh_block_list; ///< list of fresh blocks
+ std::list<CachedExtentRef> mutated_block_list; ///< list of mutated blocks
+
+ pextent_set_t retired_set; ///< list of extents mutated by this transaction
+
+ CachedExtentRef get_extent(paddr_t addr) {
+ if (auto iter = write_set.find_offset(addr);
+ iter != write_set.end()) {
+ return CachedExtentRef(&*iter);
+ } else if (
+ auto iter = read_set.find(addr);
+ iter != read_set.end()) {
+ return *iter;
+ } else {
+ return CachedExtentRef();
+ }
+ }
+
+ void add_to_retired_set(CachedExtentRef ref) {
+ ceph_assert(retired_set.count(ref->get_paddr()) == 0);
+ retired_set.insert(ref);
+ }
+
+ void add_to_read_set(CachedExtentRef ref) {
+ ceph_assert(read_set.count(ref) == 0);
+ read_set.insert(ref);
+ }
+
+ void add_fresh_extent(CachedExtentRef ref) {
+ fresh_block_list.push_back(ref);
+ ref->set_paddr(make_relative_paddr(offset));
+ offset += ref->get_length();
+ write_set.insert(*ref);
+ }
+
+ void add_mutated_extent(CachedExtentRef ref) {
+ mutated_block_list.push_back(ref);
+ write_set.insert(*ref);
+ }
+};
+using TransactionRef = std::unique_ptr<Transaction>;
+
+/**
+ * Cache
+ *
+ * This component is responsible for buffer management, including
+ * transaction lifecycle.
+ *
+ * Seastore transactions are expressed as an atomic combination of
+ * 1) newly written blocks
+ * 2) logical mutations to existing physical blocks
+ *
+ * See record_t
+ *
+ * As such, any transaction has 3 components:
+ * 1) read_set: references to extents read during the transaction
+ * See Transaction::read_set
+ * 2) write_set: references to extents to be written as:
+ * a) new physical blocks, see Transaction::fresh_block_list
+ * b) mutations to existing physical blocks,
+ * see Transaction::mutated_block_list
+ * 3) retired_set: extent refs to be retired either due to 2b or
+ * due to releasing the extent generally.
+
+ * In the case of 2b, the CachedExtent will have been copied into
+ * a fresh CachedExtentRef such that the source extent ref is present
+ * in the read set and the newly allocated extent is present in the
+ * write_set.
+ *
+ * A transaction has 3 phases:
+ * 1) construction: user calls Cache::get_transaction() and populates
+ * the returned transaction by calling Cache methods
+ * 2) submission: user calls Cache::try_start_transaction(). If
+ * succcessful, the user may construct a record and submit the
+ * transaction to the journal.
+ * 3) completion: once the transaction is durable, the user must call
+ * Cache::complete_transaction() with the block offset to complete
+ * the transaction.
+ *
+ * Internally, in phase 1, the fields in Transaction are filled in.
+ * - reads may block if the referenced extent is being written
+ * - once a read obtains a particular CachedExtentRef for a paddr_t,
+ * it'll always get the same one until overwritten
+ * - once a paddr_t is overwritten or written, subsequent reads of
+ * that addr will get the new ref
+ *
+ * In phase 2, if all extents in the read set are valid (not expired),
+ * we can commit (otherwise, we fail and the user must retry).
+ * - Expire all extents in the retired_set (they must all be valid)
+ * - Remove all extents in the retired_set from Cache::extents
+ * - Mark all extents in the write_set wait_io(), add promises to
+ * transaction
+ * - Merge Transaction::write_set into Cache::extents
+ *
+ * After phase 2, the user will submit the record to the journal.
+ * Once complete, we perform phase 3:
+ * - For each CachedExtent in block_list, call
+ * CachedExtent::complete_initial_write(paddr_t) with the block's
+ * final offset (inferred from the extent's position in the block_list
+ * and extent lengths).
+ * - For each block in mutation_list, call
+ * CachedExtent::delta_written(paddr_t) with the address of the start
+ * of the record
+ * - Complete all promises with the final record start paddr_t
+ */
+class Cache {
+public:
+ Cache(SegmentManager &segment_manager) : segment_manager(segment_manager) {}
+ ~Cache();
+
+ TransactionRef get_transaction() {
+ return std::make_unique<Transaction>();
+ }
+
+ /// Declare ref retired in t
+ void retire_extent(Transaction &t, CachedExtentRef ref) {
+ t.add_to_retired_set(ref);
+ }
+
+ /**
+ * get_root
+ *
+ * returns ref to current root or t.root if modified in t
+ */
+ using get_root_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error>;
+ using get_root_ret = get_root_ertr::future<RootBlockRef>;
+ get_root_ret get_root(Transaction &t);
+
+ /**
+ * get_extent
+ *
+ * returns ref to extent at offset~length of type T either from
+ * - extent_set if already in cache
+ * - disk
+ */
+ using get_extent_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error>;
+ template <typename T>
+ get_extent_ertr::future<TCachedExtentRef<T>> get_extent(
+ paddr_t offset, ///< [in] starting addr
+ segment_off_t length ///< [in] length
+ ) {
+ if (auto iter = extents.find_offset(offset);
+ iter != extents.end()) {
+ auto ret = TCachedExtentRef<T>(static_cast<T*>(&*iter));
+ return ret->wait_io().then([ret=std::move(ret)]() mutable {
+ return get_extent_ertr::make_ready_future<TCachedExtentRef<T>>(
+ std::move(ret));
+ });
+ } else {
+ auto ref = CachedExtent::make_cached_extent_ref<T>(
+ alloc_cache_buf(length));
+ ref->set_io_wait();
+ ref->set_paddr(offset);
+ ref->state = CachedExtent::extent_state_t::CLEAN;
+ return segment_manager.read(
+ offset,
+ length,
+ ref->get_bptr()).safe_then(
+ [this, ref=std::move(ref)]() mutable {
+ ref->complete_io();
+ return get_extent_ertr::make_ready_future<TCachedExtentRef<T>>(
+ std::move(ref));
+ },
+ get_extent_ertr::pass_further{},
+ crimson::ct_error::discard_all{});
+ }
+ }
+
+ /**
+ * get_extent
+ *
+ * returns ref to extent at offset~length of type T either from
+ * - t if modified by t
+ * - extent_set if already in cache
+ * - disk
+ */
+ template <typename T>
+ get_extent_ertr::future<TCachedExtentRef<T>> get_extent(
+ Transaction &t, ///< [in,out] current transaction
+ paddr_t offset, ///< [in] starting addr
+ segment_off_t length ///< [in] length
+ ) {
+ if (auto i = t.get_extent(offset)) {
+ return get_extent_ertr::make_ready_future<TCachedExtentRef<T>>(
+ TCachedExtentRef<T>(static_cast<T*>(&*i)));
+ } else {
+ return get_extent<T>(offset, length).safe_then(
+ [this, &t](auto ref) mutable {
+ t.add_to_read_set(ref);
+ return get_extent_ertr::make_ready_future<TCachedExtentRef<T>>(std::move(ref));
+ });
+ }
+ }
+
+ /**
+ * get_extents
+ *
+ * returns refs to extents in extents from:
+ * - t if modified by t
+ * - extent_set if already in cache
+ * - disk
+ */
+ template<typename T>
+ get_extent_ertr::future<t_pextent_list_t<T>> get_extents(
+ Transaction &t, ///< [in, out] current transaction
+ paddr_list_t &&extents ///< [in] extent list for lookup
+ ) {
+ auto retref = std::make_unique<t_pextent_list_t<T>>();
+ auto &ret = *retref;
+ auto ext = std::make_unique<paddr_list_t>(std::move(extents));
+ return crimson::do_for_each(
+ ext->begin(),
+ ext->end(),
+ [this, &t, &ret](auto &p) {
+ auto &[offset, len] = p;
+ return get_extent(t, offset, len).safe_then([&ret](auto cext) {
+ ret.push_back(std::move(cext));
+ });
+ }).safe_then([retref=std::move(retref), ext=std::move(ext)]() mutable {
+ return get_extent_ertr::make_ready_future<t_pextent_list_t<T>>(
+ std::move(*retref));
+ });
+ }
+
+ /**
+ * alloc_new_extent
+ *
+ * Allocates a fresh extent. addr will be relative until commit.
+ */
+ template <typename T>
+ TCachedExtentRef<T> alloc_new_extent(
+ Transaction &t, ///< [in, out] current transaction
+ segment_off_t length ///< [in] length
+ ) {
+ auto ret = CachedExtent::make_cached_extent_ref<T>(
+ alloc_cache_buf(length));
+ t.add_fresh_extent(ret);
+ ret->state = CachedExtent::extent_state_t::INITIAL_WRITE_PENDING;
+ return ret;
+ }
+
+ /**
+ * Allocates mutable buffer from extent_set on offset~len
+ *
+ * TODO: Note, currently all implementations literally copy the
+ * buffer. This needn't be true, CachedExtent implementations could
+ * choose to refer to the same buffer unmodified until commit and just
+ * buffer the mutations in an ancillary data structure.
+ *
+ * @param current transaction
+ * @param extent to duplicate
+ * @return mutable extent
+ */
+ CachedExtentRef duplicate_for_write(
+ Transaction &t, ///< [in, out] current transaction
+ CachedExtentRef i ///< [in] ref to existing extent
+ );
+
+ /**
+ * try_construct_record
+ *
+ * First checks for conflicts. If a racing write has mutated/retired
+ * an extent mutated by this transaction, nullopt will be returned.
+ *
+ * Otherwise, a record will be returned valid for use with Journal.
+ */
+ std::optional<record_t> try_construct_record(
+ Transaction &t ///< [in, out] current transaction
+ );
+
+ /**
+ * complete_commit
+ *
+ * Must be called upon completion of write. Releases blocks on mutating
+ * extents, fills in addresses, and calls relevant callbacks on fresh
+ * and mutated exents.
+ */
+ void complete_commit(
+ Transaction &t, ///< [in, out] current transaction
+ paddr_t final_block_start ///< [in] offset of initial block
+ );
+
+ /**
+ * mkfs
+ *
+ * Alloc initial root node and add to t. The intention is for other
+ * components to use t to adjust the resulting root ref prior to commit.
+ */
+ using mkfs_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error>;
+ mkfs_ertr::future<> mkfs(Transaction &t);
+
+ /**
+ * close
+ *
+ * TODO: currently a noop -- probably should be used to flush dirty blocks
+ */
+ using close_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error>;
+ close_ertr::future<> close();
+
+ /**
+ * replay_delta
+ *
+ * Intended for use in Journal::delta. For each delta, should decode delta,
+ * read relevant block from disk or cache (using correct type), and call
+ * CachedExtent::apply_delta marking the extent dirty.
+ *
+ * TODO: currently only handles the ROOT_LOCATION delta.
+ */
+ using replay_delta_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error>;
+ using replay_delta_ret = replay_delta_ertr::future<>;
+ replay_delta_ret replay_delta(paddr_t record_base, const delta_info_t &delta);
+
+ /**
+ * print
+ *
+ * Dump summary of contents (TODO)
+ */
+ std::ostream &print(
+ std::ostream &out) const {
+ return out;
+ }
+
+private:
+ SegmentManager &segment_manager; ///< ref to segment_manager
+ RootBlockRef root; ///< ref to current root
+ ExtentIndex extents; ///< set of live extents
+ CachedExtent::list dirty; ///< holds refs to dirty extents
+
+ /// alloc buffer for cached extent
+ bufferptr alloc_cache_buf(size_t size) {
+ // TODO: memory pooling etc
+ auto bp = ceph::bufferptr(size);
+ bp.zero();
+ return bp;
+ }
+
+ /// Add extent to extents handling dirty and refcounting
+ void add_extent(CachedExtentRef ref);
+
+ /// Remove extent from extents handling dirty and refcounting
+ void retire_extent(CachedExtentRef ref);
+};
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "crimson/os/seastore/cached_extent.h"
+
+#include "crimson/common/log.h"
+
+namespace {
+ [[maybe_unused]] seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_filestore);
+ }
+}
+
+namespace crimson::os::seastore {
+
+#ifdef DEBUG_CACHED_EXTENT_REF
+
+void intrusive_ptr_add_ref(CachedExtent *ptr)
+{
+ intrusive_ptr_add_ref(
+ static_cast<boost::intrusive_ref_counter<
+ CachedExtent,
+ boost::thread_unsafe_counter>*>(ptr));
+ logger().debug("intrusive_ptr_add_ref: {}", *ptr);
+}
+
+void intrusive_ptr_release(CachedExtent *ptr)
+{
+ logger().debug("intrusive_ptr_release: {}", *ptr);
+ intrusive_ptr_release(
+ static_cast<boost::intrusive_ref_counter<
+ CachedExtent,
+ boost::thread_unsafe_counter>*>(ptr));
+}
+
+#endif
+
+std::ostream &operator<<(std::ostream &out, CachedExtent::extent_state_t state)
+{
+ switch (state) {
+ case CachedExtent::extent_state_t::INITIAL_WRITE_PENDING:
+ return out << "INITIAL_WRITE_PENDING";
+ case CachedExtent::extent_state_t::MUTATION_PENDING:
+ return out << "MUTATION_PENDING";
+ case CachedExtent::extent_state_t::CLEAN:
+ return out << "CLEAN";
+ case CachedExtent::extent_state_t::DIRTY:
+ return out << "DIRTY";
+ case CachedExtent::extent_state_t::INVALID:
+ return out << "INVALID";
+ default:
+ return out << "UNKNOWN";
+ }
+}
+
+std::ostream &operator<<(std::ostream &out, const CachedExtent &ext)
+{
+ return ext.print(out);
+}
+
+CachedExtent::~CachedExtent()
+{
+ if (parent_index) {
+ parent_index->erase(*this);
+ }
+}
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <iostream>
+
+#include <boost/intrusive/list.hpp>
+#include <boost/intrusive_ptr.hpp>
+#include <boost/smart_ptr/intrusive_ref_counter.hpp>
+
+#include "seastar/core/shared_future.hh"
+
+#include "include/buffer.h"
+#include "crimson/common/errorator.h"
+#include "crimson/os/seastore/seastore_types.h"
+
+namespace crimson::os::seastore {
+
+class CachedExtent;
+using CachedExtentRef = boost::intrusive_ptr<CachedExtent>;
+
+// #define DEBUG_CACHED_EXTENT_REF
+#ifdef DEBUG_CACHED_EXTENT_REF
+
+void intrusive_ptr_add_ref(CachedExtent *);
+void intrusive_ptr_release(CachedExtent *);
+
+#endif
+
+template <typename T>
+using TCachedExtentRef = boost::intrusive_ptr<T>;
+
+/**
+ * CachedExtent
+ */
+class ExtentIndex;
+class CachedExtent : public boost::intrusive_ref_counter<
+ CachedExtent, boost::thread_unsafe_counter> {
+ enum class extent_state_t : uint8_t {
+ INITIAL_WRITE_PENDING, // In Transaction::write_set and fresh_block_list
+ MUTATION_PENDING, // In Transaction::write_set and mutated_block_list
+ CLEAN, // In Cache::extent_index, Transaction::read_set
+ // during write, contents match disk, version == 0
+ DIRTY, // Same as CLEAN, but contents do not match disk,
+ // version > 0
+ INVALID // Part of no ExtentIndex set
+ } state = extent_state_t::INVALID;
+ friend std::ostream &operator<<(std::ostream &, extent_state_t);
+
+public:
+ /**
+ * duplicate_for_write
+ *
+ * Implementation should return a fresh CachedExtentRef
+ * which represents a copy of *this until on_delta_write()
+ * is complete, at which point the user may assume *this
+ * will be in state INVALID. As such, the implementation
+ * may involve a copy of get_bptr(), or an ancillary
+ * structure which defers updating the actual buffer until
+ * on_delta_write().
+ */
+ virtual CachedExtentRef duplicate_for_write() = 0;
+
+ /**
+ * prepare_write
+ *
+ * Called prior to reading buffer.
+ * Implemenation may use this callback to fully write out
+ * updates to the buffer.
+ */
+ virtual void prepare_write() {}
+
+ /**
+ * on_initial_write
+ *
+ * Called after commit of extent. State will be CLEAN.
+ * Implentation may use this call to fixup the buffer
+ * with the newly available absolute get_paddr().
+ */
+ virtual void on_initial_write() {}
+
+ /**
+ * on_delta_write
+ *
+ * Called after commit of delta. State will be DIRTY.
+ * Implentation may use this call to fixup any relative
+ * references in the the buffer with the passed
+ * record_block_offset record location.
+ */
+ virtual void on_delta_write(paddr_t record_block_offset) {}
+
+ /**
+ * get_type
+ *
+ * Returns concrete type.
+ */
+ virtual extent_types_t get_type() const = 0;
+
+ friend std::ostream &operator<<(std::ostream &, extent_state_t);
+ std::ostream &print(std::ostream &out) const {
+ return out << "CachedExtent(addr=" << this
+ << ", type=" << get_type()
+ << ", version=" << version
+ << ", paddr=" << get_paddr()
+ << ", state=" << state
+ << ", refcount=" << use_count()
+ << ")";
+ }
+
+ /**
+ * get_delta
+ *
+ * Must return a valid delta usable in apply_delta() in submit_transaction
+ * if state == MUTATION_PENDING.
+ */
+ virtual ceph::bufferlist get_delta() = 0;
+
+ /**
+ * bl is a delta obtained previously from get_delta. The versions will
+ * match. Implementation should mutate buffer based on bl. base matches
+ * the address passed on_delta_write.
+ */
+ virtual void apply_delta(paddr_t base, ceph::bufferlist &bl) = 0;
+
+ /**
+ * Called on dirty CachedExtent implementation after replay.
+ * Implementation should perform any reads/in-memory-setup
+ * necessary. (for instance, the lba implementation will use this
+ * to load in lba_manager blocks)
+ */
+ using complete_load_ertr = crimson::errorator<
+ crimson::ct_error::input_output_error>;
+ virtual complete_load_ertr::future<> complete_load() {
+ return complete_load_ertr::now();
+ }
+
+ /**
+ * cast
+ *
+ * Returns a TCachedExtentRef of the specified type.
+ * TODO: add dynamic check that the requested type is actually correct.
+ */
+ template <typename T>
+ TCachedExtentRef<T> cast() {
+ return TCachedExtentRef<T>(static_cast<T*>(this));
+ }
+
+ /// Returns true if extent is part of an open transaction
+ bool is_pending() const {
+ return state == extent_state_t::INITIAL_WRITE_PENDING ||
+ state == extent_state_t::MUTATION_PENDING;
+ }
+
+ /// Returns true if extent has a pending delta
+ bool is_mutation_pending() const {
+ return state == extent_state_t::MUTATION_PENDING;
+ }
+
+ /// Returns true if extent is a fresh extent
+ bool is_initial_pending() const {
+ return state == extent_state_t::INITIAL_WRITE_PENDING;
+ }
+
+ /// Returns true if extent is clean (does not have deltas on disk)
+ bool is_clean() const {
+ ceph_assert(is_valid());
+ return state == extent_state_t::INITIAL_WRITE_PENDING ||
+ state == extent_state_t::CLEAN;
+ }
+
+ /// Returns true if extent is dirty (has deltas on disk)
+ bool is_dirty() const {
+ ceph_assert(is_valid());
+ return !is_clean();
+ }
+
+ /// Returns true if extent has not been superceded or retired
+ bool is_valid() const {
+ return state != extent_state_t::INVALID;
+ }
+
+ /**
+ * get_paddr
+ *
+ * Returns current address of extent. If is_initial_pending(), address will
+ * be relative, otherwise address will be absolute.
+ */
+ paddr_t get_paddr() const { return poffset; }
+
+ /// Returns length of extent
+ extent_len_t get_length() const { return ptr.length(); }
+
+ /// Returns version, get_version() == 0 iff is_clean()
+ extent_version_t get_version() const {
+ return version;
+ }
+
+ /// Get ref to raw buffer
+ bufferptr &get_bptr() { return ptr; }
+ const bufferptr &get_bptr() const { return ptr; }
+
+ /// Compare by paddr
+ friend bool operator< (const CachedExtent &a, const CachedExtent &b) {
+ return a.poffset < b.poffset;
+ }
+ friend bool operator> (const CachedExtent &a, const CachedExtent &b) {
+ return a.poffset > b.poffset;
+ }
+ friend bool operator== (const CachedExtent &a, const CachedExtent &b) {
+ return a.poffset == b.poffset;
+ }
+
+ virtual ~CachedExtent();
+
+private:
+ friend struct paddr_cmp;
+ friend struct ref_paddr_cmp;
+ friend class ExtentIndex;
+
+ /// Pointer to containing index (or null)
+ ExtentIndex *parent_index = nullptr;
+
+ /// hook for intrusive extent_index
+ boost::intrusive::set_member_hook<> extent_index_hook;
+ using index_member_options = boost::intrusive::member_hook<
+ CachedExtent,
+ boost::intrusive::set_member_hook<>,
+ &CachedExtent::extent_index_hook>;
+ using index = boost::intrusive::set<CachedExtent, index_member_options>;
+ friend class ExtentIndex;
+ friend class Transaction;
+
+ /// hook for intrusive ref list (mainly dirty or lru list)
+ boost::intrusive::list_member_hook<> primary_ref_list_hook;
+ using primary_ref_list_member_options = boost::intrusive::member_hook<
+ CachedExtent,
+ boost::intrusive::list_member_hook<>,
+ &CachedExtent::primary_ref_list_hook>;
+ using list = boost::intrusive::list<
+ CachedExtent,
+ primary_ref_list_member_options>;
+
+ /// Actual data contents
+ ceph::bufferptr ptr;
+
+ /// number of deltas since initial write
+ extent_version_t version = EXTENT_VERSION_NULL;
+
+ /// address of original block -- relative iff is_pending() and is_clean()
+ paddr_t poffset;
+
+ /// used to wait while in-progress commit completes
+ std::optional<seastar::shared_promise<>> io_wait_promise;
+ void set_io_wait() {
+ ceph_assert(!io_wait_promise);
+ io_wait_promise = seastar::shared_promise<>();
+ }
+ void complete_io() {
+ ceph_assert(io_wait_promise);
+ io_wait_promise->set_value();
+ io_wait_promise = std::nullopt;
+ }
+ seastar::future<> wait_io() {
+ if (!io_wait_promise) {
+ return seastar::now();
+ } else {
+ return io_wait_promise->get_shared_future();
+ }
+ }
+
+protected:
+ CachedExtent(ceph::bufferptr &&ptr) : ptr(std::move(ptr)) {}
+ CachedExtent(const CachedExtent &other)
+ : state(other.state),
+ ptr(other.ptr.c_str(), other.ptr.length()),
+ version(other.version),
+ poffset(other.poffset) {}
+
+ friend class Cache;
+ template <typename T, typename... Args>
+ static TCachedExtentRef<T> make_cached_extent_ref(Args&&... args) {
+ return new T(std::forward<Args>(args)...);
+ }
+
+ void set_paddr(paddr_t offset) { poffset = offset; }
+
+ /**
+ * maybe_generate_relative
+ *
+ * There are three kinds of addresses one might want to
+ * store within an extent:
+ * - addr for a block within the same transaction relative to the
+ * physical location of this extent in the
+ * event that we will read it in the initial read of the extent
+ * - addr relative to the physical location of the next record to a
+ * block within that record to contain a delta for this extent in
+ * the event that we'll read it from a delta and overlay it onto a
+ * dirty representation of the extent.
+ * - absolute addr to a block already written outside of the current
+ * transaction.
+ *
+ * This helper checks addr and the current state to create the correct
+ * reference.
+ */
+ paddr_t maybe_generate_relative(paddr_t addr) {
+ if (!addr.is_relative()) {
+ return addr;
+ } else if (is_mutation_pending()) {
+ return addr;
+ } else {
+ ceph_assert(get_paddr().is_relative());
+ return addr - get_paddr();
+ }
+ }
+
+};
+
+std::ostream &operator<<(std::ostream &, CachedExtent::extent_state_t);
+std::ostream &operator<<(std::ostream &, const CachedExtent&);
+
+/// Compare extents by paddr
+struct paddr_cmp {
+ bool operator()(paddr_t lhs, const CachedExtent &rhs) const {
+ return lhs < rhs.poffset;
+ }
+ bool operator()(const CachedExtent &lhs, paddr_t rhs) const {
+ return lhs.poffset < rhs;
+ }
+};
+
+/// Compare extent refs by paddr
+struct ref_paddr_cmp {
+ using is_transparent = paddr_t;
+ bool operator()(const CachedExtentRef &lhs, const CachedExtentRef &rhs) const {
+ return lhs->poffset < rhs->poffset;
+ }
+ bool operator()(const paddr_t &lhs, const CachedExtentRef &rhs) const {
+ return lhs < rhs->poffset;
+ }
+ bool operator()(const CachedExtentRef &lhs, const paddr_t &rhs) const {
+ return lhs->poffset < rhs;
+ }
+};
+
+template <typename T, typename C>
+class addr_extent_list_base_t
+ : public std::list<std::pair<T, C>> {};
+
+using pextent_list_t = addr_extent_list_base_t<paddr_t, CachedExtentRef>;
+
+template <typename T, typename C, typename Cmp>
+class addr_extent_set_base_t
+ : public std::set<C, Cmp> {};
+
+using pextent_set_t = addr_extent_set_base_t<
+ paddr_t,
+ CachedExtentRef,
+ ref_paddr_cmp
+ >;
+
+template <typename T>
+using t_pextent_list_t = addr_extent_list_base_t<paddr_t, TCachedExtentRef<T>>;
+
+/**
+ * ExtentIndex
+ *
+ * Index of CachedExtent & by poffset, does not hold a reference,
+ * user must ensure each extent is removed prior to deletion
+ */
+class ExtentIndex {
+ friend class Cache;
+ CachedExtent::index extent_index;
+public:
+ auto get_overlap(paddr_t addr, segment_off_t len) {
+ auto bottom = extent_index.upper_bound(addr, paddr_cmp());
+ if (bottom != extent_index.begin())
+ --bottom;
+ if (bottom != extent_index.end() &&
+ bottom->get_paddr().add_offset(bottom->get_length()) <= addr)
+ ++bottom;
+
+ auto top = extent_index.upper_bound(addr.add_offset(len), paddr_cmp());
+ return std::make_pair(
+ bottom,
+ top
+ );
+ }
+
+ void clear() {
+ extent_index.clear();
+ }
+
+ void insert(CachedExtent &extent) {
+ // sanity check
+ auto [a, b] = get_overlap(
+ extent.get_paddr(),
+ extent.get_length());
+ ceph_assert(a == b);
+
+ extent_index.insert(extent);
+ extent.parent_index = this;
+ }
+
+ void erase(CachedExtent &extent) {
+ extent_index.erase(extent);
+ extent.parent_index = nullptr;
+ }
+
+ bool empty() const {
+ return extent_index.empty();
+ }
+
+ auto find_offset(paddr_t offset) {
+ return extent_index.find(offset, paddr_cmp());
+ }
+
+ auto begin() {
+ return extent_index.begin();
+ }
+
+ auto end() {
+ return extent_index.end();
+ }
+
+ void merge(ExtentIndex &&other) {
+ for (auto it = other.extent_index.begin();
+ it != other.extent_index.end();
+ ) {
+ auto &ext = *it;
+ ++it;
+ other.extent_index.erase(ext);
+ extent_index.insert(ext);
+ }
+ }
+
+ template <typename T>
+ void remove(T &l) {
+ for (auto &ext : l) {
+ extent_index.erase(l);
+ }
+ }
+};
+
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "crimson/os/seastore/root_block.h"
+
+namespace crimson::os::seastore {
+
+void RootBlock::prepare_write()
+{
+ bufferlist tmp;
+ ::encode(root, tmp);
+ auto bpiter = tmp.begin();
+ bpiter.copy(tmp.length(), get_bptr().c_str());
+}
+
+CachedExtent::complete_load_ertr::future<> RootBlock::complete_load()
+{
+ auto biter = get_bptr().cbegin();
+ root.decode(biter);
+ return complete_load_ertr::now();
+}
+
+void RootBlock::set_lba_root(btree_lba_root_t lba_root)
+{
+ root.lba_root = lba_root;
+}
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include "crimson/os/seastore/cached_extent.h"
+
+namespace crimson::os::seastore {
+
+using depth_t = uint32_t;
+
+/* Belongs in lba_manager/btree, TODO: generalize this to
+ * permit more than one lba_manager implementation
+ */
+struct btree_lba_root_t {
+ depth_t lba_depth;
+ depth_t segment_depth;
+ paddr_t lba_root_addr;
+ paddr_t segment_root;
+
+ DENC(btree_lba_root_t, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.lba_depth, p);
+ denc(v.segment_depth, p);
+ denc(v.lba_root_addr, p);
+ denc(v.segment_root, p);
+ DENC_FINISH(p);
+ }
+};
+
+struct root_block_t {
+ btree_lba_root_t lba_root;
+
+ DENC(root_block_t, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.lba_root, p);
+ DENC_FINISH(p);
+ }
+};
+
+struct RootBlock : CachedExtent {
+ constexpr static segment_off_t SIZE = 4<<10;
+ using Ref = TCachedExtentRef<RootBlock>;
+
+ root_block_t root;
+
+ template <typename... T>
+ RootBlock(T&&... t) : CachedExtent(std::forward<T>(t)...) {}
+
+ CachedExtentRef duplicate_for_write() final {
+ return CachedExtentRef(new RootBlock(*this));
+ };
+
+ void prepare_write() final;
+
+ static constexpr extent_types_t TYPE = extent_types_t::ROOT;
+ extent_types_t get_type() const final {
+ return extent_types_t::ROOT;
+ }
+
+ ceph::bufferlist get_delta() final {
+ ceph_assert(0 == "TODO");
+ return ceph::bufferlist();
+ }
+
+ void apply_delta(paddr_t base, ceph::bufferlist &bl) final {
+ ceph_assert(0 == "TODO");
+ }
+
+ complete_load_ertr::future<> complete_load() final;
+
+ void set_lba_root(btree_lba_root_t lba_root);
+ btree_lba_root_t &get_lba_root() {
+ return root.lba_root;
+ }
+
+};
+using RootBlockRef = RootBlock::Ref;
+
+}
+
+WRITE_CLASS_DENC(crimson::os::seastore::btree_lba_root_t)
+WRITE_CLASS_DENC(crimson::os::seastore::root_block_t)
return segment == REL_SEG_ID;
}
+ paddr_t add_offset(segment_off_t o) const {
+ return paddr_t{segment, offset + o};
+ }
+
paddr_t add_relative(paddr_t o) const {
assert(o.is_relative());
assert(!is_relative());
constexpr laddr_t L_ADDR_LBAT = std::numeric_limits<laddr_t>::max() - 2;
// logical offset, see LBAManager, TransactionManager
-using loff_t = uint32_t;
-constexpr loff_t L_OFF_NULL = std::numeric_limits<loff_t>::max();
+using extent_len_t = uint32_t;
+constexpr extent_len_t EXTENT_LEN_MAX =
+ std::numeric_limits<extent_len_t>::max();
-struct laddr_list_t : std::list<std::pair<laddr_t, loff_t>> {
+struct laddr_list_t : std::list<std::pair<laddr_t, extent_len_t>> {
template <typename... T>
laddr_list_t(T&&... args)
- : std::list<std::pair<laddr_t, loff_t>>(std::forward<T>(args)...) {}
+ : std::list<std::pair<laddr_t, extent_len_t>>(std::forward<T>(args)...) {}
};
-struct paddr_list_t : std::list<std::pair<paddr_t, loff_t>> {
+struct paddr_list_t : std::list<std::pair<paddr_t, extent_len_t>> {
template <typename... T>
paddr_list_t(T&&... args)
- : std::list<std::pair<paddr_t, loff_t>>(std::forward<T>(args)...) {}
+ : std::list<std::pair<paddr_t, extent_len_t>>(std::forward<T>(args)...) {}
};
std::ostream &operator<<(std::ostream &out, const laddr_list_t &rhs);
unittest_seastore_journal
${CMAKE_DL_LIBS}
crimson-seastore)
+
+add_executable(unittest_seastore_cache
+ test_seastore_cache.cc
+ ../gtest_seastar.cc)
+add_ceph_unittest(unittest_seastore_cache)
+target_link_libraries(
+ unittest_seastore_cache
+ ${CMAKE_DL_LIBS}
+ crimson-seastore)
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "test/crimson/gtest_seastar.h"
+
+#include "crimson/common/log.h"
+#include "crimson/os/seastore/cache.h"
+#include "crimson/os/seastore/segment_manager/ephemeral.h"
+
+using namespace crimson;
+using namespace crimson::os;
+using namespace crimson::os::seastore;
+
+namespace {
+ [[maybe_unused]] seastar::logger& logger() {
+ return crimson::get_logger(ceph_subsys_test);
+ }
+}
+
+struct CacheTestBlock : CachedExtent {
+ constexpr static segment_off_t SIZE = 4<<10;
+ using Ref = TCachedExtentRef<CacheTestBlock>;
+
+ template <typename... T>
+ CacheTestBlock(T&&... t) : CachedExtent(std::forward<T>(t)...) {}
+
+ CachedExtentRef duplicate_for_write() final {
+ return CachedExtentRef(new CacheTestBlock(*this));
+ };
+
+ static constexpr extent_types_t TYPE = extent_types_t::TEST_BLOCK;
+ extent_types_t get_type() const final {
+ return TYPE;
+ }
+
+ ceph::bufferlist get_delta() final {
+ return ceph::bufferlist();
+ }
+
+ void apply_delta(paddr_t delta_base, ceph::bufferlist &bl) final {
+ ceph_assert(0 == "TODO");
+ }
+
+ void set_contents(char c) {
+ ::memset(get_bptr().c_str(), c, get_length());
+ }
+
+ int checksum() {
+ return ceph_crc32c(
+ 1,
+ (const unsigned char *)get_bptr().c_str(),
+ get_length());
+ }
+};
+
+struct cache_test_t : public seastar_test_suite_t {
+ segment_manager::EphemeralSegmentManager segment_manager;
+ Cache cache;
+ paddr_t current{0, 0};
+
+ cache_test_t()
+ : segment_manager(segment_manager::DEFAULT_TEST_EPHEMERAL),
+ cache(segment_manager) {}
+
+ seastar::future<std::optional<paddr_t>> submit_transaction(
+ TransactionRef t) {
+ auto record = cache.try_construct_record(*t);
+ if (!record) {
+ return seastar::make_ready_future<std::optional<paddr_t>>(
+ std::nullopt);
+ }
+
+ bufferlist bl;
+ for (auto &&block : record->extents) {
+ bl.append(block.bl);
+ }
+
+ ceph_assert((segment_off_t)bl.length() <
+ segment_manager.get_segment_size());
+ if (current.offset + (segment_off_t)bl.length() >
+ segment_manager.get_segment_size())
+ current = paddr_t{current.segment + 1, 0};
+
+ auto prev = current;
+ current.offset += bl.length();
+ return segment_manager.segment_write(
+ prev,
+ std::move(bl),
+ true
+ ).safe_then(
+ [this, prev, t=std::move(t)] {
+ cache.complete_commit(*t, prev);
+ return seastar::make_ready_future<std::optional<paddr_t>>(prev);
+ },
+ crimson::ct_error::all_same_way([](auto e) {
+ ASSERT_FALSE("failed to submit");
+ })
+ );
+ }
+
+ auto get_transaction() {
+ return TransactionRef(new Transaction);
+ }
+
+ seastar::future<> set_up_fut() final {
+ return segment_manager.init().safe_then(
+ [this] {
+ return seastar::do_with(
+ TransactionRef(new Transaction()),
+ [this](auto &transaction) {
+ return cache.mkfs(*transaction).safe_then(
+ [this, &transaction] {
+ return submit_transaction(std::move(transaction)).then(
+ [](auto p) {
+ ASSERT_TRUE(p);
+ });
+ });
+ });
+ }).handle_error(
+ crimson::ct_error::all_same_way([](auto e) {
+ ASSERT_FALSE("failed to submit");
+ })
+ );
+ }
+
+ seastar::future<> tear_down_fut() final {
+ return seastar::now();
+ }
+};
+
+TEST_F(cache_test_t, test_addr_fixup)
+{
+ run_async([this] {
+ paddr_t addr;
+ int csum = 0;
+ {
+ auto t = get_transaction();
+ auto extent = cache.alloc_new_extent<CacheTestBlock>(
+ *t,
+ CacheTestBlock::SIZE);
+ extent->set_contents('c');
+ csum = extent->checksum();
+ auto ret = submit_transaction(std::move(t)).get0();
+ ASSERT_TRUE(ret);
+ addr = extent->get_paddr();
+ }
+ {
+ auto t = get_transaction();
+ auto extent = cache.get_extent<CacheTestBlock>(
+ *t,
+ addr,
+ CacheTestBlock::SIZE).unsafe_get0();
+ ASSERT_EQ(extent->get_paddr(), addr);
+ ASSERT_EQ(extent->checksum(), csum);
+ }
+ });
+}
+
+TEST_F(cache_test_t, test_dirty_extent)
+{
+ run_async([this] {
+ paddr_t addr;
+ int csum = 0;
+ int csum2 = 0;
+ {
+ // write out initial test block
+ auto t = get_transaction();
+ auto extent = cache.alloc_new_extent<CacheTestBlock>(
+ *t,
+ CacheTestBlock::SIZE);
+ extent->set_contents('c');
+ csum = extent->checksum();
+ auto reladdr = extent->get_paddr();
+ ASSERT_TRUE(reladdr.is_relative());
+ {
+ // test that read with same transaction sees new block though
+ // uncommitted
+ auto extent = cache.get_extent<CacheTestBlock>(
+ *t,
+ reladdr,
+ CacheTestBlock::SIZE).unsafe_get0();
+ ASSERT_TRUE(extent->is_clean());
+ ASSERT_TRUE(extent->is_pending());
+ ASSERT_TRUE(extent->get_paddr().is_relative());
+ ASSERT_EQ(extent->get_version(), 0);
+ ASSERT_EQ(csum, extent->checksum());
+ }
+ auto ret = submit_transaction(std::move(t)).get0();
+ ASSERT_TRUE(ret);
+ addr = extent->get_paddr();
+ }
+ {
+ // read back test block
+ auto t = get_transaction();
+ auto extent = cache.get_extent<CacheTestBlock>(
+ *t,
+ addr,
+ CacheTestBlock::SIZE).unsafe_get0();
+ // duplicate and reset contents
+ extent = cache.duplicate_for_write(*t, extent)->cast<CacheTestBlock>();
+ extent->set_contents('c');
+ csum2 = extent->checksum();
+ ASSERT_EQ(extent->get_paddr(), addr);
+ {
+ // test that concurrent read with fresh transaction sees old
+ // block
+ auto t2 = get_transaction();
+ auto extent = cache.get_extent<CacheTestBlock>(
+ *t2,
+ addr,
+ CacheTestBlock::SIZE).unsafe_get0();
+ ASSERT_TRUE(extent->is_clean());
+ ASSERT_FALSE(extent->is_pending());
+ ASSERT_EQ(addr, extent->get_paddr());
+ ASSERT_EQ(extent->get_version(), 0);
+ ASSERT_EQ(csum, extent->checksum());
+ }
+ {
+ // test that read with same transaction sees new block
+ auto extent = cache.get_extent<CacheTestBlock>(
+ *t,
+ addr,
+ CacheTestBlock::SIZE).unsafe_get0();
+ ASSERT_TRUE(extent->is_dirty());
+ ASSERT_TRUE(extent->is_pending());
+ ASSERT_EQ(addr, extent->get_paddr());
+ ASSERT_EQ(extent->get_version(), 1);
+ ASSERT_EQ(csum2, extent->checksum());
+ }
+ // submit transaction
+ auto ret = submit_transaction(std::move(t)).get0();
+ ASSERT_TRUE(ret);
+ ASSERT_TRUE(extent->is_dirty());
+ ASSERT_EQ(addr, extent->get_paddr());
+ ASSERT_EQ(extent->get_version(), 1);
+ ASSERT_EQ(extent->checksum(), csum2);
+ }
+ {
+ // test that fresh transaction now sees newly dirty block
+ auto t = get_transaction();
+ auto extent = cache.get_extent<CacheTestBlock>(
+ *t,
+ addr,
+ CacheTestBlock::SIZE).unsafe_get0();
+ ASSERT_TRUE(extent->is_dirty());
+ ASSERT_EQ(addr, extent->get_paddr());
+ ASSERT_EQ(extent->get_version(), 1);
+ ASSERT_EQ(csum2, extent->checksum());
+ }
+ });
+}